From c7913dc089980cf211bb00639792d9457918a0ce Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Mon, 28 Oct 2024 15:15:10 -0700 Subject: [PATCH 01/41] 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 2f686a3a..c60c5b9d 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 00000000..b000cb83 --- /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 78a3c74a..5cb21a92 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 93d80858..d62503c4 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/41] 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 b000cb83..bd407ffb 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 d62503c4..6aa8f20b 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/41] 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 bd407ffb..0d16f2fb 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 d7aba989..328c0552 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/41] 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 328c0552..d7aba989 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/41] 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 0d16f2fb..b473f660 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/41] 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 12afa051..310258c3 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 ec757d48..4240e49d 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/41] 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 310258c3..12afa051 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/41] 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 4240e49d..65ffe391 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/41] 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 b473f660..df65ee29 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 12afa051..c1edf8ac 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/41] 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 df65ee29..a8f9dd21 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/41] 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 a8f9dd21..56cc8610 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/41] 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 65ffe391..6aa8f20b 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/41] 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 56cc8610..5c6cd375 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/41] 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 c1edf8ac..bcbef1f5 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/41] 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 5c6cd375..74d3d9d8 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/41] 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 74d3d9d8..9183593e 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 6aa8f20b..1bf55fc5 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/41] 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 9183593e..b7ef223e 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 1bf55fc5..999d9941 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/41] 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 999d9941..43276ef9 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/41] 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 b7ef223e..f2fd088e 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 43276ef9..baf6045c 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 00000000..7ac51f58 --- /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/41] 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 f2fd088e..d81435e2 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/41] 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 d81435e2..d0dd00f4 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/41] 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 00000000..55c765aa --- /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 00000000..debf9cf6 --- /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 00000000..0de5e6f7 --- /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 00000000..6bd7f6f7 --- /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 00000000..2f522900 --- /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/41] 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 7ac51f58..00000000 --- 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 55c765aa..00000000 --- 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 debf9cf6..00000000 --- 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 0de5e6f7..00000000 --- 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 6bd7f6f7..00000000 --- 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 2f522900..00000000 --- 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 ea7029aa569bd5207b6e94ed551c8cd695c5c243 Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Fri, 1 Nov 2024 14:07:28 -0700 Subject: [PATCH 24/41] Fix failing tests --- kernel/src/table_changes.rs | 41 +++++++++++++++++++++++++++++++++++++ 1 file changed, 41 insertions(+) create mode 100644 kernel/src/table_changes.rs diff --git a/kernel/src/table_changes.rs b/kernel/src/table_changes.rs new file mode 100644 index 00000000..7ac51f58 --- /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 9464d4e0a9e87a869580252fbb24ad37cfdce277 Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Fri, 1 Nov 2024 14:44:30 -0700 Subject: [PATCH 25/41] Basic Table Changes construction --- kernel/src/error.rs | 5 ++++- kernel/src/lib.rs | 1 + kernel/src/table.rs | 12 ++++++++++++ kernel/src/table_changes.rs | 29 ++++++++++++++++++++++++++--- 4 files changed, 43 insertions(+), 4 deletions(-) diff --git a/kernel/src/error.rs b/kernel/src/error.rs index 6396f5b8..25ca5ba1 100644 --- a/kernel/src/error.rs +++ b/kernel/src/error.rs @@ -6,7 +6,7 @@ use std::{ str::Utf8Error, }; -use crate::schema::DataType; +use crate::{schema::DataType, Version}; /// A [`std::result::Result`] that has the kernel [`Error`] as the error variant pub type DeltaResult = std::result::Result; @@ -171,6 +171,9 @@ pub enum Error { /// The file already exists at the path, prohibiting a non-overwrite write #[error("File already exists: {0}")] FileAlreadyExists(String), + + #[error("Change data feed is not enabled for version range {0} to {}", _1.map_or("end".to_string(), |x| x.to_string()))] + InvalidChangeDataFeedRange(Version, Option), } // Convenience constructors for Error types that take a String argument diff --git a/kernel/src/lib.rs b/kernel/src/lib.rs index c60c5b9d..ec4767f3 100644 --- a/kernel/src/lib.rs +++ b/kernel/src/lib.rs @@ -74,6 +74,7 @@ pub mod scan; pub mod schema; pub mod snapshot; pub mod table; +pub mod table_changes; pub mod transaction; pub(crate) mod utils; diff --git a/kernel/src/table.rs b/kernel/src/table.rs index d7aba989..73baba37 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,17 @@ impl Table { Snapshot::try_new(self.location.clone(), engine, version) } + /// Create a [`TableChanges`] representing a Change Data Feed scan from `start_version` to + /// `end_version`. If no `end_version` is specified, we use the latest version. + 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)?)) diff --git a/kernel/src/table_changes.rs b/kernel/src/table_changes.rs index 7ac51f58..0a8e9623 100644 --- a/kernel/src/table_changes.rs +++ b/kernel/src/table_changes.rs @@ -7,9 +7,11 @@ use crate::{ path::AsUrl, schema::Schema, snapshot::Snapshot, - DeltaResult, Engine, Version, + DeltaResult, Engine, Error, Version, }; +static CDF_ENABLE_FLAG: &'static str = "delta.enableChangeDataFeed"; + #[derive(Debug)] pub struct TableChanges { snapshot: Snapshot, @@ -25,16 +27,37 @@ impl TableChanges { 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 start_snapshot = + Snapshot::try_new(table_root.as_url().clone(), engine, Some(start_version))?; + + // Get the end snapshot to fetch schema + let end_snapshot = Snapshot::try_new(table_root.as_url().clone(), engine, end_version)?; + let schema = end_snapshot.schema(); + 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); } + builder = builder.set_omit_checkpoints(); let log_segment = builder.build()?; + match ( + start_snapshot.metadata().configuration.get(CDF_ENABLE_FLAG), + end_snapshot.metadata().configuration.get(CDF_ENABLE_FLAG), + ) { + (Some(start_flag), Some(end_flag)) if start_flag == "true" && end_flag == "true" => {} + _ => { + return Err(Error::InvalidChangeDataFeedRange( + start_version, + end_version, + )) + } + } + Ok(TableChanges { - snapshot, + schema: schema.clone(), + snapshot: start_snapshot, cdf_range: log_segment, }) } From f3204166a21f6d839c6813f0426081d4de4831e4 Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Fri, 1 Nov 2024 14:59:44 -0700 Subject: [PATCH 26/41] Add beginning of the Scannable trait --- kernel/src/scan/mod.rs | 72 +++++++++++++++++++++++++++++++------ kernel/src/snapshot.rs | 6 ++-- kernel/src/table_changes.rs | 11 +++++- 3 files changed, 75 insertions(+), 14 deletions(-) diff --git a/kernel/src/scan/mod.rs b/kernel/src/scan/mod.rs index bcbef1f5..07ed5391 100644 --- a/kernel/src/scan/mod.rs +++ b/kernel/src/scan/mod.rs @@ -8,12 +8,14 @@ use tracing::debug; 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::actions::{get_log_add_schema, get_log_schema, Metadata, ADD_NAME, REMOVE_NAME}; use crate::expressions::{ColumnName, Expression, ExpressionRef, Scalar}; use crate::features::ColumnMappingMode; +use crate::log_segment::LogSegment; use crate::scan::state::{DvInfo, Stats}; use crate::schema::{DataType, Schema, SchemaRef, StructField, StructType}; use crate::snapshot::Snapshot; +use crate::table_changes::TableChanges; use crate::{DeltaResult, Engine, EngineData, Error, FileMeta}; use self::log_replay::scan_action_iter; @@ -23,9 +25,59 @@ mod data_skipping; pub mod log_replay; pub mod state; +pub trait Scannable { + fn schema(&self) -> SchemaRef; + fn metadata(&self) -> &Metadata; + fn column_mapping_mode(&self) -> &ColumnMappingMode; + fn log_segment(&self) -> &LogSegment; + fn table_root(&self) -> &Url; +} +impl Scannable for Snapshot { + fn schema(&self) -> SchemaRef { + todo!() + } + + fn metadata(&self) -> &Metadata { + todo!() + } + + fn column_mapping_mode(&self) -> &ColumnMappingMode { + todo!() + } + + fn log_segment(&self) -> &LogSegment { + todo!() + } + + fn table_root(&self) -> &Url { + todo!() + } +} +impl Scannable for TableChanges { + fn schema(&self) -> SchemaRef { + todo!() + } + + fn metadata(&self) -> &Metadata { + todo!() + } + + fn column_mapping_mode(&self) -> &ColumnMappingMode { + todo!() + } + + fn log_segment(&self) -> &LogSegment { + todo!() + } + + fn table_root(&self) -> &Url { + todo!() + } +} + /// Builder to scan a snapshot of a table. pub struct ScanBuilder { - snapshot: Arc, + snapshot: Arc, schema: Option, predicate: Option, } @@ -41,7 +93,7 @@ impl std::fmt::Debug for ScanBuilder { impl ScanBuilder { /// Create a new [`ScanBuilder`] instance. - pub fn new(snapshot: impl Into>) -> Self { + pub fn new(snapshot: impl Into>) -> Self { Self { snapshot: snapshot.into(), schema: None, @@ -95,7 +147,7 @@ impl ScanBuilder { let (all_fields, read_fields, have_partition_cols) = get_state_info( logical_schema.as_ref(), &self.snapshot.metadata().partition_columns, - self.snapshot.column_mapping_mode, + *self.snapshot.column_mapping_mode(), )?; let physical_schema = Arc::new(StructType::new(read_fields)); Ok(Scan { @@ -170,7 +222,7 @@ pub type ScanData = (Box, Vec); /// The result of building a scan over a table. This can be used to get the actual data from /// scanning the table. pub struct Scan { - snapshot: Arc, + snapshot: Arc, logical_schema: SchemaRef, physical_schema: SchemaRef, predicate: Option, @@ -235,7 +287,7 @@ impl Scan { // 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.snapshot - .log_segment + .log_segment() .replay(engine, commit_read_schema, checkpoint_read_schema, None) } @@ -243,11 +295,11 @@ impl Scan { /// only be called once per scan. pub fn global_scan_state(&self) -> GlobalScanState { GlobalScanState { - table_root: self.snapshot.table_root.to_string(), + table_root: self.snapshot.table_root().to_string(), partition_columns: self.snapshot.metadata().partition_columns.clone(), logical_schema: self.logical_schema.clone(), read_schema: self.physical_schema.clone(), - column_mapping_mode: self.snapshot.column_mapping_mode, + column_mapping_mode: *self.snapshot.column_mapping_mode(), } } @@ -304,10 +356,10 @@ impl Scan { let result = scan_files_iter .map(move |scan_file| -> DeltaResult<_> { let scan_file = scan_file?; - let file_path = self.snapshot.table_root.join(&scan_file.path)?; + let file_path = self.snapshot.table_root().join(&scan_file.path)?; let mut selection_vector = scan_file .dv_info - .get_selection_vector(engine, &self.snapshot.table_root)?; + .get_selection_vector(engine, self.snapshot.table_root())?; let meta = FileMeta { last_modified: 0, size: scan_file.size as usize, diff --git a/kernel/src/snapshot.rs b/kernel/src/snapshot.rs index baf6045c..4785e16b 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, LogSegmentBuilder}; -use crate::scan::ScanBuilder; +use crate::scan::{ScanBuilder, Scannable}; use crate::schema::Schema; use crate::{DeltaResult, Engine, Error, FileSystemClient, Version}; @@ -140,12 +140,12 @@ impl Snapshot { /// Create a [`ScanBuilder`] for an `Arc`. pub fn scan_builder(self: Arc) -> ScanBuilder { - ScanBuilder::new(self) + ScanBuilder::new(self as Arc) } /// Consume this `Snapshot` to create a [`ScanBuilder`] pub fn into_scan_builder(self) -> ScanBuilder { - ScanBuilder::new(self) + ScanBuilder::new(Arc::new(self) as Arc) } } diff --git a/kernel/src/table_changes.rs b/kernel/src/table_changes.rs index 0a8e9623..6cf08703 100644 --- a/kernel/src/table_changes.rs +++ b/kernel/src/table_changes.rs @@ -1,22 +1,27 @@ //! In-memory representation of a change data feed table. +use std::sync::Arc; + use url::Url; use crate::{ + features::ColumnMappingMode, log_segment::{LogSegment, LogSegmentBuilder}, path::AsUrl, + scan::{ScanBuilder, Scannable}, schema::Schema, snapshot::Snapshot, DeltaResult, Engine, Error, Version, }; -static CDF_ENABLE_FLAG: &'static str = "delta.enableChangeDataFeed"; +static CDF_ENABLE_FLAG: &str = "delta.enableChangeDataFeed"; #[derive(Debug)] pub struct TableChanges { snapshot: Snapshot, cdf_range: LogSegment, schema: Schema, + column_mapping_mode: ColumnMappingMode, } impl TableChanges { @@ -59,6 +64,10 @@ impl TableChanges { schema: schema.clone(), snapshot: start_snapshot, cdf_range: log_segment, + column_mapping_mode: end_snapshot.column_mapping_mode, }) } + pub fn into_scan_builder(self) -> ScanBuilder { + ScanBuilder::new(Arc::new(self) as Arc) + } } From ecd6cd73cc5b51dd7fa0d27a825143adef715999 Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Mon, 4 Nov 2024 09:02:35 -0800 Subject: [PATCH 27/41] table_changes constructor --- kernel/examples/change_data/Cargo.toml | 17 ++++++++++++++++ kernel/examples/change_data/src/main.rs | 23 +++++++++++++++++++++ kernel/src/error.rs | 5 +---- kernel/src/lib.rs | 1 - kernel/src/scan/table_change_scan.rs | 1 + kernel/src/table.rs | 12 ----------- kernel/src/table_changes.rs | 27 +++---------------------- 7 files changed, 45 insertions(+), 41 deletions(-) create mode 100644 kernel/examples/change_data/Cargo.toml create mode 100644 kernel/examples/change_data/src/main.rs create mode 100644 kernel/src/scan/table_change_scan.rs diff --git a/kernel/examples/change_data/Cargo.toml b/kernel/examples/change_data/Cargo.toml new file mode 100644 index 00000000..6265eb13 --- /dev/null +++ b/kernel/examples/change_data/Cargo.toml @@ -0,0 +1,17 @@ +[package] +name = "change_data" +version = "0.1.0" +edition = "2021" +publish = false + +[dependencies] +arrow-array = { workspace = true } +arrow-schema = { workspace = true } +clap = { version = "4.5", features = ["derive"] } +delta_kernel = { path = "../../../kernel", features = [ + "cloud", + "default-engine", + "developer-visibility", +] } +env_logger = "0.11.3" +url = "2" diff --git a/kernel/examples/change_data/src/main.rs b/kernel/examples/change_data/src/main.rs new file mode 100644 index 00000000..055f78bd --- /dev/null +++ b/kernel/examples/change_data/src/main.rs @@ -0,0 +1,23 @@ +use std::{collections::HashMap, sync::Arc}; + +use delta_kernel::{ + engine::default::{executor::tokio::TokioBackgroundExecutor, DefaultEngine}, + DeltaResult, Table, +}; + +fn main() -> DeltaResult<()> { + let uri = "/Users/oussama.saoudi/delta-kernel-rs/kernel/tests/data/table-with-dv-small/"; + // build a table and get the lastest snapshot from it + let table = Table::try_from_uri(uri)?; + + let engine = DefaultEngine::try_new( + table.location(), + HashMap::::new(), + Arc::new(TokioBackgroundExecutor::new()), + )?; + + let table_changes = table.table_changes(&engine, 0, None)?; + println!("table_changes: {:?}", table_changes); + + Ok(()) +} diff --git a/kernel/src/error.rs b/kernel/src/error.rs index 25ca5ba1..6396f5b8 100644 --- a/kernel/src/error.rs +++ b/kernel/src/error.rs @@ -6,7 +6,7 @@ use std::{ str::Utf8Error, }; -use crate::{schema::DataType, Version}; +use crate::schema::DataType; /// A [`std::result::Result`] that has the kernel [`Error`] as the error variant pub type DeltaResult = std::result::Result; @@ -171,9 +171,6 @@ pub enum Error { /// The file already exists at the path, prohibiting a non-overwrite write #[error("File already exists: {0}")] FileAlreadyExists(String), - - #[error("Change data feed is not enabled for version range {0} to {}", _1.map_or("end".to_string(), |x| x.to_string()))] - InvalidChangeDataFeedRange(Version, Option), } // Convenience constructors for Error types that take a String argument diff --git a/kernel/src/lib.rs b/kernel/src/lib.rs index ec4767f3..c60c5b9d 100644 --- a/kernel/src/lib.rs +++ b/kernel/src/lib.rs @@ -74,7 +74,6 @@ pub mod scan; pub mod schema; pub mod snapshot; pub mod table; -pub mod table_changes; pub mod transaction; pub(crate) mod utils; diff --git a/kernel/src/scan/table_change_scan.rs b/kernel/src/scan/table_change_scan.rs new file mode 100644 index 00000000..fafcbaf7 --- /dev/null +++ b/kernel/src/scan/table_change_scan.rs @@ -0,0 +1 @@ +pub struct TableChangesScan {} diff --git a/kernel/src/table.rs b/kernel/src/table.rs index 73baba37..d7aba989 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,17 +78,6 @@ impl Table { Snapshot::try_new(self.location.clone(), engine, version) } - /// Create a [`TableChanges`] representing a Change Data Feed scan from `start_version` to - /// `end_version`. If no `end_version` is specified, we use the latest version. - 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)?)) diff --git a/kernel/src/table_changes.rs b/kernel/src/table_changes.rs index 6cf08703..c8822222 100644 --- a/kernel/src/table_changes.rs +++ b/kernel/src/table_changes.rs @@ -11,7 +11,7 @@ use crate::{ scan::{ScanBuilder, Scannable}, schema::Schema, snapshot::Snapshot, - DeltaResult, Engine, Error, Version, + DeltaResult, Engine, Version, }; static CDF_ENABLE_FLAG: &str = "delta.enableChangeDataFeed"; @@ -32,37 +32,16 @@ impl TableChanges { end_version: Option, ) -> DeltaResult { let fs_client = engine.get_file_system_client(); - let start_snapshot = - Snapshot::try_new(table_root.as_url().clone(), engine, Some(start_version))?; - - // Get the end snapshot to fetch schema - let end_snapshot = Snapshot::try_new(table_root.as_url().clone(), engine, end_version)?; - let schema = end_snapshot.schema(); - + 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); } - builder = builder.set_omit_checkpoints(); let log_segment = builder.build()?; - match ( - start_snapshot.metadata().configuration.get(CDF_ENABLE_FLAG), - end_snapshot.metadata().configuration.get(CDF_ENABLE_FLAG), - ) { - (Some(start_flag), Some(end_flag)) if start_flag == "true" && end_flag == "true" => {} - _ => { - return Err(Error::InvalidChangeDataFeedRange( - start_version, - end_version, - )) - } - } - Ok(TableChanges { - schema: schema.clone(), - snapshot: start_snapshot, + snapshot, cdf_range: log_segment, column_mapping_mode: end_snapshot.column_mapping_mode, }) From 585c66339834fac66468b6afa283e4a5401946d7 Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Mon, 4 Nov 2024 09:04:47 -0800 Subject: [PATCH 28/41] Revert "Add beginning of the Scannable trait" This reverts commit 470446b641bbf881ebbb8c702a147bdd852e3fdb. --- kernel/src/scan/mod.rs | 72 ++++++------------------------------- kernel/src/snapshot.rs | 6 ++-- kernel/src/table_changes.rs | 11 +----- 3 files changed, 14 insertions(+), 75 deletions(-) diff --git a/kernel/src/scan/mod.rs b/kernel/src/scan/mod.rs index 07ed5391..bcbef1f5 100644 --- a/kernel/src/scan/mod.rs +++ b/kernel/src/scan/mod.rs @@ -8,14 +8,12 @@ use tracing::debug; use url::Url; use crate::actions::deletion_vector::{split_vector, treemap_to_bools, DeletionVectorDescriptor}; -use crate::actions::{get_log_add_schema, get_log_schema, Metadata, ADD_NAME, REMOVE_NAME}; +use crate::actions::{get_log_add_schema, get_log_schema, ADD_NAME, REMOVE_NAME}; use crate::expressions::{ColumnName, Expression, ExpressionRef, Scalar}; use crate::features::ColumnMappingMode; -use crate::log_segment::LogSegment; use crate::scan::state::{DvInfo, Stats}; use crate::schema::{DataType, Schema, SchemaRef, StructField, StructType}; use crate::snapshot::Snapshot; -use crate::table_changes::TableChanges; use crate::{DeltaResult, Engine, EngineData, Error, FileMeta}; use self::log_replay::scan_action_iter; @@ -25,59 +23,9 @@ mod data_skipping; pub mod log_replay; pub mod state; -pub trait Scannable { - fn schema(&self) -> SchemaRef; - fn metadata(&self) -> &Metadata; - fn column_mapping_mode(&self) -> &ColumnMappingMode; - fn log_segment(&self) -> &LogSegment; - fn table_root(&self) -> &Url; -} -impl Scannable for Snapshot { - fn schema(&self) -> SchemaRef { - todo!() - } - - fn metadata(&self) -> &Metadata { - todo!() - } - - fn column_mapping_mode(&self) -> &ColumnMappingMode { - todo!() - } - - fn log_segment(&self) -> &LogSegment { - todo!() - } - - fn table_root(&self) -> &Url { - todo!() - } -} -impl Scannable for TableChanges { - fn schema(&self) -> SchemaRef { - todo!() - } - - fn metadata(&self) -> &Metadata { - todo!() - } - - fn column_mapping_mode(&self) -> &ColumnMappingMode { - todo!() - } - - fn log_segment(&self) -> &LogSegment { - todo!() - } - - fn table_root(&self) -> &Url { - todo!() - } -} - /// Builder to scan a snapshot of a table. pub struct ScanBuilder { - snapshot: Arc, + snapshot: Arc, schema: Option, predicate: Option, } @@ -93,7 +41,7 @@ impl std::fmt::Debug for ScanBuilder { impl ScanBuilder { /// Create a new [`ScanBuilder`] instance. - pub fn new(snapshot: impl Into>) -> Self { + pub fn new(snapshot: impl Into>) -> Self { Self { snapshot: snapshot.into(), schema: None, @@ -147,7 +95,7 @@ impl ScanBuilder { let (all_fields, read_fields, have_partition_cols) = get_state_info( logical_schema.as_ref(), &self.snapshot.metadata().partition_columns, - *self.snapshot.column_mapping_mode(), + self.snapshot.column_mapping_mode, )?; let physical_schema = Arc::new(StructType::new(read_fields)); Ok(Scan { @@ -222,7 +170,7 @@ pub type ScanData = (Box, Vec); /// The result of building a scan over a table. This can be used to get the actual data from /// scanning the table. pub struct Scan { - snapshot: Arc, + snapshot: Arc, logical_schema: SchemaRef, physical_schema: SchemaRef, predicate: Option, @@ -287,7 +235,7 @@ impl Scan { // 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.snapshot - .log_segment() + .log_segment .replay(engine, commit_read_schema, checkpoint_read_schema, None) } @@ -295,11 +243,11 @@ impl Scan { /// only be called once per scan. pub fn global_scan_state(&self) -> GlobalScanState { GlobalScanState { - table_root: self.snapshot.table_root().to_string(), + table_root: self.snapshot.table_root.to_string(), partition_columns: self.snapshot.metadata().partition_columns.clone(), logical_schema: self.logical_schema.clone(), read_schema: self.physical_schema.clone(), - column_mapping_mode: *self.snapshot.column_mapping_mode(), + column_mapping_mode: self.snapshot.column_mapping_mode, } } @@ -356,10 +304,10 @@ impl Scan { let result = scan_files_iter .map(move |scan_file| -> DeltaResult<_> { let scan_file = scan_file?; - let file_path = self.snapshot.table_root().join(&scan_file.path)?; + let file_path = self.snapshot.table_root.join(&scan_file.path)?; let mut selection_vector = scan_file .dv_info - .get_selection_vector(engine, self.snapshot.table_root())?; + .get_selection_vector(engine, &self.snapshot.table_root)?; let meta = FileMeta { last_modified: 0, size: scan_file.size as usize, diff --git a/kernel/src/snapshot.rs b/kernel/src/snapshot.rs index 4785e16b..baf6045c 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, LogSegmentBuilder}; -use crate::scan::{ScanBuilder, Scannable}; +use crate::scan::ScanBuilder; use crate::schema::Schema; use crate::{DeltaResult, Engine, Error, FileSystemClient, Version}; @@ -140,12 +140,12 @@ impl Snapshot { /// Create a [`ScanBuilder`] for an `Arc`. pub fn scan_builder(self: Arc) -> ScanBuilder { - ScanBuilder::new(self as Arc) + ScanBuilder::new(self) } /// Consume this `Snapshot` to create a [`ScanBuilder`] pub fn into_scan_builder(self) -> ScanBuilder { - ScanBuilder::new(Arc::new(self) as Arc) + ScanBuilder::new(self) } } diff --git a/kernel/src/table_changes.rs b/kernel/src/table_changes.rs index c8822222..c000f9b9 100644 --- a/kernel/src/table_changes.rs +++ b/kernel/src/table_changes.rs @@ -1,27 +1,22 @@ //! In-memory representation of a change data feed table. -use std::sync::Arc; - use url::Url; use crate::{ - features::ColumnMappingMode, log_segment::{LogSegment, LogSegmentBuilder}, path::AsUrl, - scan::{ScanBuilder, Scannable}, schema::Schema, snapshot::Snapshot, DeltaResult, Engine, Version, }; -static CDF_ENABLE_FLAG: &str = "delta.enableChangeDataFeed"; +static CDF_ENABLE_FLAG: &'static str = "delta.enableChangeDataFeed"; #[derive(Debug)] pub struct TableChanges { snapshot: Snapshot, cdf_range: LogSegment, schema: Schema, - column_mapping_mode: ColumnMappingMode, } impl TableChanges { @@ -43,10 +38,6 @@ impl TableChanges { Ok(TableChanges { snapshot, cdf_range: log_segment, - column_mapping_mode: end_snapshot.column_mapping_mode, }) } - pub fn into_scan_builder(self) -> ScanBuilder { - ScanBuilder::new(Arc::new(self) as Arc) - } } From b2af9e66a0fe4b2d0c13a7c2a4cf8f5ff5faae5e Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Mon, 4 Nov 2024 09:34:44 -0800 Subject: [PATCH 29/41] Constructor --- kernel/src/error.rs | 5 ++++- kernel/src/lib.rs | 1 + kernel/src/table.rs | 10 ++++++++++ kernel/src/table_changes.rs | 31 +++++++++++++++++++++++++++---- 4 files changed, 42 insertions(+), 5 deletions(-) diff --git a/kernel/src/error.rs b/kernel/src/error.rs index 6396f5b8..1bfac226 100644 --- a/kernel/src/error.rs +++ b/kernel/src/error.rs @@ -6,7 +6,7 @@ use std::{ str::Utf8Error, }; -use crate::schema::DataType; +use crate::{schema::DataType, Version}; /// A [`std::result::Result`] that has the kernel [`Error`] as the error variant pub type DeltaResult = std::result::Result; @@ -171,6 +171,9 @@ pub enum Error { /// The file already exists at the path, prohibiting a non-overwrite write #[error("File already exists: {0}")] FileAlreadyExists(String), + + #[error("Change data feed is disabled in range: {0} to {}", _1.map_or("end".into(), |ver| ver.to_string()))] + TableChangesDisabled(Version, Option), } // Convenience constructors for Error types that take a String argument diff --git a/kernel/src/lib.rs b/kernel/src/lib.rs index c60c5b9d..ec4767f3 100644 --- a/kernel/src/lib.rs +++ b/kernel/src/lib.rs @@ -74,6 +74,7 @@ pub mod scan; pub mod schema; pub mod snapshot; pub mod table; +pub mod table_changes; pub mod transaction; pub(crate) mod utils; diff --git a/kernel/src/table.rs b/kernel/src/table.rs index d7aba989..60dfacf9 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,15 @@ impl Table { Snapshot::try_new(self.location.clone(), engine, version) } + 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)?)) diff --git a/kernel/src/table_changes.rs b/kernel/src/table_changes.rs index c000f9b9..d461037d 100644 --- a/kernel/src/table_changes.rs +++ b/kernel/src/table_changes.rs @@ -3,20 +3,26 @@ use url::Url; use crate::{ + actions::{Metadata, Protocol}, + features::ColumnMappingMode, log_segment::{LogSegment, LogSegmentBuilder}, path::AsUrl, schema::Schema, snapshot::Snapshot, - DeltaResult, Engine, Version, + DeltaResult, Engine, Error, Version, }; -static CDF_ENABLE_FLAG: &'static str = "delta.enableChangeDataFeed"; +static CDF_ENABLE_FLAG: &str = "delta.enableChangeDataFeed"; #[derive(Debug)] pub struct TableChanges { snapshot: Snapshot, cdf_range: LogSegment, schema: Schema, + version: Version, + metadata: Metadata, + protocol: Protocol, + pub(crate) column_mapping_mode: ColumnMappingMode, } impl TableChanges { @@ -26,8 +32,20 @@ impl TableChanges { 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)); + } + 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 { @@ -36,8 +54,13 @@ impl TableChanges { let log_segment = builder.build()?; Ok(TableChanges { - snapshot, + snapshot: start_snapshot, cdf_range: 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(), }) } } From 6e814cf4de733db238292bff53138b758c4b824d Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Mon, 4 Nov 2024 09:57:38 -0800 Subject: [PATCH 30/41] TableCHangesScan copied from Scan --- kernel/src/scan/mod.rs | 6 +- kernel/src/scan/table_change_scan.rs | 1 - .../mod.rs} | 22 +- .../src/table_changes/table_changes_scan.rs | 274 ++++++++++++++++++ 4 files changed, 292 insertions(+), 11 deletions(-) delete mode 100644 kernel/src/scan/table_change_scan.rs rename kernel/src/{table_changes.rs => table_changes/mod.rs} (84%) create mode 100644 kernel/src/table_changes/table_changes_scan.rs diff --git a/kernel/src/scan/mod.rs b/kernel/src/scan/mod.rs index bcbef1f5..5acc5bfe 100644 --- a/kernel/src/scan/mod.rs +++ b/kernel/src/scan/mod.rs @@ -123,7 +123,7 @@ pub struct ScanResult { pub raw_data: DeltaResult>, /// Raw row mask. // TODO(nick) this should be allocated by the engine - raw_mask: Option>, + pub raw_mask: Option>, } impl ScanResult { @@ -394,7 +394,7 @@ fn parse_partition_value(raw: Option<&String>, data_type: &DataType) -> DeltaRes /// - fields_to_read_from_parquet - Which fields should be read from the raw parquet files. This takes /// into account column mapping /// - have_partition_cols - boolean indicating if we have partition columns in this query -fn get_state_info( +pub fn get_state_info( logical_schema: &Schema, partition_columns: &[String], column_mapping_mode: ColumnMappingMode, @@ -463,7 +463,7 @@ pub fn transform_to_logical( // We have this function because `execute` can save `all_fields` and `have_partition_cols` in the // scan, and then reuse them for each batch transform -fn transform_to_logical_internal( +pub fn transform_to_logical_internal( engine: &dyn Engine, data: Box, global_state: &GlobalScanState, diff --git a/kernel/src/scan/table_change_scan.rs b/kernel/src/scan/table_change_scan.rs deleted file mode 100644 index fafcbaf7..00000000 --- a/kernel/src/scan/table_change_scan.rs +++ /dev/null @@ -1 +0,0 @@ -pub struct TableChangesScan {} diff --git a/kernel/src/table_changes.rs b/kernel/src/table_changes/mod.rs similarity index 84% rename from kernel/src/table_changes.rs rename to kernel/src/table_changes/mod.rs index d461037d..a05d4bf0 100644 --- a/kernel/src/table_changes.rs +++ b/kernel/src/table_changes/mod.rs @@ -1,5 +1,6 @@ //! In-memory representation of a change data feed table. +use table_changes_scan::TableChangesScan; use url::Url; use crate::{ @@ -12,17 +13,20 @@ use crate::{ DeltaResult, Engine, Error, Version, }; +pub mod table_changes_scan; + static CDF_ENABLE_FLAG: &str = "delta.enableChangeDataFeed"; #[derive(Debug)] pub struct TableChanges { - snapshot: Snapshot, - cdf_range: LogSegment, - schema: Schema, - version: Version, - metadata: Metadata, - protocol: Protocol, + 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 { @@ -55,12 +59,16 @@ impl TableChanges { Ok(TableChanges { snapshot: start_snapshot, - cdf_range: log_segment, + 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) -> TableChangesScan { + todo!() + } } 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 00000000..710e5b8e --- /dev/null +++ b/kernel/src/table_changes/table_changes_scan.rs @@ -0,0 +1,274 @@ +use std::{collections::HashMap, sync::Arc}; + +use itertools::Itertools; +use tracing::debug; + +use crate::{ + actions::{ + deletion_vector::split_vector, get_log_add_schema, get_log_schema, ADD_NAME, REMOVE_NAME, + }, + scan::{ + get_state_info, + log_replay::scan_action_iter, + state::{self, DvInfo, GlobalScanState, Stats}, + transform_to_logical_internal, ColumnType, ScanData, ScanResult, + }, + schema::{SchemaRef, StructType}, + DeltaResult, Engine, EngineData, ExpressionRef, FileMeta, +}; + +use super::TableChanges; + +/// 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, +} + +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>> { + Ok(scan_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, bool)>> + Send> { + let commit_read_schema = get_log_schema().project(&[ADD_NAME, REMOVE_NAME])?; + let checkpoint_read_schema = get_log_add_schema().clone(); + + // 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( + engine, + commit_read_schema, + checkpoint_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 ScanFile { + path: String, + size: i64, + dv_info: DvInfo, + partition_values: HashMap, + } + fn scan_data_callback( + batches: &mut Vec, + path: &str, + size: i64, + _: Option, + dv_info: DvInfo, + partition_values: HashMap, + ) { + batches.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 + ); + + let global_state = Arc::new(self.global_scan_state()); + let scan_data = self.scan_data(engine)?; + let scan_files_iter = scan_data + .map(|res| { + let (data, vec) = res?; + let scan_files = vec![]; + state::visit_scan_files(data.as_ref(), &vec, scan_files, scan_data_callback) + }) + // Iterator>> to Iterator> + .flatten_ok(); + + let result = scan_files_iter + .map(move |scan_file| -> DeltaResult<_> { + let scan_file = scan_file?; + 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 412f55187ca0e074425feb606869241e91aa1444 Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Mon, 4 Nov 2024 10:05:51 -0800 Subject: [PATCH 31/41] CHange log segment builder's names --- kernel/src/table_changes/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kernel/src/table_changes/mod.rs b/kernel/src/table_changes/mod.rs index a05d4bf0..9fe8c832 100644 --- a/kernel/src/table_changes/mod.rs +++ b/kernel/src/table_changes/mod.rs @@ -53,7 +53,7 @@ impl TableChanges { 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); + builder = builder.with_end_version(end_version); } let log_segment = builder.build()?; From 1a6a6307ccb36cacea6a333954fc8ee62fb60e04 Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Tue, 5 Nov 2024 13:46:33 -0800 Subject: [PATCH 32/41] Basic iteration for CDF --- kernel/examples/change_data/Cargo.toml | 1 + .../_delta_log/00000000000000000000.json | 4 + .../_delta_log/00000000000000000001.json | 3 + .../_delta_log/00000000000000000002.json | 2 + kernel/examples/change_data/src/main.rs | 24 ++- kernel/src/actions/mod.rs | 1 + kernel/src/log_segment.rs | 17 ++ kernel/src/scan/data_skipping.rs | 1 + kernel/src/scan/log_replay.rs | 2 +- kernel/src/scan/mod.rs | 2 +- kernel/src/table_changes/metadata_scanner.rs | 168 ++++++++++++++++++ kernel/src/table_changes/mod.rs | 18 +- kernel/src/table_changes/replay_scanner.rs | 168 ++++++++++++++++++ .../src/table_changes/table_changes_scan.rs | 108 +++++++++-- 14 files changed, 492 insertions(+), 27 deletions(-) create mode 100644 kernel/examples/change_data/deltalog-getChanges/_delta_log/00000000000000000000.json create mode 100644 kernel/examples/change_data/deltalog-getChanges/_delta_log/00000000000000000001.json create mode 100644 kernel/examples/change_data/deltalog-getChanges/_delta_log/00000000000000000002.json create mode 100644 kernel/src/table_changes/metadata_scanner.rs create mode 100644 kernel/src/table_changes/replay_scanner.rs diff --git a/kernel/examples/change_data/Cargo.toml b/kernel/examples/change_data/Cargo.toml index 6265eb13..5a873054 100644 --- a/kernel/examples/change_data/Cargo.toml +++ b/kernel/examples/change_data/Cargo.toml @@ -15,3 +15,4 @@ delta_kernel = { path = "../../../kernel", features = [ ] } env_logger = "0.11.3" url = "2" +itertools = "0.13" diff --git a/kernel/examples/change_data/deltalog-getChanges/_delta_log/00000000000000000000.json b/kernel/examples/change_data/deltalog-getChanges/_delta_log/00000000000000000000.json new file mode 100644 index 00000000..a699acee --- /dev/null +++ b/kernel/examples/change_data/deltalog-getChanges/_delta_log/00000000000000000000.json @@ -0,0 +1,4 @@ +{"commitInfo":{"timestamp":1704392842074,"operation":"Manual Update","operationParameters":{},"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{},"engineInfo":"Apache-Spark/3.5.0 Delta-Lake/3.1.0-SNAPSHOT","txnId":"95ec924a-6859-4433-8008-6d6b4a0e3ba5"}} +{"protocol":{"minReaderVersion":3,"minWriterVersion":7}} +{"metaData":{"id":"testId","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"part\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}},{\"name\":\"id\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":[],"configuration":{"delta.enableChangeDataFeed": "true"}}} +{"add":{"path":"fake/path/1","partitionValues":{},"size":1,"modificationTime":1,"dataChange":true}} diff --git a/kernel/examples/change_data/deltalog-getChanges/_delta_log/00000000000000000001.json b/kernel/examples/change_data/deltalog-getChanges/_delta_log/00000000000000000001.json new file mode 100644 index 00000000..8f376bd9 --- /dev/null +++ b/kernel/examples/change_data/deltalog-getChanges/_delta_log/00000000000000000001.json @@ -0,0 +1,3 @@ +{"commitInfo":{"timestamp":1704392846030,"operation":"Manual Update","operationParameters":{},"readVersion":0,"isolationLevel":"Serializable","isBlindAppend":false,"operationMetrics":{},"engineInfo":"Apache-Spark/3.5.0 Delta-Lake/3.1.0-SNAPSHOT","txnId":"01d40235-c8b4-4f8e-8f19-8c97872217fd"}} +{"cdc":{"path":"fake/path/2","partitionValues":{"partition_foo":"partition_bar"},"size":1,"tags":{"tag_foo":"tag_bar"},"dataChange":false}} +{"remove":{"path":"fake/path/1","deletionTimestamp":100,"dataChange":true}} diff --git a/kernel/examples/change_data/deltalog-getChanges/_delta_log/00000000000000000002.json b/kernel/examples/change_data/deltalog-getChanges/_delta_log/00000000000000000002.json new file mode 100644 index 00000000..271a31dd --- /dev/null +++ b/kernel/examples/change_data/deltalog-getChanges/_delta_log/00000000000000000002.json @@ -0,0 +1,2 @@ +{"commitInfo":{"timestamp":1704392846603,"operation":"Manual Update","operationParameters":{},"readVersion":1,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{},"engineInfo":"Apache-Spark/3.5.0 Delta-Lake/3.1.0-SNAPSHOT","txnId":"6cef7579-ca93-4427-988e-9269e8db50c7"}} +{"txn":{"appId":"fakeAppId","version":3,"lastUpdated":200}} diff --git a/kernel/examples/change_data/src/main.rs b/kernel/examples/change_data/src/main.rs index 055f78bd..64d66c7b 100644 --- a/kernel/examples/change_data/src/main.rs +++ b/kernel/examples/change_data/src/main.rs @@ -1,12 +1,24 @@ use std::{collections::HashMap, sync::Arc}; +use arrow_array::RecordBatch; use delta_kernel::{ - engine::default::{executor::tokio::TokioBackgroundExecutor, DefaultEngine}, - DeltaResult, Table, + engine::{ + arrow_data::ArrowEngineData, + default::{executor::tokio::TokioBackgroundExecutor, DefaultEngine}, + }, + scan::ScanResult, + DeltaResult, EngineData, Table, }; +use itertools::Itertools; +fn into_record_batch(engine_data: DeltaResult>) -> DeltaResult { + engine_data + .and_then(ArrowEngineData::try_from_engine_data) + .map(Into::into) +} fn main() -> DeltaResult<()> { - let uri = "/Users/oussama.saoudi/delta-kernel-rs/kernel/tests/data/table-with-dv-small/"; + let uri = + "/Users/oussama.saoudi/delta-kernel-rs/kernel/examples/change_data/deltalog-getChanges"; // build a table and get the lastest snapshot from it let table = Table::try_from_uri(uri)?; @@ -17,7 +29,11 @@ fn main() -> DeltaResult<()> { )?; let table_changes = table.table_changes(&engine, 0, None)?; - println!("table_changes: {:?}", table_changes); + let x = table_changes.into_scan_builder().build()?; + let vec: Vec = x.execute(&engine)?.try_collect()?; + for res in vec { + println!("{:?}", into_record_batch(res.raw_data)?) + } Ok(()) } diff --git a/kernel/src/actions/mod.rs b/kernel/src/actions/mod.rs index b23dd651..b61a4eef 100644 --- a/kernel/src/actions/mod.rs +++ b/kernel/src/actions/mod.rs @@ -28,6 +28,7 @@ pub(crate) const METADATA_NAME: &str = "metaData"; pub(crate) const PROTOCOL_NAME: &str = "protocol"; pub(crate) const SET_TRANSACTION_NAME: &str = "txn"; pub(crate) const COMMIT_INFO_NAME: &str = "commitInfo"; +pub(crate) const CDC_NAME: &str = "cdc"; static LOG_ADD_SCHEMA: LazyLock = LazyLock::new(|| StructType::new([Option::::get_struct_field(ADD_NAME)]).into()); diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index d0dd00f4..e77c201c 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)?; diff --git a/kernel/src/scan/data_skipping.rs b/kernel/src/scan/data_skipping.rs index 7c5aea37..9056dac7 100644 --- a/kernel/src/scan/data_skipping.rs +++ b/kernel/src/scan/data_skipping.rs @@ -163,6 +163,7 @@ fn as_data_skipping_predicate(expr: &Expr) -> Option { } } +#[derive(Clone)] pub(crate) struct DataSkippingFilter { stats_schema: SchemaRef, select_stats_evaluator: Arc, diff --git a/kernel/src/scan/log_replay.rs b/kernel/src/scan/log_replay.rs index f872a8ec..a91a2ee7 100644 --- a/kernel/src/scan/log_replay.rs +++ b/kernel/src/scan/log_replay.rs @@ -108,7 +108,7 @@ pub(crate) static SCAN_ROW_SCHEMA: LazyLock> = LazyLock::new(|| ), ])) }); -static SCAN_ROW_DATATYPE: LazyLock = LazyLock::new(|| SCAN_ROW_SCHEMA.clone().into()); +pub static SCAN_ROW_DATATYPE: LazyLock = LazyLock::new(|| SCAN_ROW_SCHEMA.clone().into()); impl LogReplayScanner { /// Create a new [`LogReplayScanner`] instance diff --git a/kernel/src/scan/mod.rs b/kernel/src/scan/mod.rs index 5acc5bfe..280ef064 100644 --- a/kernel/src/scan/mod.rs +++ b/kernel/src/scan/mod.rs @@ -19,7 +19,7 @@ use crate::{DeltaResult, Engine, EngineData, Error, FileMeta}; use self::log_replay::scan_action_iter; use self::state::GlobalScanState; -mod data_skipping; +pub mod data_skipping; pub mod log_replay; pub mod state; diff --git a/kernel/src/table_changes/metadata_scanner.rs b/kernel/src/table_changes/metadata_scanner.rs new file mode 100644 index 00000000..55c765aa --- /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 index 9fe8c832..2d58ff66 100644 --- a/kernel/src/table_changes/mod.rs +++ b/kernel/src/table_changes/mod.rs @@ -1,6 +1,8 @@ //! In-memory representation of a change data feed table. -use table_changes_scan::TableChangesScan; +use std::{collections::HashMap, sync::Arc}; + +use table_changes_scan::TableChangesScanBuilder; use url::Url; use crate::{ @@ -10,11 +12,15 @@ use crate::{ path::AsUrl, schema::Schema, snapshot::Snapshot, - DeltaResult, Engine, Error, Version, + DeltaResult, Engine, EngineData, Error, Version, }; +mod metadata_scanner; +mod replay_scanner; pub mod table_changes_scan; +pub type TableChangesScanData = (Box, Vec, Arc>); + static CDF_ENABLE_FLAG: &str = "delta.enableChangeDataFeed"; #[derive(Debug)] @@ -49,12 +55,16 @@ impl TableChanges { 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 { @@ -68,7 +78,7 @@ impl TableChanges { table_root, }) } - pub fn into_scan_builder(self) -> TableChangesScan { - todo!() + 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 00000000..e732ee9b --- /dev/null +++ b/kernel/src/table_changes/replay_scanner.rs @@ -0,0 +1,168 @@ +use std::collections::{HashMap, HashSet}; +use std::sync::Arc; + +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}; + +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() { + let dv_id = remove.dv_unique_id(); + if let Some(dv_id) = dv_id { + self.remove_dvs.insert(remove.path.clone(), dv_id); + } + selection_vector[index] = true; + debug!( + "Including file in scan: ({}, {:?})", + remove.path, + remove.dv_unique_id(), + ); + } + + 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/table_changes_scan.rs b/kernel/src/table_changes/table_changes_scan.rs index 710e5b8e..b0d588c6 100644 --- a/kernel/src/table_changes/table_changes_scan.rs +++ b/kernel/src/table_changes/table_changes_scan.rs @@ -1,13 +1,17 @@ -use std::{collections::HashMap, sync::Arc}; +use std::{ + collections::{HashMap, HashSet}, + iter, + sync::Arc, +}; use itertools::Itertools; use tracing::debug; use crate::{ - actions::{ - deletion_vector::split_vector, get_log_add_schema, get_log_schema, ADD_NAME, REMOVE_NAME, - }, + actions::{deletion_vector::split_vector, get_log_schema, ADD_NAME, REMOVE_NAME}, + expressions, scan::{ + data_skipping::DataSkippingFilter, get_state_info, log_replay::scan_action_iter, state::{self, DvInfo, GlobalScanState, Stats}, @@ -17,7 +21,7 @@ use crate::{ DeltaResult, Engine, EngineData, ExpressionRef, FileMeta, }; -use super::TableChanges; +use super::{replay_scanner::TableChangesLogReplayScanner, TableChanges, TableChangesScanData}; /// Builder to scan a snapshot of a table. pub struct TableChangesScanBuilder { @@ -104,6 +108,76 @@ pub struct TableChangesScan { 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. /// @@ -132,31 +206,31 @@ impl TableChangesScan { pub fn scan_data( &self, engine: &dyn Engine, - ) -> DeltaResult>> { - Ok(scan_action_iter( + ) -> 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, bool)>> + Send> { + ) -> DeltaResult< + impl Iterator< + Item = DeltaResult>> + Send>>, + >, + > { let commit_read_schema = get_log_schema().project(&[ADD_NAME, REMOVE_NAME])?; - let checkpoint_read_schema = get_log_add_schema().clone(); // 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( - engine, - commit_read_schema, - checkpoint_read_schema, - None, - ) + 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 @@ -214,7 +288,7 @@ impl TableChangesScan { let scan_data = self.scan_data(engine)?; let scan_files_iter = scan_data .map(|res| { - let (data, vec) = res?; + let (data, vec, _) = res?; let scan_files = vec![]; state::visit_scan_files(data.as_ref(), &vec, scan_files, scan_data_callback) }) From de206c73e7f549d596155b4332ba4f9df6537985 Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Wed, 6 Nov 2024 09:39:50 -0800 Subject: [PATCH 33/41] Read path until scanfile --- .../_delta_log/00000000000000000000.json | 4 - .../_delta_log/00000000000000000001.json | 3 - .../_delta_log/00000000000000000002.json | 2 - kernel/examples/change_data/src/main.rs | 5 +- .../_delta_log/00000000000000000000.json | 4 + .../_delta_log/00000000000000000001.json | 3 + ...r_61d16c75-6994-46b7-a15b-8b538852e50e.bin | Bin 0 -> 45 bytes ...4e51-827b-c3d5516560ca-c000.snappy.parquet | Bin 0 -> 635 bytes kernel/src/scan/state.rs | 37 ++++- kernel/src/table_changes/mod.rs | 4 +- kernel/src/table_changes/replay_scanner.rs | 17 ++- kernel/src/table_changes/state.rs | 127 ++++++++++++++++++ .../src/table_changes/table_changes_scan.rs | 47 +++++-- 13 files changed, 216 insertions(+), 37 deletions(-) delete mode 100644 kernel/examples/change_data/deltalog-getChanges/_delta_log/00000000000000000000.json delete mode 100644 kernel/examples/change_data/deltalog-getChanges/_delta_log/00000000000000000001.json delete mode 100644 kernel/examples/change_data/deltalog-getChanges/_delta_log/00000000000000000002.json create mode 100644 kernel/examples/change_data/table-with-dv-small/_delta_log/00000000000000000000.json create mode 100644 kernel/examples/change_data/table-with-dv-small/_delta_log/00000000000000000001.json create mode 100644 kernel/examples/change_data/table-with-dv-small/deletion_vector_61d16c75-6994-46b7-a15b-8b538852e50e.bin create mode 100644 kernel/examples/change_data/table-with-dv-small/part-00000-fae5310a-a37d-4e51-827b-c3d5516560ca-c000.snappy.parquet create mode 100644 kernel/src/table_changes/state.rs diff --git a/kernel/examples/change_data/deltalog-getChanges/_delta_log/00000000000000000000.json b/kernel/examples/change_data/deltalog-getChanges/_delta_log/00000000000000000000.json deleted file mode 100644 index a699acee..00000000 --- a/kernel/examples/change_data/deltalog-getChanges/_delta_log/00000000000000000000.json +++ /dev/null @@ -1,4 +0,0 @@ -{"commitInfo":{"timestamp":1704392842074,"operation":"Manual Update","operationParameters":{},"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{},"engineInfo":"Apache-Spark/3.5.0 Delta-Lake/3.1.0-SNAPSHOT","txnId":"95ec924a-6859-4433-8008-6d6b4a0e3ba5"}} -{"protocol":{"minReaderVersion":3,"minWriterVersion":7}} -{"metaData":{"id":"testId","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"part\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}},{\"name\":\"id\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":[],"configuration":{"delta.enableChangeDataFeed": "true"}}} -{"add":{"path":"fake/path/1","partitionValues":{},"size":1,"modificationTime":1,"dataChange":true}} diff --git a/kernel/examples/change_data/deltalog-getChanges/_delta_log/00000000000000000001.json b/kernel/examples/change_data/deltalog-getChanges/_delta_log/00000000000000000001.json deleted file mode 100644 index 8f376bd9..00000000 --- a/kernel/examples/change_data/deltalog-getChanges/_delta_log/00000000000000000001.json +++ /dev/null @@ -1,3 +0,0 @@ -{"commitInfo":{"timestamp":1704392846030,"operation":"Manual Update","operationParameters":{},"readVersion":0,"isolationLevel":"Serializable","isBlindAppend":false,"operationMetrics":{},"engineInfo":"Apache-Spark/3.5.0 Delta-Lake/3.1.0-SNAPSHOT","txnId":"01d40235-c8b4-4f8e-8f19-8c97872217fd"}} -{"cdc":{"path":"fake/path/2","partitionValues":{"partition_foo":"partition_bar"},"size":1,"tags":{"tag_foo":"tag_bar"},"dataChange":false}} -{"remove":{"path":"fake/path/1","deletionTimestamp":100,"dataChange":true}} diff --git a/kernel/examples/change_data/deltalog-getChanges/_delta_log/00000000000000000002.json b/kernel/examples/change_data/deltalog-getChanges/_delta_log/00000000000000000002.json deleted file mode 100644 index 271a31dd..00000000 --- a/kernel/examples/change_data/deltalog-getChanges/_delta_log/00000000000000000002.json +++ /dev/null @@ -1,2 +0,0 @@ -{"commitInfo":{"timestamp":1704392846603,"operation":"Manual Update","operationParameters":{},"readVersion":1,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{},"engineInfo":"Apache-Spark/3.5.0 Delta-Lake/3.1.0-SNAPSHOT","txnId":"6cef7579-ca93-4427-988e-9269e8db50c7"}} -{"txn":{"appId":"fakeAppId","version":3,"lastUpdated":200}} diff --git a/kernel/examples/change_data/src/main.rs b/kernel/examples/change_data/src/main.rs index 64d66c7b..20b28f98 100644 --- a/kernel/examples/change_data/src/main.rs +++ b/kernel/examples/change_data/src/main.rs @@ -18,7 +18,7 @@ fn into_record_batch(engine_data: DeltaResult>) -> DeltaResu } fn main() -> DeltaResult<()> { let uri = - "/Users/oussama.saoudi/delta-kernel-rs/kernel/examples/change_data/deltalog-getChanges"; + "/Users/oussama.saoudi/delta-kernel-rs/kernel/examples/change_data/table-with-dv-small"; // build a table and get the lastest snapshot from it let table = Table::try_from_uri(uri)?; @@ -31,8 +31,9 @@ fn main() -> DeltaResult<()> { let table_changes = table.table_changes(&engine, 0, None)?; let x = table_changes.into_scan_builder().build()?; let vec: Vec = x.execute(&engine)?.try_collect()?; + println!("Vec len: {:?}", vec.len()); for res in vec { - println!("{:?}", into_record_batch(res.raw_data)?) + // println!("{:?}", into_record_batch(res.raw_data)?) } Ok(()) diff --git a/kernel/examples/change_data/table-with-dv-small/_delta_log/00000000000000000000.json b/kernel/examples/change_data/table-with-dv-small/_delta_log/00000000000000000000.json new file mode 100644 index 00000000..2d01821b --- /dev/null +++ b/kernel/examples/change_data/table-with-dv-small/_delta_log/00000000000000000000.json @@ -0,0 +1,4 @@ +{"commitInfo":{"timestamp":1677811178585,"operation":"WRITE","operationParameters":{"mode":"ErrorIfExists","partitionBy":"[]"},"isolationLevel":"WriteSerializable","isBlindAppend":true,"operationMetrics":{"numFiles":"1","numOutputRows":"10","numOutputBytes":"635"},"engineInfo":"Databricks-Runtime/","txnId":"a6a94671-55ef-450e-9546-b8465b9147de"}} +{"protocol":{"minReaderVersion":3,"minWriterVersion":7,"readerFeatures":["deletionVectors"],"writerFeatures":["deletionVectors"]}} +{"metaData":{"id":"testId","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"value\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":[],"configuration":{"delta.enableChangeDataFeed":"true","delta.enableDeletionVectors":"true","delta.columnMapping.mode":"none"},"createdTime":1677811175819}} +{"add":{"path":"part-00000-fae5310a-a37d-4e51-827b-c3d5516560ca-c000.snappy.parquet","partitionValues":{},"size":635,"modificationTime":1677811178336,"dataChange":true,"stats":"{\"numRecords\":10,\"minValues\":{\"value\":0},\"maxValues\":{\"value\":9},\"nullCount\":{\"value\":0},\"tightBounds\":true}","tags":{"INSERTION_TIME":"1677811178336000","MIN_INSERTION_TIME":"1677811178336000","MAX_INSERTION_TIME":"1677811178336000","OPTIMIZE_TARGET_SIZE":"268435456"}}} diff --git a/kernel/examples/change_data/table-with-dv-small/_delta_log/00000000000000000001.json b/kernel/examples/change_data/table-with-dv-small/_delta_log/00000000000000000001.json new file mode 100644 index 00000000..e5bcdc11 --- /dev/null +++ b/kernel/examples/change_data/table-with-dv-small/_delta_log/00000000000000000001.json @@ -0,0 +1,3 @@ +{"commitInfo":{"timestamp":1677811194429,"operation":"DELETE","operationParameters":{"predicate":"[\"(spark_catalog.delta.`/tmp/table-with-dv-small`.value IN (0, 9))\"]"},"readVersion":0,"isolationLevel":"WriteSerializable","isBlindAppend":false,"operationMetrics":{"numRemovedFiles":"0","numRemovedBytes":"0","numCopiedRows":"0","numDeletionVectorsAdded":"1","numDeletionVectorsRemoved":"0","numAddedChangeFiles":"0","executionTimeMs":"10364","numDeletedRows":"2","scanTimeMs":"9869","numAddedFiles":"0","numAddedBytes":"0","rewriteTimeMs":"479"},"engineInfo":"Databricks-Runtime/","txnId":"6d9555a2-0e3b-4c15-80c0-d5c3b0cf1277"}} +{"remove":{"path":"part-00000-fae5310a-a37d-4e51-827b-c3d5516560ca-c000.snappy.parquet","deletionTimestamp":1677811194426,"dataChange":true,"extendedFileMetadata":true,"partitionValues":{},"size":635,"tags":{"INSERTION_TIME":"1677811178336000","MIN_INSERTION_TIME":"1677811178336000","MAX_INSERTION_TIME":"1677811178336000","OPTIMIZE_TARGET_SIZE":"268435456"}}} +{"add":{"path":"part-00000-fae5310a-a37d-4e51-827b-c3d5516560ca-c000.snappy.parquet","partitionValues":{},"size":635,"modificationTime":1677811178336,"dataChange":true,"stats":"{\"numRecords\":10,\"minValues\":{\"value\":0},\"maxValues\":{\"value\":9},\"nullCount\":{\"value\":0},\"tightBounds\":false}","tags":{"INSERTION_TIME":"1677811178336000","MIN_INSERTION_TIME":"1677811178336000","MAX_INSERTION_TIME":"1677811178336000","OPTIMIZE_TARGET_SIZE":"268435456"},"deletionVector":{"storageType":"u","pathOrInlineDv":"vBn[lx{q8@P<9BNH/isA","offset":1,"sizeInBytes":36,"cardinality":2}}} diff --git a/kernel/examples/change_data/table-with-dv-small/deletion_vector_61d16c75-6994-46b7-a15b-8b538852e50e.bin b/kernel/examples/change_data/table-with-dv-small/deletion_vector_61d16c75-6994-46b7-a15b-8b538852e50e.bin new file mode 100644 index 0000000000000000000000000000000000000000..f1a01e661cdcca08ff5d67e7d2de53381980735a GIT binary patch literal 45 lcmZQ%U|>+Wc-b+>@u6oHnRZaa}xk;gAOoS z#<6}t%0rSCNt@&m3E|*~#Cc*mq6q?Bjf0sEC&T3yyEjN{9KCms%M}*aS7r27rN~Tj zzA#1W7Im$F+m7qFUCSlpIkYWjA7Cc8`45BN8(r(ouK2DKhm&oqokhKfIaLJUzYWIu zPlHKl678-<$u_Z>3nwp@5?4qBej=(UKN%Q>#iA`8S!W3Kv+Rn6JI+Zl%15 zS5`$GRbJ1F6QviWH~GBwGEAG$c3l+NBa^IBOI45~tF^{Z6NZvi&--82o2)mRFB=fg z4w&qQd5e|0q&+nA%=X0kY0=qF(izCXnGE_3#gJBXUG{z7KkJ-?b)pv?9F9Xjj^mLU lg%~U_#xTV3XgCPMU~!JGNsB^M@u}mw^bwBeg)ZpZ{R3QVoUi}@ literal 0 HcmV?d00001 diff --git a/kernel/src/scan/state.rs b/kernel/src/scan/state.rs index 2dc30f23..849bc896 100644 --- a/kernel/src/scan/state.rs +++ b/kernel/src/scan/state.rs @@ -12,6 +12,7 @@ use crate::{ schema::SchemaRef, DataVisitor, DeltaResult, Engine, EngineData, Error, }; +use roaring::RoaringTreemap; use serde::{Deserialize, Serialize}; use tracing::warn; @@ -28,7 +29,7 @@ pub struct GlobalScanState { } /// this struct can be used by an engine to materialize a selection vector -#[derive(Debug)] +#[derive(Debug, Clone)] pub struct DvInfo { pub(crate) deletion_vector: Option, } @@ -51,19 +52,43 @@ impl DvInfo { self.deletion_vector.is_some() } - pub fn get_selection_vector( + pub fn as_dv_tree_map( &self, engine: &dyn Engine, table_root: &url::Url, - ) -> DeltaResult>> { - let dv_treemap = self - .deletion_vector + ) -> DeltaResult> { + self.deletion_vector .as_ref() .map(|dv_descriptor| { let fs_client = engine.get_file_system_client(); dv_descriptor.read(fs_client, table_root) }) - .transpose()?; + .transpose() + } + + pub fn get_diff_selection_vector( + &self, + other: &DvInfo, + engine: &dyn Engine, + table_root: &url::Url, + ) -> DeltaResult>> { + if let (Some(self_tm), Some(other_tm)) = ( + self.as_dv_tree_map(engine, table_root)?, + other.as_dv_tree_map(engine, table_root)?, + ) { + let diff = self_tm - other_tm; + Ok(Some(treemap_to_bools(diff))) + } else { + Ok(None) + } + } + + pub fn get_selection_vector( + &self, + engine: &dyn Engine, + table_root: &url::Url, + ) -> DeltaResult>> { + let dv_treemap = self.as_dv_tree_map(engine, table_root)?; Ok(dv_treemap.map(treemap_to_bools)) } diff --git a/kernel/src/table_changes/mod.rs b/kernel/src/table_changes/mod.rs index 2d58ff66..debf9cf6 100644 --- a/kernel/src/table_changes/mod.rs +++ b/kernel/src/table_changes/mod.rs @@ -10,6 +10,7 @@ use crate::{ features::ColumnMappingMode, log_segment::{LogSegment, LogSegmentBuilder}, path::AsUrl, + scan::state::DvInfo, schema::Schema, snapshot::Snapshot, DeltaResult, Engine, EngineData, Error, Version, @@ -17,9 +18,10 @@ use crate::{ mod metadata_scanner; mod replay_scanner; +mod state; pub mod table_changes_scan; -pub type TableChangesScanData = (Box, Vec, Arc>); +pub type TableChangesScanData = (Box, Vec, Arc>); static CDF_ENABLE_FLAG: &str = "delta.enableChangeDataFeed"; diff --git a/kernel/src/table_changes/replay_scanner.rs b/kernel/src/table_changes/replay_scanner.rs index e732ee9b..0de5e6f7 100644 --- a/kernel/src/table_changes/replay_scanner.rs +++ b/kernel/src/table_changes/replay_scanner.rs @@ -3,12 +3,14 @@ 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}; @@ -65,7 +67,7 @@ impl DataVisitor for AddRemoveCdcVisitor { pub(crate) struct TableChangesLogReplayScanner { filter: Option, - pub remove_dvs: HashMap, + pub remove_dvs: HashMap, } impl TableChangesLogReplayScanner { @@ -114,7 +116,6 @@ impl TableChangesLogReplayScanner { 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: @@ -127,16 +128,18 @@ impl TableChangesLogReplayScanner { ); } for (remove, index) in removes.into_iter() { - let dv_id = remove.dv_unique_id(); - if let Some(dv_id) = dv_id { - self.remove_dvs.insert(remove.path.clone(), dv_id); - } - selection_vector[index] = true; 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 diff --git a/kernel/src/table_changes/state.rs b/kernel/src/table_changes/state.rs new file mode 100644 index 00000000..6bd7f6f7 --- /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 index b0d588c6..2f522900 100644 --- a/kernel/src/table_changes/table_changes_scan.rs +++ b/kernel/src/table_changes/table_changes_scan.rs @@ -8,7 +8,7 @@ use itertools::Itertools; use tracing::debug; use crate::{ - actions::{deletion_vector::split_vector, get_log_schema, ADD_NAME, REMOVE_NAME}, + actions::{deletion_vector::split_vector, get_log_schema, Add, Remove, ADD_NAME, REMOVE_NAME}, expressions, scan::{ data_skipping::DataSkippingFilter, @@ -18,6 +18,7 @@ use crate::{ transform_to_logical_internal, ColumnType, ScanData, ScanResult, }, schema::{SchemaRef, StructType}, + table_changes::replay_scanner::AddRemoveCdcVisitor, DeltaResult, Engine, EngineData, ExpressionRef, FileMeta, }; @@ -257,6 +258,10 @@ impl TableChangesScan { &'a self, engine: &'a dyn Engine, ) -> DeltaResult> + 'a> { + struct ScanFileContext { + pub files: Vec, + pub remove_dv: Arc>, + } struct ScanFile { path: String, size: i64, @@ -264,14 +269,14 @@ impl TableChangesScan { partition_values: HashMap, } fn scan_data_callback( - batches: &mut Vec, + context: &mut ScanFileContext, path: &str, size: i64, _: Option, dv_info: DvInfo, partition_values: HashMap, ) { - batches.push(ScanFile { + context.files.push(ScanFile { path: path.to_string(), size, dv_info, @@ -279,6 +284,14 @@ impl TableChangesScan { }); } + 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 @@ -286,18 +299,28 @@ impl TableChangesScan { let global_state = Arc::new(self.global_scan_state()); let scan_data = self.scan_data(engine)?; - let scan_files_iter = scan_data - .map(|res| { - let (data, vec, _) = res?; - let scan_files = vec![]; - state::visit_scan_files(data.as_ref(), &vec, scan_files, scan_data_callback) + 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()))) }) - // Iterator>> to Iterator> - .flatten_ok(); + .flatten_ok() + .collect_vec(); let result = scan_files_iter - .map(move |scan_file| -> DeltaResult<_> { - let scan_file = scan_file?; + .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 From 298c70c688fbddff9350ec341765c657884b3e3b Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Wed, 6 Nov 2024 16:02:19 -0800 Subject: [PATCH 34/41] Remove non-constructor code --- kernel/src/actions/mod.rs | 1 + kernel/src/log_segment.rs | 31 +- kernel/src/table_changes/metadata_scanner.rs | 168 -------- kernel/src/table_changes/mod.rs | 107 ++++- kernel/src/table_changes/replay_scanner.rs | 171 -------- kernel/src/table_changes/state.rs | 127 ------ .../src/table_changes/table_changes_scan.rs | 371 ------------------ 7 files changed, 112 insertions(+), 864 deletions(-) delete mode 100644 kernel/src/table_changes/metadata_scanner.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/actions/mod.rs b/kernel/src/actions/mod.rs index b61a4eef..bf4e6979 100644 --- a/kernel/src/actions/mod.rs +++ b/kernel/src/actions/mod.rs @@ -28,6 +28,7 @@ pub(crate) const METADATA_NAME: &str = "metaData"; pub(crate) const PROTOCOL_NAME: &str = "protocol"; pub(crate) const SET_TRANSACTION_NAME: &str = "txn"; pub(crate) const COMMIT_INFO_NAME: &str = "commitInfo"; +#[allow(unused)] pub(crate) const CDC_NAME: &str = "cdc"; static LOG_ADD_SCHEMA: LazyLock = diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index e77c201c..3bea8f2a 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -21,12 +21,12 @@ use itertools::Itertools; #[derive(Debug)] #[cfg_attr(feature = "developer-visibility", visibility::make(pub))] pub(crate) struct LogSegment { - pub version: Version, - pub log_root: Url, + pub(crate) version: Version, + pub(crate) log_root: Url, /// Reverse order sorted commit files in the log segment - pub commit_files: Vec, + pub(crate) commit_files: Vec, /// checkpoint files in the log segment. - pub checkpoint_files: Vec, + pub(crate) checkpoint_files: Vec, } impl LogSegment { @@ -70,6 +70,7 @@ impl LogSegment { Ok(commit_stream.chain(checkpoint_stream)) } + #[allow(unused)] pub(crate) fn replay_commits( &self, engine: &dyn Engine, @@ -132,7 +133,7 @@ impl LogSegment { } } -pub struct LogSegmentBuilder<'a> { +pub(crate) struct LogSegmentBuilder<'a> { fs_client: Arc, log_root: &'a Url, checkpoint: Option, @@ -142,7 +143,7 @@ pub struct LogSegmentBuilder<'a> { in_order_commit_files: bool, } impl<'a> LogSegmentBuilder<'a> { - pub fn new(fs_client: Arc, log_root: &'a Url) -> Self { + pub(crate) fn new(fs_client: Arc, log_root: &'a Url) -> Self { LogSegmentBuilder { fs_client, log_root, @@ -154,28 +155,28 @@ impl<'a> LogSegmentBuilder<'a> { } } - pub fn with_checkpoint(mut self, checkpoint: CheckpointMetadata) -> Self { + pub(crate) fn with_checkpoint(mut self, checkpoint: CheckpointMetadata) -> Self { let _ = self.checkpoint.insert(checkpoint); self } - pub fn with_start_version(mut self, version: Version) -> Self { + pub(crate) 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 { + pub(crate) fn with_end_version(mut self, version: Version) -> Self { let _ = self.end_version.insert(version); self } - pub fn with_no_checkpoint_files(mut self) -> Self { + pub(crate) fn with_no_checkpoint_files(mut self) -> Self { self.no_checkpoint_files = true; self } - pub fn with_in_order_commit_files(mut self) -> Self { + pub(crate) fn with_in_order_commit_files(mut self) -> Self { self.in_order_commit_files = true; self } - pub fn build(self) -> DeltaResult { + pub(crate) fn build(self) -> DeltaResult { let Self { fs_client, log_root, @@ -249,7 +250,7 @@ impl<'a> LogSegmentBuilder<'a> { .collect(), }) } - pub fn list_log_files_from_version( + pub(crate) fn list_log_files_from_version( fs_client: &dyn FileSystemClient, log_root: &Url, version: Option, @@ -298,7 +299,7 @@ impl<'a> LogSegmentBuilder<'a> { } /// List all log files after a given checkpoint. - pub fn list_log_files_with_checkpoint( + pub(crate) fn list_log_files_with_checkpoint( checkpoint_metadata: &CheckpointMetadata, fs_client: &dyn FileSystemClient, log_root: &Url, @@ -339,7 +340,7 @@ impl<'a> LogSegmentBuilder<'a> { /// List relevant log files. /// /// Relevant files are the max checkpoint found and all subsequent commits. - pub fn list_log_files( + pub(crate) fn list_log_files( fs_client: &dyn FileSystemClient, log_root: &Url, ) -> DeltaResult<(Vec, Vec)> { diff --git a/kernel/src/table_changes/metadata_scanner.rs b/kernel/src/table_changes/metadata_scanner.rs deleted file mode 100644 index 55c765aa..00000000 --- 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 index debf9cf6..c5809aaa 100644 --- a/kernel/src/table_changes/mod.rs +++ b/kernel/src/table_changes/mod.rs @@ -2,24 +2,17 @@ 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, + scan::{get_state_info, state::DvInfo, ColumnType}, + schema::{Schema, SchemaRef, StructType}, snapshot::Snapshot, - DeltaResult, Engine, EngineData, Error, Version, + DeltaResult, Engine, EngineData, Error, ExpressionRef, Version, }; - -mod metadata_scanner; -mod replay_scanner; -mod state; -pub mod table_changes_scan; +use url::Url; pub type TableChangesScanData = (Box, Vec, Arc>); @@ -28,7 +21,8 @@ static CDF_ENABLE_FLAG: &str = "delta.enableChangeDataFeed"; #[derive(Debug)] pub struct TableChanges { pub snapshot: Snapshot, - pub log_segment: LogSegment, + #[allow(unused)] + pub(crate) log_segment: LogSegment, pub schema: Schema, pub version: Version, pub metadata: Metadata, @@ -84,3 +78,92 @@ impl TableChanges { TableChangesScanBuilder::new(self) } } + +/// 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 [`TableChanges`]. + /// + /// 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 [`TableChanges`]. See + /// [`TableChangesScanBuilder::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 [`TableChangesScan`]. + /// + /// 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 + /// [`TableChangesScan`] 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, + }) + } +} + +/// The result of building a [`TableChanges`] scan over a table. This can be used to get a change +/// data feed from the table +#[allow(unused)] +pub struct TableChangesScan { + table_changes: Arc, + logical_schema: SchemaRef, + physical_schema: SchemaRef, + predicate: Option, + all_fields: Vec, + have_partition_cols: bool, +} diff --git a/kernel/src/table_changes/replay_scanner.rs b/kernel/src/table_changes/replay_scanner.rs deleted file mode 100644 index 0de5e6f7..00000000 --- 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 6bd7f6f7..00000000 --- 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 2f522900..00000000 --- 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 3a043b0f34869af35d4f4251870610b82cf23cb7 Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Wed, 6 Nov 2024 16:16:18 -0800 Subject: [PATCH 35/41] Remove change_data testing file --- kernel/examples/change_data/Cargo.toml | 18 -------- kernel/examples/change_data/src/main.rs | 40 ------------------ .../_delta_log/00000000000000000000.json | 4 -- .../_delta_log/00000000000000000001.json | 3 -- ...r_61d16c75-6994-46b7-a15b-8b538852e50e.bin | Bin 45 -> 0 bytes ...4e51-827b-c3d5516560ca-c000.snappy.parquet | Bin 635 -> 0 bytes 6 files changed, 65 deletions(-) delete mode 100644 kernel/examples/change_data/Cargo.toml delete mode 100644 kernel/examples/change_data/src/main.rs delete mode 100644 kernel/examples/change_data/table-with-dv-small/_delta_log/00000000000000000000.json delete mode 100644 kernel/examples/change_data/table-with-dv-small/_delta_log/00000000000000000001.json delete mode 100644 kernel/examples/change_data/table-with-dv-small/deletion_vector_61d16c75-6994-46b7-a15b-8b538852e50e.bin delete mode 100644 kernel/examples/change_data/table-with-dv-small/part-00000-fae5310a-a37d-4e51-827b-c3d5516560ca-c000.snappy.parquet diff --git a/kernel/examples/change_data/Cargo.toml b/kernel/examples/change_data/Cargo.toml deleted file mode 100644 index 5a873054..00000000 --- a/kernel/examples/change_data/Cargo.toml +++ /dev/null @@ -1,18 +0,0 @@ -[package] -name = "change_data" -version = "0.1.0" -edition = "2021" -publish = false - -[dependencies] -arrow-array = { workspace = true } -arrow-schema = { workspace = true } -clap = { version = "4.5", features = ["derive"] } -delta_kernel = { path = "../../../kernel", features = [ - "cloud", - "default-engine", - "developer-visibility", -] } -env_logger = "0.11.3" -url = "2" -itertools = "0.13" diff --git a/kernel/examples/change_data/src/main.rs b/kernel/examples/change_data/src/main.rs deleted file mode 100644 index 20b28f98..00000000 --- a/kernel/examples/change_data/src/main.rs +++ /dev/null @@ -1,40 +0,0 @@ -use std::{collections::HashMap, sync::Arc}; - -use arrow_array::RecordBatch; -use delta_kernel::{ - engine::{ - arrow_data::ArrowEngineData, - default::{executor::tokio::TokioBackgroundExecutor, DefaultEngine}, - }, - scan::ScanResult, - DeltaResult, EngineData, Table, -}; -use itertools::Itertools; - -fn into_record_batch(engine_data: DeltaResult>) -> DeltaResult { - engine_data - .and_then(ArrowEngineData::try_from_engine_data) - .map(Into::into) -} -fn main() -> DeltaResult<()> { - let uri = - "/Users/oussama.saoudi/delta-kernel-rs/kernel/examples/change_data/table-with-dv-small"; - // build a table and get the lastest snapshot from it - let table = Table::try_from_uri(uri)?; - - let engine = DefaultEngine::try_new( - table.location(), - HashMap::::new(), - Arc::new(TokioBackgroundExecutor::new()), - )?; - - let table_changes = table.table_changes(&engine, 0, None)?; - let x = table_changes.into_scan_builder().build()?; - let vec: Vec = x.execute(&engine)?.try_collect()?; - println!("Vec len: {:?}", vec.len()); - for res in vec { - // println!("{:?}", into_record_batch(res.raw_data)?) - } - - Ok(()) -} diff --git a/kernel/examples/change_data/table-with-dv-small/_delta_log/00000000000000000000.json b/kernel/examples/change_data/table-with-dv-small/_delta_log/00000000000000000000.json deleted file mode 100644 index 2d01821b..00000000 --- a/kernel/examples/change_data/table-with-dv-small/_delta_log/00000000000000000000.json +++ /dev/null @@ -1,4 +0,0 @@ -{"commitInfo":{"timestamp":1677811178585,"operation":"WRITE","operationParameters":{"mode":"ErrorIfExists","partitionBy":"[]"},"isolationLevel":"WriteSerializable","isBlindAppend":true,"operationMetrics":{"numFiles":"1","numOutputRows":"10","numOutputBytes":"635"},"engineInfo":"Databricks-Runtime/","txnId":"a6a94671-55ef-450e-9546-b8465b9147de"}} -{"protocol":{"minReaderVersion":3,"minWriterVersion":7,"readerFeatures":["deletionVectors"],"writerFeatures":["deletionVectors"]}} -{"metaData":{"id":"testId","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"value\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":[],"configuration":{"delta.enableChangeDataFeed":"true","delta.enableDeletionVectors":"true","delta.columnMapping.mode":"none"},"createdTime":1677811175819}} -{"add":{"path":"part-00000-fae5310a-a37d-4e51-827b-c3d5516560ca-c000.snappy.parquet","partitionValues":{},"size":635,"modificationTime":1677811178336,"dataChange":true,"stats":"{\"numRecords\":10,\"minValues\":{\"value\":0},\"maxValues\":{\"value\":9},\"nullCount\":{\"value\":0},\"tightBounds\":true}","tags":{"INSERTION_TIME":"1677811178336000","MIN_INSERTION_TIME":"1677811178336000","MAX_INSERTION_TIME":"1677811178336000","OPTIMIZE_TARGET_SIZE":"268435456"}}} diff --git a/kernel/examples/change_data/table-with-dv-small/_delta_log/00000000000000000001.json b/kernel/examples/change_data/table-with-dv-small/_delta_log/00000000000000000001.json deleted file mode 100644 index e5bcdc11..00000000 --- a/kernel/examples/change_data/table-with-dv-small/_delta_log/00000000000000000001.json +++ /dev/null @@ -1,3 +0,0 @@ -{"commitInfo":{"timestamp":1677811194429,"operation":"DELETE","operationParameters":{"predicate":"[\"(spark_catalog.delta.`/tmp/table-with-dv-small`.value IN (0, 9))\"]"},"readVersion":0,"isolationLevel":"WriteSerializable","isBlindAppend":false,"operationMetrics":{"numRemovedFiles":"0","numRemovedBytes":"0","numCopiedRows":"0","numDeletionVectorsAdded":"1","numDeletionVectorsRemoved":"0","numAddedChangeFiles":"0","executionTimeMs":"10364","numDeletedRows":"2","scanTimeMs":"9869","numAddedFiles":"0","numAddedBytes":"0","rewriteTimeMs":"479"},"engineInfo":"Databricks-Runtime/","txnId":"6d9555a2-0e3b-4c15-80c0-d5c3b0cf1277"}} -{"remove":{"path":"part-00000-fae5310a-a37d-4e51-827b-c3d5516560ca-c000.snappy.parquet","deletionTimestamp":1677811194426,"dataChange":true,"extendedFileMetadata":true,"partitionValues":{},"size":635,"tags":{"INSERTION_TIME":"1677811178336000","MIN_INSERTION_TIME":"1677811178336000","MAX_INSERTION_TIME":"1677811178336000","OPTIMIZE_TARGET_SIZE":"268435456"}}} -{"add":{"path":"part-00000-fae5310a-a37d-4e51-827b-c3d5516560ca-c000.snappy.parquet","partitionValues":{},"size":635,"modificationTime":1677811178336,"dataChange":true,"stats":"{\"numRecords\":10,\"minValues\":{\"value\":0},\"maxValues\":{\"value\":9},\"nullCount\":{\"value\":0},\"tightBounds\":false}","tags":{"INSERTION_TIME":"1677811178336000","MIN_INSERTION_TIME":"1677811178336000","MAX_INSERTION_TIME":"1677811178336000","OPTIMIZE_TARGET_SIZE":"268435456"},"deletionVector":{"storageType":"u","pathOrInlineDv":"vBn[lx{q8@P<9BNH/isA","offset":1,"sizeInBytes":36,"cardinality":2}}} diff --git a/kernel/examples/change_data/table-with-dv-small/deletion_vector_61d16c75-6994-46b7-a15b-8b538852e50e.bin b/kernel/examples/change_data/table-with-dv-small/deletion_vector_61d16c75-6994-46b7-a15b-8b538852e50e.bin deleted file mode 100644 index f1a01e661cdcca08ff5d67e7d2de53381980735a..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 45 lcmZQ%U|>+Wc-b+>@u6oHnRZaa}xk;gAOoS z#<6}t%0rSCNt@&m3E|*~#Cc*mq6q?Bjf0sEC&T3yyEjN{9KCms%M}*aS7r27rN~Tj zzA#1W7Im$F+m7qFUCSlpIkYWjA7Cc8`45BN8(r(ouK2DKhm&oqokhKfIaLJUzYWIu zPlHKl678-<$u_Z>3nwp@5?4qBej=(UKN%Q>#iA`8S!W3Kv+Rn6JI+Zl%15 zS5`$GRbJ1F6QviWH~GBwGEAG$c3l+NBa^IBOI45~tF^{Z6NZvi&--82o2)mRFB=fg z4w&qQd5e|0q&+nA%=X0kY0=qF(izCXnGE_3#gJBXUG{z7KkJ-?b)pv?9F9Xjj^mLU lg%~U_#xTV3XgCPMU~!JGNsB^M@u}mw^bwBeg)ZpZ{R3QVoUi}@ From 759f53765c1159e28c2cfa14ffe649374bfa858c Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Wed, 6 Nov 2024 16:20:28 -0800 Subject: [PATCH 36/41] Remove unnecessary changes --- kernel/src/scan/data_skipping.rs | 1 - kernel/src/scan/log_replay.rs | 2 +- kernel/src/scan/mod.rs | 6 +++--- kernel/src/scan/state.rs | 37 ++++++-------------------------- 4 files changed, 10 insertions(+), 36 deletions(-) diff --git a/kernel/src/scan/data_skipping.rs b/kernel/src/scan/data_skipping.rs index 9056dac7..7c5aea37 100644 --- a/kernel/src/scan/data_skipping.rs +++ b/kernel/src/scan/data_skipping.rs @@ -163,7 +163,6 @@ fn as_data_skipping_predicate(expr: &Expr) -> Option { } } -#[derive(Clone)] pub(crate) struct DataSkippingFilter { stats_schema: SchemaRef, select_stats_evaluator: Arc, diff --git a/kernel/src/scan/log_replay.rs b/kernel/src/scan/log_replay.rs index a91a2ee7..f872a8ec 100644 --- a/kernel/src/scan/log_replay.rs +++ b/kernel/src/scan/log_replay.rs @@ -108,7 +108,7 @@ pub(crate) static SCAN_ROW_SCHEMA: LazyLock> = LazyLock::new(|| ), ])) }); -pub static SCAN_ROW_DATATYPE: LazyLock = LazyLock::new(|| SCAN_ROW_SCHEMA.clone().into()); +static SCAN_ROW_DATATYPE: LazyLock = LazyLock::new(|| SCAN_ROW_SCHEMA.clone().into()); impl LogReplayScanner { /// Create a new [`LogReplayScanner`] instance diff --git a/kernel/src/scan/mod.rs b/kernel/src/scan/mod.rs index 280ef064..fa28c5aa 100644 --- a/kernel/src/scan/mod.rs +++ b/kernel/src/scan/mod.rs @@ -19,7 +19,7 @@ use crate::{DeltaResult, Engine, EngineData, Error, FileMeta}; use self::log_replay::scan_action_iter; use self::state::GlobalScanState; -pub mod data_skipping; +mod data_skipping; pub mod log_replay; pub mod state; @@ -123,7 +123,7 @@ pub struct ScanResult { pub raw_data: DeltaResult>, /// Raw row mask. // TODO(nick) this should be allocated by the engine - pub raw_mask: Option>, + raw_mask: Option>, } impl ScanResult { @@ -440,7 +440,7 @@ pub fn selection_vector( /// Transform the raw data read from parquet into the correct logical form, based on the provided /// global scan state and partition values -pub fn transform_to_logical( +fn transform_to_logical( engine: &dyn Engine, data: Box, global_state: &GlobalScanState, diff --git a/kernel/src/scan/state.rs b/kernel/src/scan/state.rs index 849bc896..2dc30f23 100644 --- a/kernel/src/scan/state.rs +++ b/kernel/src/scan/state.rs @@ -12,7 +12,6 @@ use crate::{ schema::SchemaRef, DataVisitor, DeltaResult, Engine, EngineData, Error, }; -use roaring::RoaringTreemap; use serde::{Deserialize, Serialize}; use tracing::warn; @@ -29,7 +28,7 @@ pub struct GlobalScanState { } /// this struct can be used by an engine to materialize a selection vector -#[derive(Debug, Clone)] +#[derive(Debug)] pub struct DvInfo { pub(crate) deletion_vector: Option, } @@ -52,43 +51,19 @@ impl DvInfo { self.deletion_vector.is_some() } - pub fn as_dv_tree_map( + pub fn get_selection_vector( &self, engine: &dyn Engine, table_root: &url::Url, - ) -> DeltaResult> { - self.deletion_vector + ) -> DeltaResult>> { + let dv_treemap = self + .deletion_vector .as_ref() .map(|dv_descriptor| { let fs_client = engine.get_file_system_client(); dv_descriptor.read(fs_client, table_root) }) - .transpose() - } - - pub fn get_diff_selection_vector( - &self, - other: &DvInfo, - engine: &dyn Engine, - table_root: &url::Url, - ) -> DeltaResult>> { - if let (Some(self_tm), Some(other_tm)) = ( - self.as_dv_tree_map(engine, table_root)?, - other.as_dv_tree_map(engine, table_root)?, - ) { - let diff = self_tm - other_tm; - Ok(Some(treemap_to_bools(diff))) - } else { - Ok(None) - } - } - - pub fn get_selection_vector( - &self, - engine: &dyn Engine, - table_root: &url::Url, - ) -> DeltaResult>> { - let dv_treemap = self.as_dv_tree_map(engine, table_root)?; + .transpose()?; Ok(dv_treemap.map(treemap_to_bools)) } From 42896ecc3ed90c5dd1faacc79d3a14447db46155 Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Wed, 6 Nov 2024 16:24:35 -0800 Subject: [PATCH 37/41] Fix comments --- kernel/src/table_changes/mod.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/kernel/src/table_changes/mod.rs b/kernel/src/table_changes/mod.rs index c5809aaa..c2b593b7 100644 --- a/kernel/src/table_changes/mod.rs +++ b/kernel/src/table_changes/mod.rs @@ -79,7 +79,7 @@ impl TableChanges { } } -/// Builder to scan a snapshot of a table. +/// Builder to read the `TableChanges` of a table. pub struct TableChangesScanBuilder { table_changes: Arc, schema: Option, @@ -87,7 +87,7 @@ pub struct TableChangesScanBuilder { } impl TableChangesScanBuilder { - /// Create a new [`ScanBuilder`] instance. + /// Create a new [`TableChangesScanBuilder`] instance. pub fn new(table_changes: impl Into>) -> Self { Self { table_changes: table_changes.into(), @@ -102,7 +102,7 @@ impl TableChangesScanBuilder { /// two columns by using the schema `[a, b]`. /// /// [`Schema`]: crate::schema::Schema - /// [`Snapshot`]: crate::snapshot::Snapshot + /// [`TableChanges`]: crate::table_changes:TableChanges: pub fn with_schema(mut self, schema: SchemaRef) -> Self { self.schema = Some(schema); self From c5587154659b7e6b40925431cb2b2560940098ef Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Wed, 6 Nov 2024 18:55:59 -0800 Subject: [PATCH 38/41] Add preliminary test for table changes --- kernel/src/table_changes/mod.rs | 22 ++++++++++++++++++++++ 1 file changed, 22 insertions(+) diff --git a/kernel/src/table_changes/mod.rs b/kernel/src/table_changes/mod.rs index c2b593b7..fcf22e27 100644 --- a/kernel/src/table_changes/mod.rs +++ b/kernel/src/table_changes/mod.rs @@ -41,6 +41,8 @@ impl TableChanges { 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)?; + println!("Start snapshot: {:?}", start_snapshot); + println!("End snapshot: {:?}", end_snapshot); let start_flag = start_snapshot.metadata().configuration.get(CDF_ENABLE_FLAG); let end_flag = end_snapshot.metadata().configuration.get(CDF_ENABLE_FLAG); @@ -51,6 +53,8 @@ impl TableChanges { return Err(Error::TableChangesDisabled(start_version, end_version)); } + println!("Validated flags"); + // 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); @@ -63,6 +67,7 @@ impl TableChanges { .with_in_order_commit_files(); let log_segment = builder.build()?; + println!("Built log segment"); Ok(TableChanges { snapshot: start_snapshot, log_segment, @@ -167,3 +172,20 @@ pub struct TableChangesScan { all_fields: Vec, have_partition_cols: bool, } + +#[cfg(test)] +mod tests { + use crate::{engine::sync::SyncEngine, Table}; + + #[test] + fn get_valid_cdf_ranges() { + let path = "./tests/data/table-with-cdf"; + let engine = Box::new(SyncEngine::new()); + let table = Table::try_from_uri(path).unwrap(); + let start_version = 0; + let end_version = Some(1); + table + .table_changes(engine.as_ref(), start_version, end_version) + .unwrap(); + } +} From 44db9ea894eef38465ce3011bec7d2cd7d444243 Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Wed, 6 Nov 2024 19:13:47 -0800 Subject: [PATCH 39/41] fix failing test --- kernel/src/log_segment.rs | 30 ++++++++++++++---------------- kernel/src/scan/mod.rs | 2 +- 2 files changed, 15 insertions(+), 17 deletions(-) diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index 3bea8f2a..0cccf184 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -197,39 +197,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, @@ -237,6 +229,12 @@ 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 { version: version_eff, log_root: log_url, diff --git a/kernel/src/scan/mod.rs b/kernel/src/scan/mod.rs index fa28c5aa..32944a5e 100644 --- a/kernel/src/scan/mod.rs +++ b/kernel/src/scan/mod.rs @@ -440,7 +440,7 @@ pub fn selection_vector( /// Transform the raw data read from parquet into the correct logical form, based on the provided /// global scan state and partition values -fn transform_to_logical( +pub fn transform_to_logical( engine: &dyn Engine, data: Box, global_state: &GlobalScanState, From ab672e66ef3a0ad7d16bc436c38f4e041312168a Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Thu, 7 Nov 2024 09:38:53 -0800 Subject: [PATCH 40/41] Add test for valid and invalid cdf ranges --- kernel/src/table_changes/mod.rs | 36 +++++++++++++++++++++------------ 1 file changed, 23 insertions(+), 13 deletions(-) diff --git a/kernel/src/table_changes/mod.rs b/kernel/src/table_changes/mod.rs index fcf22e27..6d17c797 100644 --- a/kernel/src/table_changes/mod.rs +++ b/kernel/src/table_changes/mod.rs @@ -41,20 +41,16 @@ impl TableChanges { 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)?; - println!("Start snapshot: {:?}", start_snapshot); - println!("End snapshot: {:?}", end_snapshot); 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) { + let is_cdf_enabled = |flag_res: Option<&String>| flag_res.is_some_and(|val| val == "true"); + if !is_cdf_enabled(start_flag) || !is_cdf_enabled(end_flag) { return Err(Error::TableChangesDisabled(start_version, end_version)); } - println!("Validated flags"); - // 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); @@ -67,7 +63,6 @@ impl TableChanges { .with_in_order_commit_files(); let log_segment = builder.build()?; - println!("Built log segment"); Ok(TableChanges { snapshot: start_snapshot, log_segment, @@ -175,17 +170,32 @@ pub struct TableChangesScan { #[cfg(test)] mod tests { + use crate::Error; use crate::{engine::sync::SyncEngine, Table}; #[test] - fn get_valid_cdf_ranges() { + fn get_cdf_ranges() { let path = "./tests/data/table-with-cdf"; let engine = Box::new(SyncEngine::new()); let table = Table::try_from_uri(path).unwrap(); - let start_version = 0; - let end_version = Some(1); - table - .table_changes(engine.as_ref(), start_version, end_version) - .unwrap(); + + let valid_ranges = [(0, Some(1)), (0, Some(0)), (1, Some(1))]; + for (start_version, end_version) in valid_ranges { + assert!(table + .table_changes(engine.as_ref(), start_version, end_version) + .is_ok()) + } + + let invalid_ranges = [ + (0, None), + (0, Some(2)), + (1, Some(2)), + (2, None), + (2, Some(2)), + ]; + for (start_version, end_version) in invalid_ranges { + let res = table.table_changes(engine.as_ref(), start_version, end_version); + assert!(matches!(res, Err(Error::TableChangesDisabled(_, _)))) + } } } From 6241ed15770b9ade50f2e9160e9f917f4da778bf Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Thu, 7 Nov 2024 11:24:43 -0800 Subject: [PATCH 41/41] address pr comments --- kernel/src/log_segment.rs | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index 0cccf184..509ab632 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -165,7 +165,7 @@ impl<'a> LogSegmentBuilder<'a> { self } pub(crate) fn with_end_version(mut self, version: Version) -> Self { - let _ = self.end_version.insert(version); + self.end_version = Some(version); self } pub(crate) fn with_no_checkpoint_files(mut self) -> Self { @@ -231,7 +231,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(); }