From a2fc20fb22cfd9fd5f2e8fcdc6e47cf0608406f8 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Sun, 7 Jan 2024 16:35:43 +0100 Subject: [PATCH] feat: use EagerSnapshot --- crates/benchmarks/src/bin/merge.rs | 4 +- .../tests/integration_s3_dynamodb.rs | 2 +- .../src/delta_datafusion/expr.rs | 6 +- .../src/delta_datafusion/mod.rs | 72 ++-- crates/deltalake-core/src/errors.rs | 3 + .../src/kernel/snapshot/log_segment.rs | 192 +++++---- .../deltalake-core/src/kernel/snapshot/mod.rs | 232 ++++++++++- .../src/kernel/snapshot/parse.rs | 66 ++- .../src/kernel/snapshot/serde.rs | 102 ++--- crates/deltalake-core/src/lib.rs | 71 ++-- .../src/operations/constraints.rs | 9 +- .../src/operations/convert_to_delta.rs | 3 +- .../deltalake-core/src/operations/create.rs | 14 +- .../deltalake-core/src/operations/delete.rs | 30 +- .../src/operations/filesystem_check.rs | 10 +- crates/deltalake-core/src/operations/load.rs | 2 +- .../src/operations/merge/mod.rs | 79 ++-- crates/deltalake-core/src/operations/mod.rs | 25 +- .../deltalake-core/src/operations/optimize.rs | 27 +- .../deltalake-core/src/operations/restore.rs | 17 +- .../transaction/conflict_checker.rs | 5 +- .../src/operations/transaction/mod.rs | 17 +- .../src/operations/transaction/protocol.rs | 124 +++--- .../src/operations/transaction/state.rs | 36 +- .../src/operations/transaction/test_utils.rs | 2 +- .../deltalake-core/src/operations/update.rs | 36 +- .../deltalake-core/src/operations/vacuum.rs | 46 ++- crates/deltalake-core/src/operations/write.rs | 203 +++++----- .../src/protocol/checkpoints.rs | 32 +- crates/deltalake-core/src/protocol/mod.rs | 20 +- crates/deltalake-core/src/table/builder.rs | 32 +- crates/deltalake-core/src/table/mod.rs | 357 ++++++---------- crates/deltalake-core/src/table/state.rs | 382 +++++------------- .../deltalake-core/src/table/state_arrow.rs | 58 ++- crates/deltalake-core/src/writer/mod.rs | 5 +- .../deltalake-core/src/writer/test_utils.rs | 3 +- .../deltalake-core/tests/checkpoint_writer.rs | 19 +- .../tests/command_filesystem_check.rs | 29 +- .../deltalake-core/tests/command_optimize.rs | 26 +- .../deltalake-core/tests/command_restore.rs | 16 +- .../tests/commit_info_format.rs | 4 +- crates/deltalake-core/tests/fs_common/mod.rs | 2 +- crates/deltalake-core/tests/integration.rs | 12 +- .../tests/integration_datafusion.rs | 34 +- .../tests/read_delta_log_test.rs | 4 +- crates/deltalake-test/src/concurrent.rs | 8 +- crates/deltalake-test/src/lib.rs | 4 +- crates/deltalake-test/src/read.rs | 22 +- ...-b8b4-bef3de0de409-c000.snappy.parquet.crc | Bin 12 -> 0 bytes ...-bc07-d46c948aa415-c000.snappy.parquet.crc | Bin 12 -> 0 bytes ...-8606-f8311a96f6dc-c000.snappy.parquet.crc | Bin 12 -> 0 bytes ...-ad28-ff32ddab96d2-c000.snappy.parquet.crc | Bin 12 -> 0 bytes ...-ae49-fc48b973e37e-c000.snappy.parquet.crc | Bin 12 -> 0 bytes ...-9507-7ccf67924649-c000.snappy.parquet.crc | Bin 12 -> 0 bytes ...-815e-7eb62007a15c-c000.snappy.parquet.crc | Bin 12 -> 0 bytes ...-bb0f-15cde3fb14eb-c000.snappy.parquet.crc | Bin 12 -> 0 bytes ...-98c1-7a69872fd797-c000.snappy.parquet.crc | Bin 12 -> 0 bytes ...-bcfd-7de5788dfe8d-c000.snappy.parquet.crc | Bin 12 -> 0 bytes ...-a195-5f1ae583e553-c000.snappy.parquet.crc | Bin 12 -> 0 bytes ...-99e6-23f1ac2b7b7c-c000.snappy.parquet.crc | Bin 12 -> 0 bytes ...-b3f6-9f256992c633-c000.snappy.parquet.crc | Bin 12 -> 0 bytes ...-a411-46d4295da531-c000.snappy.parquet.crc | Bin 12 -> 0 bytes ...-a05e-8032113a6568-c000.snappy.parquet.crc | Bin 12 -> 0 bytes ...-a9da-7c6f53f6406b-c000.snappy.parquet.crc | Bin 12 -> 0 bytes ...-bc1f-de9bd8ae025b-c000.snappy.parquet.crc | Bin 12 -> 0 bytes ...-834e-dcc098fc9005-c000.snappy.parquet.crc | Bin 12 -> 0 bytes ...0000000000000000010.checkpoint.parquet.crc | Bin 96 -> 0 bytes delta-inspect/src/main.rs | 4 +- python/src/lib.rs | 104 +++-- 69 files changed, 1338 insertions(+), 1274 deletions(-) delete mode 100644 crates/deltalake-test/tests/data/delta-0.2.0/.part-00000-512e1537-8aaa-4193-b8b4-bef3de0de409-c000.snappy.parquet.crc delete mode 100644 crates/deltalake-test/tests/data/delta-0.2.0/.part-00000-7c2deba3-1994-4fb8-bc07-d46c948aa415-c000.snappy.parquet.crc delete mode 100644 crates/deltalake-test/tests/data/delta-0.2.0/.part-00000-b44fcdb0-8b06-4f3a-8606-f8311a96f6dc-c000.snappy.parquet.crc delete mode 100644 crates/deltalake-test/tests/data/delta-0.2.0/.part-00000-cb6b150b-30b8-4662-ad28-ff32ddab96d2-c000.snappy.parquet.crc delete mode 100644 crates/deltalake-test/tests/data/delta-0.2.0/.part-00001-185eca06-e017-4dea-ae49-fc48b973e37e-c000.snappy.parquet.crc delete mode 100644 crates/deltalake-test/tests/data/delta-0.2.0/.part-00001-4327c977-2734-4477-9507-7ccf67924649-c000.snappy.parquet.crc delete mode 100644 crates/deltalake-test/tests/data/delta-0.2.0/.part-00001-c373a5bd-85f0-4758-815e-7eb62007a15c-c000.snappy.parquet.crc delete mode 100644 crates/deltalake-test/tests/data/simple_table_with_checkpoint/.part-00000-136c36f5-639d-4e95-bb0f-15cde3fb14eb-c000.snappy.parquet.crc delete mode 100644 crates/deltalake-test/tests/data/simple_table_with_checkpoint/.part-00000-1abe25d3-0da6-46c5-98c1-7a69872fd797-c000.snappy.parquet.crc delete mode 100644 crates/deltalake-test/tests/data/simple_table_with_checkpoint/.part-00000-3810fbe0-9892-431d-bcfd-7de5788dfe8d-c000.snappy.parquet.crc delete mode 100644 crates/deltalake-test/tests/data/simple_table_with_checkpoint/.part-00000-3fa65c69-4e55-4b18-a195-5f1ae583e553-c000.snappy.parquet.crc delete mode 100644 crates/deltalake-test/tests/data/simple_table_with_checkpoint/.part-00000-72ecc4d6-2e44-4df4-99e6-23f1ac2b7b7c-c000.snappy.parquet.crc delete mode 100644 crates/deltalake-test/tests/data/simple_table_with_checkpoint/.part-00000-7d239c98-d74b-4b02-b3f6-9f256992c633-c000.snappy.parquet.crc delete mode 100644 crates/deltalake-test/tests/data/simple_table_with_checkpoint/.part-00000-8e7dc8c1-337b-40b8-a411-46d4295da531-c000.snappy.parquet.crc delete mode 100644 crates/deltalake-test/tests/data/simple_table_with_checkpoint/.part-00000-9afd9224-729f-4420-a05e-8032113a6568-c000.snappy.parquet.crc delete mode 100644 crates/deltalake-test/tests/data/simple_table_with_checkpoint/.part-00000-e93060ad-9c8c-4170-a9da-7c6f53f6406b-c000.snappy.parquet.crc delete mode 100644 crates/deltalake-test/tests/data/simple_table_with_checkpoint/.part-00000-e9c6df9a-e585-4c70-bc1f-de9bd8ae025b-c000.snappy.parquet.crc delete mode 100644 crates/deltalake-test/tests/data/simple_table_with_checkpoint/.part-00000-f0e955c5-a1e3-4eec-834e-dcc098fc9005-c000.snappy.parquet.crc delete mode 100644 crates/deltalake-test/tests/data/simple_table_with_checkpoint/_delta_log/.00000000000000000010.checkpoint.parquet.crc diff --git a/crates/benchmarks/src/bin/merge.rs b/crates/benchmarks/src/bin/merge.rs index e6abebc5ca..d174154e38 100644 --- a/crates/benchmarks/src/bin/merge.rs +++ b/crates/benchmarks/src/bin/merge.rs @@ -193,10 +193,10 @@ async fn benchmark_merge_tpcds( merge: fn(DataFrame, DeltaTable) -> Result, ) -> Result<(core::time::Duration, MergeMetrics), DataFusionError> { let table = DeltaTableBuilder::from_uri(path).load().await?; - let file_count = table.state.files().len(); + let file_count = table.snapshot()?.files()?.len(); let provider = DeltaTableProvider::try_new( - table.state.clone(), + table.snapshot()?.clone(), table.log_store(), DeltaScanConfig { file_column_name: Some("file_path".to_string()), diff --git a/crates/deltalake-aws/tests/integration_s3_dynamodb.rs b/crates/deltalake-aws/tests/integration_s3_dynamodb.rs index 502607e868..4b878aee6b 100644 --- a/crates/deltalake-aws/tests/integration_s3_dynamodb.rs +++ b/crates/deltalake-aws/tests/integration_s3_dynamodb.rs @@ -322,7 +322,7 @@ async fn append_to_table( table.log_store().as_ref(), &actions, operation, - &table.state, + Some(table.snapshot()?), metadata, ) .await diff --git a/crates/deltalake-core/src/delta_datafusion/expr.rs b/crates/deltalake-core/src/delta_datafusion/expr.rs index 347d093658..03849f4df9 100644 --- a/crates/deltalake-core/src/delta_datafusion/expr.rs +++ b/crates/deltalake-core/src/delta_datafusion/expr.rs @@ -575,7 +575,8 @@ mod test { .cast_to::( &arrow_schema::DataType::Utf8, &table - .state + .snapshot() + .unwrap() .input_schema() .unwrap() .as_ref() @@ -612,7 +613,8 @@ mod test { assert_eq!(test.expected, actual); let actual_expr = table - .state + .snapshot() + .unwrap() .parse_predicate_expression(actual, &session.state()) .unwrap(); diff --git a/crates/deltalake-core/src/delta_datafusion/mod.rs b/crates/deltalake-core/src/delta_datafusion/mod.rs index 0b46ecc2fa..f05ad46700 100644 --- a/crates/deltalake-core/src/delta_datafusion/mod.rs +++ b/crates/deltalake-core/src/delta_datafusion/mod.rs @@ -158,7 +158,7 @@ impl DeltaTableState { let mut downgrade = false; let unknown_stats = Statistics::new_unknown(&schema); - let files = self.files(); + let files = self.files()?; // Initalize statistics let mut table_stats = match files.first() { @@ -269,8 +269,8 @@ fn get_prune_stats(table: &DeltaTable, column: &Column, get_max: bool) -> Option let data_type = field.data_type().try_into().ok()?; let partition_columns = &table.metadata().ok()?.partition_columns; - - let values = table.get_state().files().iter().map(|add| { + let files = table.snapshot().ok()?.files().ok()?; + let values = files.iter().map(|add| { if partition_columns.contains(&column.name) { let value = add.partition_values.get(&column.name).unwrap(); let value = match value { @@ -308,6 +308,7 @@ fn get_prune_stats(table: &DeltaTable, column: &Column, get_max: bool) -> Option ScalarValue::iter_to_array(values).ok() } +// TODO only implement this for Snapshot, not for DeltaTable impl PruningStatistics for DeltaTable { /// return the minimum values for the named column, if known. /// Note: the returned array must contain `num_containers()` rows @@ -324,7 +325,7 @@ impl PruningStatistics for DeltaTable { /// return the number of containers (e.g. row groups) being /// pruned with these statistics fn num_containers(&self) -> usize { - self.get_state().files().len() + self.get_state().unwrap().files().unwrap().len() } /// return the number of null values for the named column as an @@ -333,8 +334,8 @@ impl PruningStatistics for DeltaTable { /// Note: the returned array must contain `num_containers()` rows. fn null_counts(&self, column: &Column) -> Option { let partition_columns = &self.metadata().ok()?.partition_columns; - - let values = self.get_state().files().iter().map(|add| { + let files = self.snapshot().ok()?.files().ok()?; + let values = files.iter().map(|add| { if let Ok(Some(statistics)) = add.get_stats() { if partition_columns.contains(&column.name) { let value = add.partition_values.get(&column.name).unwrap(); @@ -564,7 +565,7 @@ impl<'a> DeltaScanBuilder<'a> { PruningPredicate::try_new(predicate.clone(), logical_schema.clone())?; let files_to_prune = pruning_predicate.prune(self.snapshot)?; self.snapshot - .files() + .files()? .iter() .zip(files_to_prune.into_iter()) .filter_map( @@ -578,7 +579,7 @@ impl<'a> DeltaScanBuilder<'a> { ) .collect() } else { - self.snapshot.files().to_owned() + self.snapshot.files()? } } }; @@ -588,7 +589,7 @@ impl<'a> DeltaScanBuilder<'a> { // However we may want to do some additional balancing in case we are far off from the above. let mut file_groups: HashMap, Vec> = HashMap::new(); - let table_partition_cols = &self.snapshot.metadata()?.partition_columns; + let table_partition_cols = &self.snapshot.metadata().partition_columns; for action in files.iter() { let mut part = partitioned_file_from_action(action, table_partition_cols, &schema); @@ -666,6 +667,7 @@ impl<'a> DeltaScanBuilder<'a> { } } +// TODO: implement this for Snapshot, not for DeltaTable #[async_trait] impl TableProvider for DeltaTable { fn as_any(&self) -> &dyn Any { @@ -673,7 +675,7 @@ impl TableProvider for DeltaTable { } fn schema(&self) -> Arc { - self.state.arrow_schema().unwrap() + self.snapshot().unwrap().arrow_schema().unwrap() } fn table_type(&self) -> TableType { @@ -698,7 +700,7 @@ impl TableProvider for DeltaTable { register_store(self.log_store(), session.runtime_env().clone()); let filter_expr = conjunction(filters.iter().cloned()); - let scan = DeltaScanBuilder::new(&self.state, self.log_store(), session) + let scan = DeltaScanBuilder::new(self.snapshot()?, self.log_store(), session) .with_projection(projection) .with_limit(limit) .with_filter(filter_expr) @@ -716,7 +718,7 @@ impl TableProvider for DeltaTable { } fn statistics(&self) -> Option { - self.state.datafusion_table_statistics().ok() + self.get_state()?.datafusion_table_statistics().ok() } } @@ -1065,6 +1067,15 @@ pub struct DeltaDataChecker { } impl DeltaDataChecker { + /// Create a new DeltaDataChecker with no invariants or constraints + pub fn empty() -> Self { + Self { + invariants: vec![], + constraints: vec![], + ctx: DeltaSessionContext::default().into(), + } + } + /// Create a new DeltaDataChecker with a specified set of invariants pub fn new_with_invariants(invariants: Vec) -> Self { Self { @@ -1091,13 +1102,7 @@ impl DeltaDataChecker { /// Create a new DeltaDataChecker pub fn new(snapshot: &DeltaTableState) -> Self { - // TODO remove unwrap - let invariants = if let Ok(m) = snapshot.metadata() { - m.schema().unwrap().get_invariants().unwrap() - } else { - vec![] - }; - + let invariants = snapshot.schema().get_invariants().unwrap_or_default(); let constraints = snapshot.table_config().get_constraints(); Self { invariants, @@ -1418,7 +1423,7 @@ pub(crate) async fn find_files_scan<'a>( expression: Expr, ) -> DeltaResult> { let candidate_map: HashMap = snapshot - .files() + .files()? .iter() .map(|add| (add.path.clone(), add.to_owned())) .collect(); @@ -1478,7 +1483,7 @@ pub(crate) async fn scan_memory_table( snapshot: &DeltaTableState, predicate: &Expr, ) -> DeltaResult> { - let actions = snapshot.files().to_owned(); + let actions = snapshot.files()?; let batch = snapshot.add_actions_table(true)?; let mut arrays = Vec::new(); @@ -1535,7 +1540,7 @@ pub async fn find_files<'a>( state: &SessionState, predicate: Option, ) -> DeltaResult { - let current_metadata = snapshot.metadata()?; + let current_metadata = snapshot.metadata(); match &predicate { Some(predicate) => { @@ -1566,7 +1571,7 @@ pub async fn find_files<'a>( } } None => Ok(FindFiles { - candidates: snapshot.files().to_owned(), + candidates: snapshot.files()?, partition_scan: true, }), } @@ -1912,11 +1917,13 @@ mod tests { .unwrap(); let config = DeltaScanConfigBuilder::new() .with_file_column_name(&"file_source") - .build(&table.state) + .build(&table.snapshot().unwrap()) .unwrap(); let log_store = table.log_store(); - let provider = DeltaTableProvider::try_new(table.state, log_store, config).unwrap(); + let provider = + DeltaTableProvider::try_new(table.snapshot().unwrap().clone(), log_store, config) + .unwrap(); let ctx = SessionContext::new(); ctx.register_table("test", Arc::new(provider)).unwrap(); @@ -1973,10 +1980,14 @@ mod tests { .await .unwrap(); - let config = DeltaScanConfigBuilder::new().build(&table.state).unwrap(); + let config = DeltaScanConfigBuilder::new() + .build(table.snapshot().unwrap()) + .unwrap(); let log_store = table.log_store(); - let provider = DeltaTableProvider::try_new(table.state, log_store, config).unwrap(); + let provider = + DeltaTableProvider::try_new(table.snapshot().unwrap().clone(), log_store, config) + .unwrap(); let ctx = SessionContext::new(); ctx.register_table("test", Arc::new(provider)).unwrap(); @@ -2024,10 +2035,13 @@ mod tests { .await .unwrap(); - let config = DeltaScanConfigBuilder::new().build(&table.state).unwrap(); + let config = DeltaScanConfigBuilder::new() + .build(table.snapshot().unwrap()) + .unwrap(); let log = table.log_store(); - let provider = DeltaTableProvider::try_new(table.state, log, config).unwrap(); + let provider = + DeltaTableProvider::try_new(table.snapshot().unwrap().clone(), log, config).unwrap(); let ctx: SessionContext = DeltaSessionContext::default().into(); ctx.register_table("test", Arc::new(provider)).unwrap(); diff --git a/crates/deltalake-core/src/errors.rs b/crates/deltalake-core/src/errors.rs index 67963042f8..92f9a88d67 100644 --- a/crates/deltalake-core/src/errors.rs +++ b/crates/deltalake-core/src/errors.rs @@ -210,6 +210,9 @@ pub enum DeltaTableError { #[error("Table metadata is invalid: {0}")] MetadataError(String), + + #[error("Table has not yet been initialized")] + NotInitialized, } impl From for DeltaTableError { diff --git a/crates/deltalake-core/src/kernel/snapshot/log_segment.rs b/crates/deltalake-core/src/kernel/snapshot/log_segment.rs index 1df2922b67..bbf565e5cf 100644 --- a/crates/deltalake-core/src/kernel/snapshot/log_segment.rs +++ b/crates/deltalake-core/src/kernel/snapshot/log_segment.rs @@ -4,9 +4,7 @@ use std::io::{BufRead, BufReader, Cursor}; use std::sync::Arc; use std::task::{ready, Poll}; -use arrow_array::{ - Array, Int32Array, Int64Array, ListArray, RecordBatch, StringArray, StructArray, -}; +use arrow_array::{Array, Int32Array, ListArray, RecordBatch, StringArray, StructArray}; use arrow_json::{reader::Decoder, ReaderBuilder}; use arrow_schema::SchemaRef as ArrowSchemaRef; use bytes::{Buf, Bytes}; @@ -23,15 +21,16 @@ use parquet::arrow::async_reader::{ParquetObjectReader, ParquetRecordBatchStream use regex::Regex; use serde::{Deserialize, Serialize}; use serde_json::Value; +use tracing::debug; use crate::kernel::schema::Schema; -use crate::kernel::snapshot::extract::{extract_and_cast, extract_and_cast_opt}; use crate::kernel::{Action, ActionType, Metadata, Protocol, StructType}; use crate::operations::transaction::get_commit_bytes; use crate::protocol::DeltaOperation; use crate::{DeltaResult, DeltaTableConfig, DeltaTableError}; -use super::extract::{read_primitive, read_primitive_opt, read_str, read_str_opt}; +use super::extract::{extract_and_cast, extract_and_cast_opt, read_primitive}; +use super::parse::read_metadata; const LAST_CHECKPOINT_FILE_NAME: &str = "_last_checkpoint"; const BATCH_SIZE: usize = 1024; @@ -52,7 +51,7 @@ lazy_static! { StructType::new(vec![ActionType::Remove.schema_field().clone(),]); } -trait PathExt { +pub(super) trait PathExt { fn child(&self, path: impl AsRef) -> DeltaResult; /// Returns the last path segment if not terminated with a "/" fn filename(&self) -> Option<&str>; @@ -110,7 +109,7 @@ struct FileInfo { version: Option, } -#[derive(Debug, Clone)] +#[derive(Debug, Clone, PartialEq)] pub(super) struct LogSegment { pub(super) version: i64, pub(super) commit_files: VecDeque, @@ -133,7 +132,7 @@ impl LogSegment { (Some(cp), Some(v)) if cp.version <= v => { list_log_files_with_checkpoint(&cp, store, &log_url).await? } - _ => list_log_files(store, &log_url, version).await?, + _ => list_log_files(store, &log_url, version, None).await?, }; // remove all files above requested version @@ -141,14 +140,20 @@ impl LogSegment { commit_files.retain(|meta| meta.location.commit_version() <= Some(version)); } + let mut segment = Self { + version: 0, + commit_files: commit_files.into(), + checkpoint_files, + }; + if segment.commit_files.is_empty() && segment.checkpoint_files.is_empty() { + return Err(DeltaTableError::NotATable("no log files".into())); + } // get the effective version from chosen files - let version_eff = commit_files - .first() - .or(checkpoint_files.first()) - .and_then(|f| f.location.commit_version()) - .ok_or(DeltaTableError::Generic( - "failed to get effective version".into(), - ))?; // TODO: A more descriptive error + let version_eff = segment.file_version().ok_or(DeltaTableError::Generic( + "failed to get effective version".into(), + ))?; // TODO: A more descriptive error + segment.version = version_eff; + segment.validate()?; if let Some(v) = version { if version_eff != v { @@ -157,11 +162,64 @@ impl LogSegment { } } - Ok(Self { - version: version_eff, + Ok(segment) + } + + pub async fn try_new_slice( + table_root: &Path, + start_version: i64, + end_version: Option, + store: &dyn ObjectStore, + ) -> DeltaResult { + debug!( + "try_new_slice: start_version: {}, end_version: {:?}", + start_version, end_version + ); + let log_url = table_root.child("_delta_log"); + let (mut commit_files, checkpoint_files) = + list_log_files(store, &log_url, end_version, Some(start_version)).await?; + // remove all files above requested version + if let Some(version) = end_version { + commit_files.retain(|meta| meta.location.commit_version() <= Some(version)); + } + let mut segment = Self { + version: start_version, commit_files: commit_files.into(), checkpoint_files, - }) + }; + segment.version = segment + .file_version() + .unwrap_or(end_version.unwrap_or(start_version)); + Ok(segment) + } + + pub fn validate(&self) -> DeltaResult<()> { + let checkpoint_version = self + .checkpoint_files + .iter() + .filter_map(|f| f.location.commit_version()) + .max(); + if let Some(v) = checkpoint_version { + if !self + .commit_files + .iter() + .all(|f| f.location.commit_version() > Some(v)) + { + return Err(DeltaTableError::Generic("inconsistent log segment".into())); + } + } + Ok(()) + } + + pub fn file_version(&self) -> Option { + self.commit_files + .iter() + .filter_map(|f| f.location.commit_version()) + .max() + .or(self + .checkpoint_files + .first() + .and_then(|f| f.location.commit_version())) } #[cfg(test)] @@ -188,6 +246,13 @@ impl LogSegment { self.version } + pub fn version_timestamp(&self, version: i64) -> Option> { + self.commit_files + .iter() + .find(|f| f.location.commit_version() == Some(version)) + .map(|f| f.last_modified) + } + pub(super) fn commit_stream( &self, store: Arc, @@ -232,7 +297,7 @@ impl LogSegment { &self, store: Arc, config: &DeltaTableConfig, - ) -> DeltaResult<(Protocol, Metadata)> { + ) -> DeltaResult<(Option, Option)> { lazy_static::lazy_static! { static ref READ_SCHEMA: StructType = StructType::new(vec![ ActionType::Protocol.schema_field().clone(), @@ -257,7 +322,7 @@ impl LogSegment { }; } if maybe_protocol.is_some() && maybe_metadata.is_some() { - return Ok((maybe_protocol.unwrap(), maybe_metadata.unwrap())); + return Ok((maybe_protocol, maybe_metadata)); } } @@ -275,22 +340,11 @@ impl LogSegment { }; } if maybe_protocol.is_some() && maybe_metadata.is_some() { - return Ok((maybe_protocol.unwrap(), maybe_metadata.unwrap())); + return Ok((maybe_protocol, maybe_metadata)); } } - match (maybe_protocol, maybe_metadata) { - (Some(protocol), Some(metadata)) => Ok((protocol, metadata)), - (Some(_), None) => Err(DeltaTableError::Generic( - "Missing metadata action".to_string(), - )), - (None, Some(_)) => Err(DeltaTableError::Generic( - "Missing protocol action".to_string(), - )), - (None, None) => Err(DeltaTableError::Generic( - "Missing protocol and metadata actions".to_string(), - )), - } + Ok((maybe_protocol, maybe_metadata)) } /// Advance the log segment with new commits @@ -332,7 +386,7 @@ impl LogSegment { } } -fn read_protocol(batch: &RecordBatch) -> DeltaResult> { +pub(super) fn read_protocol(batch: &RecordBatch) -> DeltaResult> { if let Some(arr) = extract_and_cast_opt::(batch, "protocol") { let min_reader_version = extract_and_cast::(arr, "minReaderVersion")?; let min_writer_version = extract_and_cast::(arr, "minWriterVersion")?; @@ -377,35 +431,6 @@ fn read_protocol(batch: &RecordBatch) -> DeltaResult> { Ok(None) } -fn read_metadata(batch: &RecordBatch) -> DeltaResult> { - if let Some(arr) = extract_and_cast_opt::(batch, "metaData") { - let id = extract_and_cast::(arr, "id")?; - let name = extract_and_cast::(arr, "name")?; - let description = extract_and_cast::(arr, "description")?; - // let format = extract_and_cast::(arr, "format")?; - let schema_string = extract_and_cast::(arr, "schemaString")?; - // let partition_columns = extract_and_cast::(arr, "partitionColumns")?; - // let configuration = extract_and_cast::(arr, "configuration")?; - let created_time = extract_and_cast::(arr, "createdTime")?; - - for idx in 0..arr.len() { - if arr.is_valid(idx) { - return Ok(Some(Metadata { - id: read_str(id, idx)?.to_string(), - name: read_str_opt(name, idx).map(|s| s.to_string()), - description: read_str_opt(description, idx).map(|s| s.to_string()), - format: Default::default(), - schema_string: read_str(schema_string, idx)?.to_string(), - partition_columns: Default::default(), - configuration: Default::default(), - created_time: read_primitive_opt(created_time, idx), - })); - } - } - } - Ok(None) -} - fn decode_stream> + Unpin>( mut decoder: Decoder, mut input: S, @@ -548,13 +573,14 @@ async fn list_log_files_with_checkpoint( /// List relevant log files. /// /// Relevant files are the max checkpoint found and all subsequent commits. -async fn list_log_files( +pub(super) async fn list_log_files( fs_client: &dyn ObjectStore, log_root: &Path, max_version: Option, + start_version: Option, ) -> DeltaResult<(Vec, Vec)> { - let max_version = max_version.unwrap_or(i64::MAX); - let start_from = log_root.child(format!("{:020}", 0).as_str()); + let max_version = max_version.unwrap_or(i64::MAX - 1); + let start_from = log_root.child(format!("{:020}", start_version.unwrap_or(0)).as_str()); let mut max_checkpoint_version = -1_i64; let mut commit_files = Vec::with_capacity(25); @@ -565,7 +591,9 @@ async fn list_log_files( .try_collect::>() .await? { - if meta.location.commit_version() <= Some(max_version) { + if meta.location.commit_version().unwrap_or(i64::MAX) <= max_version + && meta.location.commit_version() >= start_version + { if meta.location.is_checkpoint_file() { let version = meta.location.commit_version().unwrap_or(0); match version.cmp(&max_checkpoint_version) { @@ -601,6 +629,27 @@ pub(super) mod tests { pub(crate) async fn test_log_segment(context: &IntegrationContext) -> TestResult { read_log_files(context).await?; read_metadata(context).await?; + log_segment_serde(context).await?; + + Ok(()) + } + + async fn log_segment_serde(context: &IntegrationContext) -> TestResult { + let store = context + .table_builder(TestTables::Simple) + .build_storage()? + .object_store(); + + let segment = LogSegment::try_new(&Path::default(), None, store.as_ref()).await?; + let bytes = serde_json::to_vec(&segment).unwrap(); + let actual: LogSegment = serde_json::from_slice(&bytes).unwrap(); + assert_eq!(actual.version(), segment.version()); + assert_eq!(actual.commit_files.len(), segment.commit_files.len()); + assert_eq!( + actual.checkpoint_files.len(), + segment.checkpoint_files.len() + ); + Ok(()) } @@ -620,11 +669,11 @@ pub(super) mod tests { assert_eq!(log.len(), 2); assert_eq!(check.len(), 1); - let (log, check) = list_log_files(store.as_ref(), &log_path, None).await?; + let (log, check) = list_log_files(store.as_ref(), &log_path, None, None).await?; assert_eq!(log.len(), 2); assert_eq!(check.len(), 1); - let (log, check) = list_log_files(store.as_ref(), &log_path, Some(8)).await?; + let (log, check) = list_log_files(store.as_ref(), &log_path, Some(8), None).await?; assert_eq!(log.len(), 3); assert_eq!(check.len(), 1); @@ -643,11 +692,11 @@ pub(super) mod tests { .build_storage()? .object_store(); - let (log, check) = list_log_files(store.as_ref(), &log_path, None).await?; + let (log, check) = list_log_files(store.as_ref(), &log_path, None, None).await?; assert_eq!(log.len(), 5); assert_eq!(check.len(), 0); - let (log, check) = list_log_files(store.as_ref(), &log_path, Some(2)).await?; + let (log, check) = list_log_files(store.as_ref(), &log_path, Some(2), None).await?; assert_eq!(log.len(), 3); assert_eq!(check.len(), 0); @@ -663,6 +712,7 @@ pub(super) mod tests { let (protocol, _metadata) = segment .read_metadata(store.clone(), &Default::default()) .await?; + let protocol = protocol.unwrap(); let expected = Protocol { min_reader_version: 3, diff --git a/crates/deltalake-core/src/kernel/snapshot/mod.rs b/crates/deltalake-core/src/kernel/snapshot/mod.rs index 1cd0dab74d..6d9b2c61c9 100644 --- a/crates/deltalake-core/src/kernel/snapshot/mod.rs +++ b/crates/deltalake-core/src/kernel/snapshot/mod.rs @@ -12,7 +12,7 @@ use futures::{StreamExt, TryStreamExt}; use object_store::path::Path; use object_store::ObjectStore; -use self::log_segment::{CommitData, LogSegment}; +use self::log_segment::{CommitData, LogSegment, PathExt}; use self::parse::{extract_adds, extract_removes}; use self::replay::{LogReplayScanner, ReplayStream}; use super::{Action, Add, CommitInfo, Metadata, Protocol, Remove}; @@ -27,7 +27,7 @@ mod replay; mod serde; /// A snapshot of a Delta table -#[derive(Debug, Serialize, Deserialize, Clone)] +#[derive(Debug, Serialize, Deserialize, Clone, PartialEq)] pub struct Snapshot { log_segment: LogSegment, config: DeltaTableConfig, @@ -49,10 +49,16 @@ impl Snapshot { ) -> DeltaResult { let log_segment = LogSegment::try_new(table_root, version, store.as_ref()).await?; let (protocol, metadata) = log_segment.read_metadata(store.clone(), &config).await?; + if !metadata.is_some() && protocol.is_some() { + return Err(DeltaTableError::Generic( + "Cannot read metadata from log segment".into(), + )); + }; + let metadata = metadata.unwrap(); + let protocol = protocol.unwrap(); let schema = serde_json::from_str(&metadata.schema_string)?; Ok(Self { log_segment, - // store, config, protocol, metadata, @@ -61,6 +67,91 @@ impl Snapshot { }) } + #[cfg(test)] + pub fn new_test<'a>( + commits: impl IntoIterator, + ) -> DeltaResult<(Self, RecordBatch)> { + use arrow_select::concat::concat_batches; + let (log_segment, batches) = LogSegment::new_test(commits)?; + let batch = batches.into_iter().collect::, _>>()?; + let batch = concat_batches(&batch[0].schema(), &batch)?; + let protocol = log_segment::read_protocol(&batch)?.unwrap(); + let metadata = parse::read_metadata(&batch)?.unwrap(); + let schema = serde_json::from_str(&metadata.schema_string)?; + Ok(( + Self { + log_segment, + config: Default::default(), + protocol, + metadata, + schema, + table_url: Path::default().to_string(), + }, + batch, + )) + } + + /// Update the snapshot to the given version + pub async fn update( + &mut self, + store: Arc, + target_version: Option, + ) -> DeltaResult<()> { + self.update_inner(store, target_version).await?; + Ok(()) + } + + async fn update_inner( + &mut self, + store: Arc, + target_version: Option, + ) -> DeltaResult> { + if let Some(version) = target_version { + if version == self.version() { + return Ok(None); + } + if version < self.version() { + return Err(DeltaTableError::Generic( + "Cannoit downgrade snapshot".into(), + )); + } + } + let log_segment = LogSegment::try_new_slice( + &Path::default(), + self.version() + 1, + target_version, + store.as_ref(), + ) + .await?; + if log_segment.commit_files.is_empty() && log_segment.checkpoint_files.is_empty() { + return Ok(None); + } + + let (protocol, metadata) = log_segment + .read_metadata(store.clone(), &self.config) + .await?; + if let Some(protocol) = protocol { + self.protocol = protocol; + } + if let Some(metadata) = metadata { + self.metadata = metadata; + self.schema = serde_json::from_str(&self.metadata.schema_string)?; + } + + if !log_segment.checkpoint_files.is_empty() { + self.log_segment.checkpoint_files = log_segment.checkpoint_files.clone(); + self.log_segment.commit_files = log_segment.commit_files.clone(); + } else { + for file in &log_segment.commit_files { + self.log_segment.commit_files.push_front(file.clone()); + } + } + + self.log_segment.version = log_segment.version; + + Ok(Some(log_segment)) + } + /// Get the table version of the snapshot pub fn version(&self) -> i64 { self.log_segment.version() @@ -110,11 +201,34 @@ impl Snapshot { } /// Get the commit infos in the snapshot - pub(crate) fn commit_infos( + pub(crate) async fn commit_infos( &self, store: Arc, - ) -> BoxStream<'_, DeltaResult>> { - futures::stream::iter(self.log_segment.commit_files.clone()) + limit: Option, + ) -> DeltaResult>>> { + let log_root = self.table_root().child("_delta_log"); + let start_from = log_root.child( + format!( + "{:020}", + limit + .map(|l| (self.version() - l as i64 + 1).max(0)) + .unwrap_or(0) + ) + .as_str(), + ); + + let mut commit_files = Vec::new(); + for meta in store + .list_with_offset(Some(&log_root), &start_from) + .try_collect::>() + .await? + { + if meta.location.is_commit_file() { + commit_files.push(meta); + } + } + commit_files.sort_unstable_by(|a, b| b.location.cmp(&a.location)); + Ok(futures::stream::iter(commit_files) .map(move |meta| { let store = store.clone(); async move { @@ -133,7 +247,7 @@ impl Snapshot { } }) .buffered(self.config.log_buffer_size) - .boxed() + .boxed()) } pub(crate) fn tombstones( @@ -160,7 +274,7 @@ impl Snapshot { } /// A snapshot of a Delta table that has been eagerly loaded into memory. -#[derive(Debug, Clone)] +#[derive(Debug, Clone, PartialEq)] pub struct EagerSnapshot { snapshot: Snapshot, files: Vec, @@ -179,6 +293,55 @@ impl EagerSnapshot { Ok(Self { snapshot, files }) } + #[cfg(test)] + pub fn new_test<'a>(commits: impl IntoIterator) -> DeltaResult { + let (snapshot, batch) = Snapshot::new_test(commits)?; + let mut files = Vec::new(); + let mut scanner = LogReplayScanner::new(); + files.push(scanner.process_files_batch(&batch, true)?); + Ok(Self { snapshot, files }) + } + + /// Update the snapshot to the given version + pub async fn update( + &mut self, + store: Arc, + target_version: Option, + ) -> DeltaResult<()> { + if Some(self.version()) == target_version { + return Ok(()); + } + let new_slice = self + .snapshot + .update_inner(store.clone(), target_version) + .await?; + if let Some(new_slice) = new_slice { + let files = std::mem::take(&mut self.files); + let log_stream = new_slice.commit_stream( + store.clone(), + &log_segment::COMMIT_SCHEMA, + &self.snapshot.config, + )?; + let checkpoint_stream = if new_slice.checkpoint_files.is_empty() { + futures::stream::iter(files.into_iter().map(Ok)).boxed() + } else { + new_slice + .checkpoint_stream( + store, + &log_segment::CHECKPOINT_SCHEMA, + &self.snapshot.config, + ) + .boxed() + }; + let files = ReplayStream::new(log_stream, checkpoint_stream) + .try_collect() + .await?; + + self.files = files; + } + Ok(()) + } + pub(crate) fn snapshot(&self) -> &Snapshot { &self.snapshot } @@ -188,6 +351,14 @@ impl EagerSnapshot { self.snapshot.version() } + /// Get the timestamp of the given version + pub fn version_timestamp(&self, version: i64) -> Option { + self.snapshot + .log_segment + .version_timestamp(version) + .map(|ts| ts.timestamp_millis()) + } + /// Get the table schema of the snapshot pub fn schema(&self) -> &StructType { self.snapshot.schema() @@ -213,6 +384,11 @@ impl EagerSnapshot { self.snapshot.table_config() } + /// Get the number of files in the snapshot + pub fn files_count(&self) -> usize { + self.files.iter().map(|f| f.num_rows() as usize).sum() + } + /// Get the files in the snapshot pub fn file_actions(&self) -> DeltaResult + '_> { Ok(self.files.iter().flat_map(|b| extract_adds(b)).flatten()) @@ -223,8 +399,26 @@ impl EagerSnapshot { &mut self, commits: impl IntoIterator, ) -> DeltaResult { + let mut metadata = None; + let mut protocol = None; + let mut send = Vec::new(); + for commit in commits { + if metadata.is_none() { + metadata = commit.0.iter().find_map(|a| match a { + Action::Metadata(metadata) => Some(metadata.clone()), + _ => None, + }); + } + if protocol.is_none() { + protocol = commit.0.iter().find_map(|a| match a { + Action::Protocol(protocol) => Some(protocol.clone()), + _ => None, + }); + } + send.push(commit); + } let actions = self.snapshot.log_segment.advance( - commits, + send, &self.table_root(), &log_segment::COMMIT_SCHEMA, &self.snapshot.config, @@ -236,6 +430,7 @@ impl EagerSnapshot { for batch in actions { files.push(scanner.process_files_batch(&batch?, true)?); } + self.files = files .into_iter() .chain( @@ -245,6 +440,14 @@ impl EagerSnapshot { ) .collect(); + if let Some(metadata) = metadata { + self.snapshot.metadata = metadata; + self.snapshot.schema = serde_json::from_str(&self.snapshot.metadata.schema_string)?; + } + if let Some(protocol) = protocol { + self.snapshot.protocol = protocol; + } + Ok(self.snapshot.version()) } } @@ -286,12 +489,17 @@ mod tests { let snapshot = Snapshot::try_new(&Path::default(), store.clone(), Default::default(), None).await?; + let bytes = serde_json::to_vec(&snapshot).unwrap(); + let actual: Snapshot = serde_json::from_slice(&bytes).unwrap(); + assert_eq!(actual, snapshot); + let schema_string = r#"{"type":"struct","fields":[{"name":"id","type":"long","nullable":true,"metadata":{}}]}"#; let expected: StructType = serde_json::from_str(schema_string)?; assert_eq!(snapshot.schema(), &expected); let infos = snapshot - .commit_infos(store.clone()) + .commit_infos(store.clone(), None) + .await? .try_collect::>() .await?; let infos = infos.into_iter().flatten().collect_vec(); @@ -355,6 +563,10 @@ mod tests { EagerSnapshot::try_new(&Path::default(), store.clone(), Default::default(), None) .await?; + let bytes = serde_json::to_vec(&snapshot).unwrap(); + let actual: EagerSnapshot = serde_json::from_slice(&bytes).unwrap(); + assert_eq!(actual, snapshot); + let schema_string = r#"{"type":"struct","fields":[{"name":"id","type":"long","nullable":true,"metadata":{}}]}"#; let expected: StructType = serde_json::from_str(schema_string)?; assert_eq!(snapshot.schema(), &expected); diff --git a/crates/deltalake-core/src/kernel/snapshot/parse.rs b/crates/deltalake-core/src/kernel/snapshot/parse.rs index 49d99141f3..cb6a704fd8 100644 --- a/crates/deltalake-core/src/kernel/snapshot/parse.rs +++ b/crates/deltalake-core/src/kernel/snapshot/parse.rs @@ -1,15 +1,48 @@ use arrow::array::StructArray; -use arrow_array::{Array, BooleanArray, Int32Array, Int64Array, MapArray, StringArray}; +use arrow_array::{Array, BooleanArray, Int32Array, Int64Array, ListArray, MapArray, StringArray}; +use percent_encoding::percent_decode_str; use super::extract::{ extract_and_cast, extract_and_cast_opt, read_bool, read_bool_opt, read_primitive, read_primitive_opt, read_str, read_str_opt, ProvidesColumnByName, }; use crate::{ - kernel::{Add, DeletionVectorDescriptor, Remove}, - DeltaResult, + kernel::{Add, DeletionVectorDescriptor, Metadata, Remove}, + DeltaResult, DeltaTableError, }; +pub(super) fn read_metadata(batch: &dyn ProvidesColumnByName) -> DeltaResult> { + if let Some(arr) = extract_and_cast_opt::(batch, "metaData") { + let id = extract_and_cast::(arr, "id")?; + let name = extract_and_cast::(arr, "name")?; + let description = extract_and_cast::(arr, "description")?; + // let format = extract_and_cast::(arr, "format")?; + let schema_string = extract_and_cast::(arr, "schemaString")?; + let partition_columns = extract_and_cast_opt::(arr, "partitionColumns"); + let configuration = extract_and_cast_opt::(arr, "configuration"); + let created_time = extract_and_cast::(arr, "createdTime")?; + + for idx in 0..arr.len() { + if arr.is_valid(idx) { + return Ok(Some(Metadata { + id: read_str(id, idx)?.to_string(), + name: read_str_opt(name, idx).map(|s| s.to_string()), + description: read_str_opt(description, idx).map(|s| s.to_string()), + format: Default::default(), + schema_string: read_str(schema_string, idx)?.to_string(), + partition_columns: collect_string_list(&partition_columns, idx) + .unwrap_or_default(), + configuration: configuration + .and_then(|pv| collect_map(&pv.value(idx)).map(|m| m.collect())) + .unwrap_or_default(), + created_time: read_primitive_opt(created_time, idx), + })); + } + } + } + Ok(None) +} + pub(super) fn extract_adds(array: &dyn ProvidesColumnByName) -> DeltaResult> { let mut result = Vec::new(); @@ -52,8 +85,13 @@ pub(super) fn extract_adds(array: &dyn ProvidesColumnByName) -> DeltaResult DeltaResult Option, idx: usize) -> Option> { + arr.and_then(|val| { + let values = val.value(idx); + let values = values.as_ref().as_any().downcast_ref::()?; + Some( + values + .iter() + .filter_map(|v| v.map(|vv| vv.to_string())) + .collect(), + ) + }) +} diff --git a/crates/deltalake-core/src/kernel/snapshot/serde.rs b/crates/deltalake-core/src/kernel/snapshot/serde.rs index b5a145c430..7eea72762d 100644 --- a/crates/deltalake-core/src/kernel/snapshot/serde.rs +++ b/crates/deltalake-core/src/kernel/snapshot/serde.rs @@ -1,4 +1,3 @@ -use arrow_array::RecordBatch; use arrow_ipc::reader::FileReader; use arrow_ipc::writer::FileWriter; use chrono::{DateTime, Utc}; @@ -15,6 +14,8 @@ struct FileInfo { path: String, size: usize, last_modified: DateTime, + e_tag: Option, + version: Option, } impl Serialize for LogSegment { @@ -29,6 +30,8 @@ impl Serialize for LogSegment { path: f.location.to_string(), size: f.size, last_modified: f.last_modified, + e_tag: f.e_tag.clone(), + version: f.version.clone(), }) .collect::>(); let checkpoint_files = self @@ -38,6 +41,8 @@ impl Serialize for LogSegment { path: f.location.to_string(), size: f.size, last_modified: f.last_modified, + e_tag: f.e_tag.clone(), + version: f.version.clone(), }) .collect::>(); @@ -81,8 +86,8 @@ impl<'de> Visitor<'de> for LogSegmentVisitor { location: f.path.into(), size: f.size, last_modified: f.last_modified, - version: None, - e_tag: None, + version: f.version, + e_tag: f.e_tag, }) .collect(), checkpoint_files: checkpoint_files @@ -108,72 +113,26 @@ impl<'de> Deserialize<'de> for LogSegment { } } -struct RecordBatchData(RecordBatch); - -impl Serialize for RecordBatchData { - fn serialize(&self, serializer: S) -> Result - where - S: serde::Serializer, - { - let mut buffer = vec![]; - let mut writer = FileWriter::try_new(&mut buffer, self.0.schema().as_ref()) - .map_err(|e| serde::ser::Error::custom(e))?; - writer - .write(&self.0) - .map_err(|e| serde::ser::Error::custom(e))?; - writer.finish().map_err(|e| serde::ser::Error::custom(e))?; - let data = writer - .into_inner() - .map_err(|e| serde::ser::Error::custom(e))?; - serializer.serialize_bytes(&data) - } -} - -struct RecordBatchesVisitor; - -impl<'de> Visitor<'de> for RecordBatchesVisitor { - type Value = RecordBatchData; - - fn expecting(&self, formatter: &mut fmt::Formatter) -> fmt::Result { - formatter.write_str("struct RecordBatchData") - } - - fn visit_bytes(self, v: &[u8]) -> Result - where - E: de::Error, - { - let mut reader = FileReader::try_new(std::io::Cursor::new(v), None) - .map_err(|e| de::Error::custom(format!("failed to read ipc record batch: {}", e)))?; - let rb = reader - .next() - .ok_or(de::Error::custom("missing ipc data"))? - .map_err(|e| de::Error::custom(format!("failed to read ipc record batch: {}", e)))?; - Ok(RecordBatchData(rb)) - } -} - -impl<'de> Deserialize<'de> for RecordBatchData { - fn deserialize(deserializer: D) -> Result - where - D: Deserializer<'de>, - { - deserializer.deserialize_bytes(RecordBatchesVisitor) - } -} - impl Serialize for EagerSnapshot { fn serialize(&self, serializer: S) -> Result where S: serde::Serializer, { - let data = self - .files - .iter() - .map(|rb| RecordBatchData(rb.clone())) - .collect::>(); let mut seq = serializer.serialize_seq(None)?; seq.serialize_element(&self.snapshot)?; - seq.serialize_element(&data)?; + for batch in self.files.iter() { + let mut buffer = vec![]; + let mut writer = FileWriter::try_new(&mut buffer, batch.schema().as_ref()) + .map_err(|e| serde::ser::Error::custom(e))?; + writer + .write(&batch) + .map_err(|e| serde::ser::Error::custom(e))?; + writer.finish().map_err(|e| serde::ser::Error::custom(e))?; + let data = writer + .into_inner() + .map_err(|e| serde::ser::Error::custom(e))?; + seq.serialize_element(&data)?; + } seq.end() } } @@ -192,13 +151,24 @@ impl<'de> Visitor<'de> for EagerSnapshotVisitor { where V: SeqAccess<'de>, { + println!("eager: {:?}", "start"); let snapshot = seq .next_element()? .ok_or_else(|| de::Error::invalid_length(0, &self))?; - let data: Vec = seq - .next_element()? - .ok_or_else(|| de::Error::invalid_length(1, &self))?; - let files = data.into_iter().map(|rb| rb.0).collect::>(); + let mut files = Vec::new(); + while let Some(elem) = seq.next_element::>()? { + let mut reader = + FileReader::try_new(std::io::Cursor::new(elem), None).map_err(|e| { + de::Error::custom(format!("failed to read ipc record batch: {}", e)) + })?; + let rb = reader + .next() + .ok_or(de::Error::custom("missing ipc data"))? + .map_err(|e| { + de::Error::custom(format!("failed to read ipc record batch: {}", e)) + })?; + files.push(rb); + } Ok(EagerSnapshot { snapshot, files }) } } diff --git a/crates/deltalake-core/src/lib.rs b/crates/deltalake-core/src/lib.rs index 4844b6a1f0..72cbb0ba39 100644 --- a/crates/deltalake-core/src/lib.rs +++ b/crates/deltalake-core/src/lib.rs @@ -182,10 +182,10 @@ mod tests { .await .unwrap(); assert_eq!(table.version(), 3); - assert_eq!(table.protocol().min_writer_version, 2); - assert_eq!(table.protocol().min_reader_version, 1); + assert_eq!(table.protocol().unwrap().min_writer_version, 2); + assert_eq!(table.protocol().unwrap().min_reader_version, 1); assert_eq!( - table.get_files_iter().collect_vec(), + table.get_files_iter().unwrap().collect_vec(), vec![ Path::from("part-00000-cb6b150b-30b8-4662-ad28-ff32ddab96d2-c000.snappy.parquet"), Path::from("part-00000-7c2deba3-1994-4fb8-bc07-d46c948aa415-c000.snappy.parquet"), @@ -194,6 +194,7 @@ mod tests { ); let tombstones = table .get_state() + .unwrap() .all_tombstones(table.object_store().clone()) .await .unwrap() @@ -203,7 +204,7 @@ mod tests { path: "part-00000-512e1537-8aaa-4193-b8b4-bef3de0de409-c000.snappy.parquet".to_string(), deletion_timestamp: Some(1564524298213), data_change: false, - extended_file_metadata: Some(false), + extended_file_metadata: None, deletion_vector: None, partition_values: None, tags: None, @@ -224,8 +225,8 @@ mod tests { table_to_update.update().await.unwrap(); assert_eq!( - table_newest_version.get_files_iter().collect_vec(), - table_to_update.get_files_iter().collect_vec() + table_newest_version.get_files_iter().unwrap().collect_vec(), + table_to_update.get_files_iter().unwrap().collect_vec() ); } #[tokio::test] @@ -235,10 +236,10 @@ mod tests { .await .unwrap(); assert_eq!(table.version(), 0); - assert_eq!(table.protocol().min_writer_version, 2); - assert_eq!(table.protocol().min_reader_version, 1); + assert_eq!(table.protocol().unwrap().min_writer_version, 2); + assert_eq!(table.protocol().unwrap().min_reader_version, 1); assert_eq!( - table.get_files_iter().collect_vec(), + table.get_files_iter().unwrap().collect_vec(), vec![ Path::from("part-00000-b44fcdb0-8b06-4f3a-8606-f8311a96f6dc-c000.snappy.parquet"), Path::from("part-00001-185eca06-e017-4dea-ae49-fc48b973e37e-c000.snappy.parquet"), @@ -249,10 +250,10 @@ mod tests { .await .unwrap(); assert_eq!(table.version(), 2); - assert_eq!(table.protocol().min_writer_version, 2); - assert_eq!(table.protocol().min_reader_version, 1); + assert_eq!(table.protocol().unwrap().min_writer_version, 2); + assert_eq!(table.protocol().unwrap().min_reader_version, 1); assert_eq!( - table.get_files_iter().collect_vec(), + table.get_files_iter().unwrap().collect_vec(), vec![ Path::from("part-00000-7c2deba3-1994-4fb8-bc07-d46c948aa415-c000.snappy.parquet"), Path::from("part-00001-c373a5bd-85f0-4758-815e-7eb62007a15c-c000.snappy.parquet"), @@ -263,10 +264,10 @@ mod tests { .await .unwrap(); assert_eq!(table.version(), 3); - assert_eq!(table.protocol().min_writer_version, 2); - assert_eq!(table.protocol().min_reader_version, 1); + assert_eq!(table.protocol().unwrap().min_writer_version, 2); + assert_eq!(table.protocol().unwrap().min_reader_version, 1); assert_eq!( - table.get_files_iter().collect_vec(), + table.get_files_iter().unwrap().collect_vec(), vec![ Path::from("part-00000-cb6b150b-30b8-4662-ad28-ff32ddab96d2-c000.snappy.parquet"), Path::from("part-00000-7c2deba3-1994-4fb8-bc07-d46c948aa415-c000.snappy.parquet"), @@ -281,20 +282,21 @@ mod tests { .await .unwrap(); assert_eq!(table.version(), 1); - assert_eq!(table.protocol().min_writer_version, 2); - assert_eq!(table.protocol().min_reader_version, 1); + assert_eq!(table.protocol().unwrap().min_writer_version, 2); + assert_eq!(table.protocol().unwrap().min_reader_version, 1); assert_eq!( - table.get_files_iter().collect_vec(), + table.get_files_iter().unwrap().collect_vec(), vec![ + Path::from("part-00000-04ec9591-0b73-459e-8d18-ba5711d6cbe1-c000.snappy.parquet"), Path::from("part-00000-c9b90f86-73e6-46c8-93ba-ff6bfaf892a1-c000.snappy.parquet"), - Path::from("part-00000-04ec9591-0b73-459e-8d18-ba5711d6cbe1-c000.snappy.parquet") ] ); - assert_eq!(table.get_stats().count(), 2); + assert_eq!(table.get_stats().unwrap().count(), 2); assert_eq!( table .get_stats() + .unwrap() .map(|x| x.unwrap().unwrap().num_records) .sum::(), 4 @@ -303,12 +305,14 @@ mod tests { assert_eq!( table .get_stats() + .unwrap() .map(|x| x.unwrap().unwrap().null_count["value"].as_value().unwrap()) .collect::>(), vec![0, 0] ); let tombstones = table .get_state() + .unwrap() .all_tombstones(table.object_store().clone()) .await .unwrap() @@ -324,7 +328,7 @@ mod tests { base_row_id: None, default_row_commit_version: None, deletion_vector: None, - tags: None, + tags: Some(HashMap::new()), })); } @@ -334,21 +338,21 @@ mod tests { .await .unwrap(); assert_eq!(table.version(), 1); - assert_eq!(table.protocol().min_writer_version, 2); - assert_eq!(table.protocol().min_reader_version, 1); + assert_eq!(table.protocol().unwrap().min_writer_version, 2); + assert_eq!(table.protocol().unwrap().min_reader_version, 1); assert_eq!( - table.get_files_iter().collect_vec(), + table.get_files_iter().unwrap().collect_vec(), vec![ - Path::from("part-00000-c9b90f86-73e6-46c8-93ba-ff6bfaf892a1-c000.snappy.parquet"), Path::from("part-00000-04ec9591-0b73-459e-8d18-ba5711d6cbe1-c000.snappy.parquet"), + Path::from("part-00000-c9b90f86-73e6-46c8-93ba-ff6bfaf892a1-c000.snappy.parquet"), ] ); table.load_version(0).await.unwrap(); assert_eq!(table.version(), 0); - assert_eq!(table.protocol().min_writer_version, 2); - assert_eq!(table.protocol().min_reader_version, 1); + assert_eq!(table.protocol().unwrap().min_writer_version, 2); + assert_eq!(table.protocol().unwrap().min_reader_version, 1); assert_eq!( - table.get_files_iter().collect_vec(), + table.get_files_iter().unwrap().collect_vec(), vec![ Path::from("part-00000-c9b90f86-73e6-46c8-93ba-ff6bfaf892a1-c000.snappy.parquet"), Path::from("part-00001-911a94a2-43f6-4acb-8620-5e68c2654989-c000.snappy.parquet"), @@ -465,7 +469,7 @@ mod tests { .unwrap(); assert_eq!( - table.get_files_iter().collect_vec(), + table.get_files_iter().unwrap().collect_vec(), vec![ Path::parse( "x=A%2FA/part-00007-b350e235-2832-45df-9918-6cab4f7578f7.c000.snappy.parquet" @@ -533,7 +537,8 @@ mod tests { ) -> crate::protocol::Stats { table .get_file_uris() - .zip(table.get_stats()) + .unwrap() + .zip(table.get_stats().unwrap()) .filter_map(|(file_uri, file_stats)| { if file_uri.ends_with(file_name) { file_stats.unwrap() @@ -584,7 +589,7 @@ mod tests { if let PeekCommit::New(version, actions) = peek { assert_eq!(table.version(), 9); - assert!(!table.get_files_iter().any(|f| f + assert!(!table.get_files_iter().unwrap().any(|f| f == Path::from( "part-00000-f0e955c5-a1e3-4eec-834e-dcc098fc9005-c000.snappy.parquet" ))); @@ -595,7 +600,7 @@ mod tests { table.update_incremental(None).await.unwrap(); assert_eq!(table.version(), 10); - assert!(table.get_files_iter().any(|f| f + assert!(table.get_files_iter().unwrap().any(|f| f == Path::from( "part-00000-f0e955c5-a1e3-4eec-834e-dcc098fc9005-c000.snappy.parquet" ))); @@ -664,7 +669,7 @@ mod tests { .unwrap(); assert_eq!(table.version(), 2); assert_eq!( - table.get_files_iter().collect_vec(), + table.get_files_iter().unwrap().collect_vec(), vec![Path::from( "part-00000-7444aec4-710a-4a4c-8abe-3323499043e9.c000.snappy.parquet" ),] diff --git a/crates/deltalake-core/src/operations/constraints.rs b/crates/deltalake-core/src/operations/constraints.rs index a38540c93a..e0c035da36 100644 --- a/crates/deltalake-core/src/operations/constraints.rs +++ b/crates/deltalake-core/src/operations/constraints.rs @@ -103,7 +103,7 @@ impl std::future::IntoFuture for ConstraintBuilder { .expr .ok_or_else(|| DeltaTableError::Generic("No Expresion provided".to_string()))?; - let mut metadata = this.snapshot.metadata()?.clone(); + let mut metadata = this.snapshot.metadata().clone(); let configuration_key = format!("delta.constraints.{}", name); if metadata.configuration.contains_key(&configuration_key) { @@ -217,13 +217,12 @@ impl std::future::IntoFuture for ConstraintBuilder { this.log_store.as_ref(), &actions, operations.clone(), - &this.snapshot, + Some(&this.snapshot), None, ) .await?; - this.snapshot - .merge(actions, &operations, version, true, true)?; + this.snapshot.merge(actions, &operations, version)?; Ok(DeltaTable::new_with_state(this.log_store, this.snapshot)) }) } @@ -254,7 +253,7 @@ mod tests { async fn get_constraint_op_params(table: &mut DeltaTable) -> String { let commit_info = table.history(None).await.unwrap(); - let last_commit = &commit_info[commit_info.len() - 1]; + let last_commit = &commit_info[0]; last_commit .operation_parameters .as_ref() diff --git a/crates/deltalake-core/src/operations/convert_to_delta.rs b/crates/deltalake-core/src/operations/convert_to_delta.rs index 361a2b0405..7e39a4ab19 100644 --- a/crates/deltalake-core/src/operations/convert_to_delta.rs +++ b/crates/deltalake-core/src/operations/convert_to_delta.rs @@ -498,7 +498,7 @@ mod tests { "Testing location: {test_data_from:?}" ); - let mut files = table.get_files_iter().collect_vec(); + let mut files = table.get_files_iter().unwrap().collect_vec(); files.sort(); assert_eq!( files, expected_paths, @@ -518,6 +518,7 @@ mod tests { let mut partition_values = table .get_partition_values() + .unwrap() .flat_map(|map| map.clone()) .collect::>(); partition_values.sort(); diff --git a/crates/deltalake-core/src/operations/create.rs b/crates/deltalake-core/src/operations/create.rs index 7ff55cd513..a47c091b0b 100644 --- a/crates/deltalake-core/src/operations/create.rs +++ b/crates/deltalake-core/src/operations/create.rs @@ -292,6 +292,7 @@ impl std::future::IntoFuture for CreateBuilder { let app_metadata = this.metadata.clone(); let (mut table, actions, operation) = this.into_table_and_actions()?; let log_store = table.log_store(); + let table_state = if log_store.is_delta_table_location().await? { match mode { SaveMode::ErrorIfExists => return Err(CreateError::TableAlreadyExists.into()), @@ -302,11 +303,11 @@ impl std::future::IntoFuture for CreateBuilder { } SaveMode::Overwrite => { table.load().await?; - &table.state + Some(table.snapshot()?) } } } else { - &table.state + None }; let version = commit( @@ -317,6 +318,7 @@ impl std::future::IntoFuture for CreateBuilder { app_metadata, ) .await?; + table.load_version(version).await?; Ok(table) @@ -392,11 +394,11 @@ mod tests { .unwrap(); assert_eq!(table.version(), 0); assert_eq!( - table.protocol().min_reader_version, + table.protocol().unwrap().min_reader_version, PROTOCOL.default_reader_version() ); assert_eq!( - table.protocol().min_writer_version, + table.protocol().unwrap().min_writer_version, PROTOCOL.default_writer_version() ); assert_eq!(table.get_schema().unwrap(), &schema); @@ -414,8 +416,8 @@ mod tests { .with_actions(vec![Action::Protocol(protocol)]) .await .unwrap(); - assert_eq!(table.protocol().min_reader_version, 0); - assert_eq!(table.protocol().min_writer_version, 0); + assert_eq!(table.protocol().unwrap().min_reader_version, 0); + assert_eq!(table.protocol().unwrap().min_writer_version, 0); let table = CreateBuilder::new() .with_location("memory://") diff --git a/crates/deltalake-core/src/operations/delete.rs b/crates/deltalake-core/src/operations/delete.rs index 979340c753..70dd68799f 100644 --- a/crates/deltalake-core/src/operations/delete.rs +++ b/crates/deltalake-core/src/operations/delete.rs @@ -138,7 +138,7 @@ async fn excute_non_empty_expr( let input_schema = snapshot.input_schema()?; let input_dfschema: DFSchema = input_schema.clone().as_ref().clone().try_into()?; - let table_partition_cols = snapshot.metadata()?.partition_columns.clone(); + let table_partition_cols = snapshot.metadata().partition_columns.clone(); let scan = DeltaScanBuilder::new(snapshot, log_store.clone(), state) .with_files(rewrite) @@ -159,7 +159,7 @@ async fn excute_non_empty_expr( Arc::new(FilterExec::try_new(predicate_expr, scan.clone())?); let add_actions = write_execution_plan( - snapshot, + Some(snapshot), state.clone(), filter.clone(), table_partition_cols.clone(), @@ -265,7 +265,7 @@ async fn execute( log_store.as_ref(), &actions, operation.clone(), - snapshot, + Some(snapshot), Some(app_metadata), ) .await?; @@ -316,7 +316,7 @@ impl std::future::IntoFuture for DeleteBuilder { .await?; if let Some(op) = &operation { - this.snapshot.merge(actions, op, version, true, true)?; + this.snapshot.merge(actions, op, version)?; } let table = DeltaTable::new_with_state(this.log_store, this.snapshot); @@ -400,19 +400,19 @@ mod tests { .await .unwrap(); assert_eq!(table.version(), 1); - assert_eq!(table.get_file_uris().count(), 1); + assert_eq!(table.get_files_count(), 1); let (table, metrics) = DeltaOps(table).delete().await.unwrap(); assert_eq!(table.version(), 2); - assert_eq!(table.get_file_uris().count(), 0); + assert_eq!(table.get_files_count(), 0); assert_eq!(metrics.num_added_files, 0); assert_eq!(metrics.num_removed_files, 1); assert_eq!(metrics.num_deleted_rows, None); assert_eq!(metrics.num_copied_rows, None); let commit_info = table.history(None).await.unwrap(); - let last_commit = &commit_info[commit_info.len() - 1]; + let last_commit = &commit_info[0]; let _extra_info = last_commit.info.clone(); // assert_eq!( // extra_info["operationMetrics"], @@ -462,7 +462,7 @@ mod tests { .await .unwrap(); assert_eq!(table.version(), 1); - assert_eq!(table.get_file_uris().count(), 1); + assert_eq!(table.get_files_count(), 1); let batch = RecordBatch::try_new( Arc::clone(&schema), @@ -486,7 +486,7 @@ mod tests { .await .unwrap(); assert_eq!(table.version(), 2); - assert_eq!(table.get_file_uris().count(), 2); + assert_eq!(table.get_files_count(), 2); let (table, metrics) = DeltaOps(table) .delete() @@ -494,7 +494,7 @@ mod tests { .await .unwrap(); assert_eq!(table.version(), 3); - assert_eq!(table.get_file_uris().count(), 2); + assert_eq!(table.get_files_count(), 2); assert_eq!(metrics.num_added_files, 1); assert_eq!(metrics.num_removed_files, 1); @@ -503,7 +503,7 @@ mod tests { assert_eq!(metrics.num_copied_rows, Some(3)); let commit_info = table.history(None).await.unwrap(); - let last_commit = &commit_info[commit_info.len() - 1]; + let last_commit = &commit_info[0]; let parameters = last_commit.operation_parameters.clone().unwrap(); assert_eq!(parameters["predicate"], json!("value = 1")); @@ -642,7 +642,7 @@ mod tests { .await .unwrap(); assert_eq!(table.version(), 1); - assert_eq!(table.get_file_uris().count(), 2); + assert_eq!(table.get_files_count(), 2); let (table, metrics) = DeltaOps(table) .delete() @@ -650,7 +650,7 @@ mod tests { .await .unwrap(); assert_eq!(table.version(), 2); - assert_eq!(table.get_file_uris().count(), 1); + assert_eq!(table.get_files_count(), 1); assert_eq!(metrics.num_added_files, 0); assert_eq!(metrics.num_removed_files, 1); @@ -700,7 +700,7 @@ mod tests { .await .unwrap(); assert_eq!(table.version(), 1); - assert_eq!(table.get_file_uris().count(), 3); + assert_eq!(table.get_files_count(), 3); let (table, metrics) = DeltaOps(table) .delete() @@ -712,7 +712,7 @@ mod tests { .await .unwrap(); assert_eq!(table.version(), 2); - assert_eq!(table.get_file_uris().count(), 2); + assert_eq!(table.get_files_count(), 2); assert_eq!(metrics.num_added_files, 0); assert_eq!(metrics.num_removed_files, 1); diff --git a/crates/deltalake-core/src/operations/filesystem_check.rs b/crates/deltalake-core/src/operations/filesystem_check.rs index cacc4c11c4..0c9249fd41 100644 --- a/crates/deltalake-core/src/operations/filesystem_check.rs +++ b/crates/deltalake-core/src/operations/filesystem_check.rs @@ -100,17 +100,17 @@ impl FileSystemCheckBuilder { } async fn create_fsck_plan(&self) -> DeltaResult { - let mut files_relative: HashMap<&str, &Add> = - HashMap::with_capacity(self.snapshot.files().len()); + let mut files_relative: HashMap = + HashMap::with_capacity(self.snapshot.files()?.len()); let log_store = self.log_store.clone(); - for active in self.snapshot.files() { + for active in self.snapshot.files()? { if is_absolute_path(&active.path)? { return Err(DeltaTableError::Generic( "Filesystem check does not support absolute paths".to_string(), )); } else { - files_relative.insert(&active.path, active); + files_relative.insert(active.path.clone(), active); } } @@ -189,7 +189,7 @@ impl FileSystemCheckPlan { self.log_store.as_ref(), &actions, DeltaOperation::FileSystemCheck {}, - snapshot, + Some(snapshot), // TODO pass through metadata Some(app_metadata), ) diff --git a/crates/deltalake-core/src/operations/load.rs b/crates/deltalake-core/src/operations/load.rs index 0189381922..ce2e46f9b6 100644 --- a/crates/deltalake-core/src/operations/load.rs +++ b/crates/deltalake-core/src/operations/load.rs @@ -50,7 +50,7 @@ impl std::future::IntoFuture for LoadBuilder { PROTOCOL.can_read_from(&this.snapshot)?; let table = DeltaTable::new_with_state(this.log_store, this.snapshot); - let schema = table.state.arrow_schema()?; + let schema = table.snapshot()?.arrow_schema()?; let projection = this .columns .map(|cols| { diff --git a/crates/deltalake-core/src/operations/merge/mod.rs b/crates/deltalake-core/src/operations/merge/mod.rs index c1917dab81..4e3c326dc0 100644 --- a/crates/deltalake-core/src/operations/merge/mod.rs +++ b/crates/deltalake-core/src/operations/merge/mod.rs @@ -822,13 +822,6 @@ async fn try_construct_early_filter( target_name: &TableReference<'_>, ) -> DeltaResult> { let table_metadata = table_snapshot.metadata(); - - if table_metadata.is_err() { - return Ok(None); - } - - let table_metadata = table_metadata.unwrap(); - let partition_columns = &table_metadata.partition_columns; if partition_columns.is_empty() { @@ -917,7 +910,7 @@ async fn execute( let mut metrics = MergeMetrics::default(); let exec_start = Instant::now(); - let current_metadata = snapshot.metadata()?; + let current_metadata = snapshot.metadata(); // TODO: Given the join predicate, remove any expression that involve the // source table and keep expressions that only involve the target table. @@ -1149,7 +1142,7 @@ async fn execute( let mut new_columns = projection; let mut write_projection = Vec::new(); - for delta_field in snapshot.schema().unwrap().fields() { + for delta_field in snapshot.schema().fields() { let mut when_expr = Vec::with_capacity(operations_size); let mut then_expr = Vec::with_capacity(operations_size); @@ -1310,7 +1303,7 @@ async fn execute( let rewrite_start = Instant::now(); let add_actions = write_execution_plan( - snapshot, + Some(snapshot), state.clone(), write, table_partition_cols.clone(), @@ -1341,7 +1334,7 @@ async fn execute( { let lock = survivors.lock().unwrap(); - for action in snapshot.files() { + for action in snapshot.files()? { if lock.contains(&action.path) { metrics.num_target_files_removed += 1; actions.push(Action::Remove(Remove { @@ -1402,7 +1395,7 @@ async fn execute( log_store.as_ref(), &actions, operation.clone(), - snapshot, + Some(snapshot), Some(app_metadata), ) .await?; @@ -1470,7 +1463,7 @@ impl std::future::IntoFuture for MergeBuilder { .await?; if let Some(op) = &operation { - this.snapshot.merge(actions, op, version, true, true)?; + this.snapshot.merge(actions, op, version)?; } let table = DeltaTable::new_with_state(this.log_store, this.snapshot); @@ -1595,14 +1588,14 @@ mod tests { let table = write_data(table, &schema).await; assert_eq!(table.version(), 1); - assert_eq!(table.get_file_uris().count(), 1); + assert_eq!(table.get_files_count(), 1); (table, merge_source(schema)) } async fn assert_merge(table: DeltaTable, metrics: MergeMetrics) { assert_eq!(table.version(), 2); - assert!(table.get_file_uris().count() >= 1); + assert!(table.get_files_count() >= 1); assert!(metrics.num_target_files_added >= 1); assert_eq!(metrics.num_target_files_removed, 1); assert_eq!(metrics.num_target_rows_copied, 1); @@ -1631,7 +1624,7 @@ mod tests { async fn test_merge() { let (table, source) = setup().await; - let (mut table, metrics) = DeltaOps(table) + let (table, metrics) = DeltaOps(table) .merge(source, col("target.id").eq(col("source.id"))) .with_source_alias("source") .with_target_alias("target") @@ -1658,7 +1651,7 @@ mod tests { .unwrap(); let commit_info = table.history(None).await.unwrap(); - let last_commit = &commit_info[commit_info.len() - 1]; + let last_commit = &commit_info[0]; let parameters = last_commit.operation_parameters.clone().unwrap(); assert_eq!(parameters["predicate"], json!("target.id = source.id")); assert_eq!( @@ -1683,7 +1676,7 @@ mod tests { // Also validates that update and set operations can contain the target alias let (table, source) = setup().await; - let (mut table, metrics) = DeltaOps(table) + let (table, metrics) = DeltaOps(table) .merge(source, "target.id = source.id") .with_source_alias("source") .with_target_alias("target") @@ -1710,7 +1703,7 @@ mod tests { .unwrap(); let commit_info = table.history(None).await.unwrap(); - let last_commit = &commit_info[commit_info.len() - 1]; + let last_commit = &commit_info[0]; let parameters = last_commit.operation_parameters.clone().unwrap(); assert_eq!(parameters["predicate"], json!("target.id = source.id")); assert_eq!( @@ -1850,7 +1843,7 @@ mod tests { let table = write_data(table, &schema).await; assert_eq!(table.version(), 1); - assert_eq!(table.get_file_uris().count(), 2); + assert_eq!(table.get_files_count(), 2); let ctx = SessionContext::new(); let batch = RecordBatch::try_new( @@ -1906,7 +1899,7 @@ mod tests { .unwrap(); assert_eq!(table.version(), 2); - assert!(table.get_file_uris().count() >= 3); + assert!(table.get_files_count() >= 3); assert!(metrics.num_target_files_added >= 3); assert_eq!(metrics.num_target_files_removed, 2); assert_eq!(metrics.num_target_rows_copied, 1); @@ -1940,7 +1933,7 @@ mod tests { let table = write_data(table, &schema).await; assert_eq!(table.version(), 1); - assert_eq!(table.get_file_uris().count(), 4); + assert_eq!(table.get_files_count(), 4); let ctx = SessionContext::new(); let batch = RecordBatch::try_new( @@ -1979,7 +1972,7 @@ mod tests { .unwrap(); assert_eq!(table.version(), 2); - assert!(table.get_file_uris().count() >= 3); + assert!(table.get_files_count() >= 3); assert_eq!(metrics.num_target_files_added, 3); assert_eq!(metrics.num_target_files_removed, 2); assert_eq!(metrics.num_target_rows_copied, 0); @@ -2013,7 +2006,7 @@ mod tests { let table = write_data(table, &schema).await; assert_eq!(table.version(), 1); - assert_eq!(table.get_file_uris().count(), 2); + assert_eq!(table.get_files_count(), 2); let ctx = SessionContext::new(); let batch = RecordBatch::try_new( @@ -2031,7 +2024,7 @@ mod tests { .unwrap(); let source = ctx.read_batch(batch).unwrap(); - let (mut table, metrics) = DeltaOps(table) + let (table, metrics) = DeltaOps(table) .merge(source, col("target.id").eq(col("source.id"))) .with_source_alias("source") .with_target_alias("target") @@ -2041,7 +2034,7 @@ mod tests { .unwrap(); assert_eq!(table.version(), 2); - assert!(table.get_file_uris().count() >= 2); + assert!(table.get_files_count() >= 2); assert_eq!(metrics.num_target_files_added, 2); assert_eq!(metrics.num_target_files_removed, 2); assert_eq!(metrics.num_target_rows_copied, 2); @@ -2052,7 +2045,7 @@ mod tests { assert_eq!(metrics.num_source_rows, 3); let commit_info = table.history(None).await.unwrap(); - let last_commit = &commit_info[commit_info.len() - 1]; + let last_commit = &commit_info[0]; let parameters = last_commit.operation_parameters.clone().unwrap(); let extra_info = last_commit.info.clone(); assert_eq!( @@ -2082,7 +2075,7 @@ mod tests { let table = write_data(table, &schema).await; assert_eq!(table.version(), 1); - assert_eq!(table.get_file_uris().count(), 2); + assert_eq!(table.get_files_count(), 2); let ctx = SessionContext::new(); let batch = RecordBatch::try_new( @@ -2100,7 +2093,7 @@ mod tests { .unwrap(); let source = ctx.read_batch(batch).unwrap(); - let (mut table, metrics) = DeltaOps(table) + let (table, metrics) = DeltaOps(table) .merge(source, col("target.id").eq(col("source.id"))) .with_source_alias("source") .with_target_alias("target") @@ -2110,7 +2103,7 @@ mod tests { .unwrap(); assert_eq!(table.version(), 2); - assert!(table.get_file_uris().count() >= 2); + assert!(table.get_files_count() >= 2); assert_eq!(metrics.num_target_files_added, 1); assert_eq!(metrics.num_target_files_removed, 1); assert_eq!(metrics.num_target_rows_copied, 1); @@ -2121,7 +2114,7 @@ mod tests { assert_eq!(metrics.num_source_rows, 3); let commit_info = table.history(None).await.unwrap(); - let last_commit = &commit_info[commit_info.len() - 1]; + let last_commit = &commit_info[0]; let parameters = last_commit.operation_parameters.clone().unwrap(); assert_eq!(parameters["predicate"], json!("target.id = source.id")); assert_eq!( @@ -2151,7 +2144,7 @@ mod tests { let table = write_data(table, &schema).await; assert_eq!(table.version(), 1); - assert_eq!(table.get_file_uris().count(), 2); + assert_eq!(table.get_files_count(), 2); let ctx = SessionContext::new(); let batch = RecordBatch::try_new( @@ -2169,7 +2162,7 @@ mod tests { .unwrap(); let source = ctx.read_batch(batch).unwrap(); - let (mut table, metrics) = DeltaOps(table) + let (table, metrics) = DeltaOps(table) .merge(source, col("target.id").eq(col("source.id"))) .with_source_alias("source") .with_target_alias("target") @@ -2179,7 +2172,7 @@ mod tests { .unwrap(); assert_eq!(table.version(), 2); - assert_eq!(table.get_file_uris().count(), 2); + assert_eq!(table.get_files_count(), 2); assert_eq!(metrics.num_target_files_added, 2); assert_eq!(metrics.num_target_files_removed, 2); assert_eq!(metrics.num_target_rows_copied, 2); @@ -2190,7 +2183,7 @@ mod tests { assert_eq!(metrics.num_source_rows, 3); let commit_info = table.history(None).await.unwrap(); - let last_commit = &commit_info[commit_info.len() - 1]; + let last_commit = &commit_info[0]; let parameters = last_commit.operation_parameters.clone().unwrap(); assert_eq!(parameters["predicate"], json!("target.id = source.id")); assert_eq!( @@ -2214,7 +2207,7 @@ mod tests { let table = write_data(table, &schema).await; assert_eq!(table.version(), 1); - assert_eq!(table.get_file_uris().count(), 2); + assert_eq!(table.get_files_count(), 2); let ctx = SessionContext::new(); let batch = RecordBatch::try_new( @@ -2232,7 +2225,7 @@ mod tests { .unwrap(); let source = ctx.read_batch(batch).unwrap(); - let (mut table, metrics) = DeltaOps(table) + let (table, metrics) = DeltaOps(table) .merge(source, col("target.id").eq(col("source.id"))) .with_source_alias("source") .with_target_alias("target") @@ -2254,7 +2247,7 @@ mod tests { assert_eq!(metrics.num_source_rows, 3); let commit_info = table.history(None).await.unwrap(); - let last_commit = &commit_info[commit_info.len() - 1]; + let last_commit = &commit_info[0]; let parameters = last_commit.operation_parameters.clone().unwrap(); assert_eq!(parameters["predicate"], json!("target.id = source.id")); assert_eq!( @@ -2281,7 +2274,7 @@ mod tests { let table = setup_table(Some(vec!["modified"])).await; assert_eq!(table.version(), 0); - assert_eq!(table.get_file_uris().count(), 0); + assert_eq!(table.get_files_count(), 0); let ctx = SessionContext::new(); let batch = RecordBatch::try_new( @@ -2325,7 +2318,7 @@ mod tests { .unwrap(); assert_eq!(table.version(), 1); - assert!(table.get_file_uris().count() >= 2); + assert!(table.get_files_count() >= 2); assert!(metrics.num_target_files_added >= 2); assert_eq!(metrics.num_target_files_removed, 0); assert_eq!(metrics.num_target_rows_copied, 0); @@ -2398,7 +2391,7 @@ mod tests { let table = write_data(table, &arrow_schema).await; assert_eq!(table.version(), 1); - assert_eq!(table.get_file_uris().count(), 1); + assert_eq!(table.get_files_count(), 1); let (table, _metrics) = DeltaOps(table) .merge(source, "target.Id = source.Id") @@ -2561,7 +2554,7 @@ mod tests { let table = setup_table(Some(vec!["id"])).await; assert_eq!(table.version(), 0); - assert_eq!(table.get_file_uris().count(), 0); + assert_eq!(table.get_files_count(), 0); let ctx = SessionContext::new(); let batch = RecordBatch::try_new( @@ -2602,7 +2595,7 @@ mod tests { let pred = try_construct_early_filter( join_predicate, - &table.state, + table.snapshot().unwrap(), &ctx.state(), &source, &source_name, diff --git a/crates/deltalake-core/src/operations/mod.rs b/crates/deltalake-core/src/operations/mod.rs index ee3fb45114..8b495e7a97 100644 --- a/crates/deltalake-core/src/operations/mod.rs +++ b/crates/deltalake-core/src/operations/mod.rs @@ -55,6 +55,8 @@ pub mod write; #[cfg(all(feature = "arrow", feature = "parquet"))] pub mod writer; +// TODO make ops consume a snapshot ... + /// High level interface for executing commands against a DeltaTable pub struct DeltaOps(pub DeltaTable); @@ -132,7 +134,7 @@ impl DeltaOps { #[cfg(feature = "datafusion")] #[must_use] pub fn load(self) -> LoadBuilder { - LoadBuilder::new(self.0.log_store, self.0.state) + LoadBuilder::new(self.0.log_store, self.0.state.unwrap()) } /// Write data to Delta table @@ -145,40 +147,40 @@ impl DeltaOps { /// Vacuum stale files from delta table #[must_use] pub fn vacuum(self) -> VacuumBuilder { - VacuumBuilder::new(self.0.log_store, self.0.state) + VacuumBuilder::new(self.0.log_store, self.0.state.unwrap()) } /// Audit active files with files present on the filesystem #[must_use] pub fn filesystem_check(self) -> FileSystemCheckBuilder { - FileSystemCheckBuilder::new(self.0.log_store, self.0.state) + FileSystemCheckBuilder::new(self.0.log_store, self.0.state.unwrap()) } /// Audit active files with files present on the filesystem #[cfg(all(feature = "arrow", feature = "parquet"))] #[must_use] pub fn optimize<'a>(self) -> OptimizeBuilder<'a> { - OptimizeBuilder::new(self.0.log_store, self.0.state) + OptimizeBuilder::new(self.0.log_store, self.0.state.unwrap()) } /// Delete data from Delta table #[cfg(feature = "datafusion")] #[must_use] pub fn delete(self) -> DeleteBuilder { - DeleteBuilder::new(self.0.log_store, self.0.state) + DeleteBuilder::new(self.0.log_store, self.0.state.unwrap()) } /// Update data from Delta table #[cfg(feature = "datafusion")] #[must_use] pub fn update(self) -> UpdateBuilder { - UpdateBuilder::new(self.0.log_store, self.0.state) + UpdateBuilder::new(self.0.log_store, self.0.state.unwrap()) } /// Restore delta table to a specified version or datetime #[must_use] pub fn restore(self) -> RestoreBuilder { - RestoreBuilder::new(self.0.log_store, self.0.state) + RestoreBuilder::new(self.0.log_store, self.0.state.unwrap()) } /// Update data from Delta table @@ -189,14 +191,19 @@ impl DeltaOps { source: datafusion::prelude::DataFrame, predicate: E, ) -> MergeBuilder { - MergeBuilder::new(self.0.log_store, self.0.state, predicate.into(), source) + MergeBuilder::new( + self.0.log_store, + self.0.state.unwrap(), + predicate.into(), + source, + ) } /// Add a check constraint to a table #[cfg(feature = "datafusion")] #[must_use] pub fn add_constraint(self) -> ConstraintBuilder { - ConstraintBuilder::new(self.0.log_store, self.0.state) + ConstraintBuilder::new(self.0.log_store, self.0.state.unwrap()) } } diff --git a/crates/deltalake-core/src/operations/optimize.rs b/crates/deltalake-core/src/operations/optimize.rs index 01f628d6e3..b753e34af9 100644 --- a/crates/deltalake-core/src/operations/optimize.rs +++ b/crates/deltalake-core/src/operations/optimize.rs @@ -24,7 +24,7 @@ use std::collections::HashMap; use std::sync::Arc; use std::time::{Duration, Instant, SystemTime, UNIX_EPOCH}; -use arrow::datatypes::{Schema as ArrowSchema, SchemaRef as ArrowSchemaRef}; +use arrow::datatypes::SchemaRef as ArrowSchemaRef; use arrow_array::RecordBatch; use futures::future::BoxFuture; use futures::stream::BoxStream; @@ -717,7 +717,7 @@ impl MergePlan { app_metadata.insert("operationMetrics".to_owned(), map); } - table.update_incremental(None).await?; + table.update().await?; debug!("committing {} actions", actions.len()); //// TODO: Check for remove actions on optimized partitions. If a //// optimized partition was updated then abort the commit. Requires (#593). @@ -725,7 +725,7 @@ impl MergePlan { table.log_store.as_ref(), &actions, self.task_parameters.input_parameters.clone().into(), - table.get_state(), + Some(table.snapshot()?), Some(app_metadata.clone()), ) .await?; @@ -779,7 +779,7 @@ pub fn create_merge_plan( ) -> Result { let target_size = target_size.unwrap_or_else(|| snapshot.table_config().target_file_size()); - let partitions_keys = &snapshot.metadata()?.partition_columns; + let partitions_keys = &snapshot.metadata().partition_columns; let (operations, metrics) = match optimize_type { OptimizeType::Compact => { @@ -791,14 +791,8 @@ pub fn create_merge_plan( }; let input_parameters = OptimizeInput { target_size }; - let file_schema = arrow_schema_without_partitions( - &Arc::new( - >::try_from( - &snapshot.metadata()?.schema()?, - )?, - ), - partitions_keys, - ); + let file_schema = + arrow_schema_without_partitions(&Arc::new(snapshot.schema().try_into()?), partitions_keys); Ok(MergePlan { operations, @@ -868,7 +862,7 @@ fn build_compaction_plan( let mut partition_files: HashMap> = HashMap::new(); for add in snapshot.get_active_add_actions_by_partitions(filters)? { metrics.total_considered_files += 1; - let object_meta = ObjectMeta::try_from(add)?; + let object_meta = ObjectMeta::try_from(&add)?; if (object_meta.size as i64) > target_size { metrics.total_files_skipped += 1; continue; @@ -944,8 +938,7 @@ fn build_zorder_plan( ))); } let field_names = snapshot - .metadata()? - .schema()? + .schema() .fields() .iter() .map(|field| field.name().to_string()) @@ -966,7 +959,7 @@ fn build_zorder_plan( let mut partition_files: HashMap = HashMap::new(); for add in snapshot.get_active_add_actions_by_partitions(filters)? { metrics.total_considered_files += 1; - let object_meta = ObjectMeta::try_from(add)?; + let object_meta = ObjectMeta::try_from(&add)?; let part = PartitionTuples::from_hashmap(partition_keys, &add.partition_values); partition_files @@ -1076,6 +1069,7 @@ pub(super) mod zorder { use arrow_buffer::bit_util::{get_bit_raw, set_bit_raw, unset_bit_raw}; use arrow_row::{Row, RowConverter, SortField}; use arrow_schema::ArrowError; + // use arrow_schema::Schema as ArrowSchema; /// Execution context for Z-order scan #[cfg(not(feature = "datafusion"))] @@ -1307,6 +1301,7 @@ pub(super) mod zorder { #[tokio::test] async fn test_zorder_mixed_case() { + use arrow_schema::Schema as ArrowSchema; let schema = Arc::new(ArrowSchema::new(vec![ Field::new("moDified", DataType::Utf8, true), Field::new("ID", DataType::Utf8, true), diff --git a/crates/deltalake-core/src/operations/restore.rs b/crates/deltalake-core/src/operations/restore.rs index dea87ed1ba..d64c1ea4f7 100644 --- a/crates/deltalake-core/src/operations/restore.rs +++ b/crates/deltalake-core/src/operations/restore.rs @@ -153,6 +153,7 @@ async fn execute( return Err(DeltaTableError::from(RestoreError::InvalidRestoreParameter)); } let mut table = DeltaTable::new(log_store.clone(), DeltaTableConfig::default()); + let version = match datetime_to_restore { Some(datetime) => { table.load_with_datetime(datetime).await?; @@ -170,8 +171,8 @@ async fn execute( snapshot.version(), ))); } - let state_to_restore_files = table.get_state().files().clone(); - let latest_state_files = snapshot.files().clone(); + let state_to_restore_files = table.snapshot()?.files()?; + let latest_state_files = snapshot.files()?; let state_to_restore_files_set = HashSet::::from_iter(state_to_restore_files.iter().cloned()); let latest_state_files_set = HashSet::::from_iter(latest_state_files.iter().cloned()); @@ -220,27 +221,27 @@ async fn execute( let mut actions = vec![]; let protocol = if protocol_downgrade_allowed { Protocol { - min_reader_version: table.protocol().min_reader_version, - min_writer_version: table.protocol().min_writer_version, + min_reader_version: table.protocol()?.min_reader_version, + min_writer_version: table.protocol()?.min_writer_version, writer_features: if snapshot.protocol().min_writer_version < 7 { None } else { - table.protocol().writer_features.clone() + table.protocol()?.writer_features.clone() }, reader_features: if snapshot.protocol().min_reader_version < 3 { None } else { - table.protocol().reader_features.clone() + table.protocol()?.reader_features.clone() }, } } else { Protocol { min_reader_version: max( - table.protocol().min_reader_version, + table.protocol()?.min_reader_version, snapshot.protocol().min_reader_version, ), min_writer_version: max( - table.protocol().min_writer_version, + table.protocol()?.min_writer_version, snapshot.protocol().min_writer_version, ), writer_features: snapshot.protocol().writer_features.clone(), diff --git a/crates/deltalake-core/src/operations/transaction/conflict_checker.rs b/crates/deltalake-core/src/operations/transaction/conflict_checker.rs index d6a02c3d10..5f22ad9a82 100644 --- a/crates/deltalake-core/src/operations/transaction/conflict_checker.rs +++ b/crates/deltalake-core/src/operations/transaction/conflict_checker.rs @@ -173,7 +173,7 @@ impl<'a> TransactionInfo<'a> { #[cfg(feature = "datafusion")] /// Files read by the transaction - pub fn read_files(&self) -> Result, CommitConflictError> { + pub fn read_files(&self) -> Result, CommitConflictError> { if let Some(predicate) = &self.read_predicates { Ok(Either::Left( self.read_snapshot @@ -463,7 +463,6 @@ impl<'a> ConflictChecker<'a> { .txn_info .read_snapshot .metadata() - .map_err(|_|CommitConflictError::NoMetadata)? .partition_columns; AddContainer::new(&added_files_to_check, partition_columns, arrow_schema) .predicate_matches(predicate.clone()) @@ -679,7 +678,7 @@ mod tests { read_whole_table: bool, ) -> Result<(), CommitConflictError> { let setup_actions = setup.unwrap_or_else(|| init_table_actions(None)); - let state = DeltaTableState::from_actions(setup_actions, 0).unwrap(); + let state = DeltaTableState::from_actions(setup_actions).unwrap(); let transaction_info = TransactionInfo::new(&state, reads, &actions, read_whole_table); let summary = WinningCommitSummary { actions: concurrent, diff --git a/crates/deltalake-core/src/operations/transaction/mod.rs b/crates/deltalake-core/src/operations/transaction/mod.rs index d9b4e52463..8542e00d52 100644 --- a/crates/deltalake-core/src/operations/transaction/mod.rs +++ b/crates/deltalake-core/src/operations/transaction/mod.rs @@ -175,7 +175,7 @@ pub async fn commit( log_store: &dyn LogStore, actions: &Vec, operation: DeltaOperation, - read_snapshot: &DeltaTableState, + read_snapshot: Option<&DeltaTableState>, app_metadata: Option>, ) -> DeltaResult { commit_with_retries( @@ -197,11 +197,14 @@ pub async fn commit_with_retries( log_store: &dyn LogStore, actions: &Vec, operation: DeltaOperation, - read_snapshot: &DeltaTableState, + read_snapshot: Option<&DeltaTableState>, app_metadata: Option>, max_retries: usize, ) -> DeltaResult { - PROTOCOL.can_commit(read_snapshot, actions)?; + if let Some(read_snapshot) = read_snapshot { + PROTOCOL.can_commit(read_snapshot, actions)?; + } + let tmp_commit = prepare_commit( log_store.object_store().as_ref(), &operation, @@ -210,8 +213,14 @@ pub async fn commit_with_retries( ) .await?; - let mut attempt_number = 1; + if read_snapshot.is_none() { + log_store.write_commit_entry(0, &tmp_commit).await?; + return Ok(0); + } + + let read_snapshot = read_snapshot.unwrap(); + let mut attempt_number = 1; while attempt_number <= max_retries { let version = read_snapshot.version() + attempt_number as i64; match log_store.write_commit_entry(version, &tmp_commit).await { diff --git a/crates/deltalake-core/src/operations/transaction/protocol.rs b/crates/deltalake-core/src/operations/transaction/protocol.rs index 9c20755935..07a7b75405 100644 --- a/crates/deltalake-core/src/operations/transaction/protocol.rs +++ b/crates/deltalake-core/src/operations/transaction/protocol.rs @@ -169,8 +169,11 @@ pub static INSTANCE: Lazy = Lazy::new(|| { let mut writer_features = HashSet::new(); writer_features.insert(WriterFeatures::AppendOnly); - writer_features.insert(WriterFeatures::Invariants); - writer_features.insert(WriterFeatures::CheckConstraints); + #[cfg(feature = "datafusion")] + { + writer_features.insert(WriterFeatures::Invariants); + writer_features.insert(WriterFeatures::CheckConstraints); + } // writer_features.insert(WriterFeatures::ChangeDataFeed); // writer_features.insert(WriterFeatures::GeneratedColumns); // writer_features.insert(WriterFeatures::ColumnMapping); @@ -240,37 +243,37 @@ mod tests { let checker = ProtocolChecker::new(HashSet::new(), WRITER_V2.clone()); let actions = create_actions(1, "true", vec![]); - let snapshot = DeltaTableState::from_actions(actions, 1).unwrap(); + let snapshot = DeltaTableState::from_actions(actions).unwrap(); assert!(checker.can_commit(&snapshot, &append_actions).is_ok()); assert!(checker.can_commit(&snapshot, &change_actions).is_ok()); assert!(checker.can_commit(&snapshot, &neutral_actions).is_ok()); let actions = create_actions(2, "true", vec![]); - let snapshot = DeltaTableState::from_actions(actions, 1).unwrap(); + let snapshot = DeltaTableState::from_actions(actions).unwrap(); assert!(checker.can_commit(&snapshot, &append_actions).is_ok()); assert!(checker.can_commit(&snapshot, &change_actions).is_err()); assert!(checker.can_commit(&snapshot, &neutral_actions).is_ok()); let actions = create_actions(2, "false", vec![]); - let snapshot = DeltaTableState::from_actions(actions, 1).unwrap(); + let snapshot = DeltaTableState::from_actions(actions).unwrap(); assert!(checker.can_commit(&snapshot, &append_actions).is_ok()); assert!(checker.can_commit(&snapshot, &change_actions).is_ok()); assert!(checker.can_commit(&snapshot, &neutral_actions).is_ok()); let actions = create_actions(7, "true", vec![WriterFeatures::AppendOnly]); - let snapshot = DeltaTableState::from_actions(actions, 1).unwrap(); + let snapshot = DeltaTableState::from_actions(actions).unwrap(); assert!(checker.can_commit(&snapshot, &append_actions).is_ok()); assert!(checker.can_commit(&snapshot, &change_actions).is_err()); assert!(checker.can_commit(&snapshot, &neutral_actions).is_ok()); let actions = create_actions(7, "false", vec![WriterFeatures::AppendOnly]); - let snapshot = DeltaTableState::from_actions(actions, 1).unwrap(); + let snapshot = DeltaTableState::from_actions(actions).unwrap(); assert!(checker.can_commit(&snapshot, &append_actions).is_ok()); assert!(checker.can_commit(&snapshot, &change_actions).is_ok()); assert!(checker.can_commit(&snapshot, &neutral_actions).is_ok()); let actions = create_actions(7, "true", vec![]); - let snapshot = DeltaTableState::from_actions(actions, 1).unwrap(); + let snapshot = DeltaTableState::from_actions(actions).unwrap(); assert!(checker.can_commit(&snapshot, &append_actions).is_ok()); assert!(checker.can_commit(&snapshot, &change_actions).is_ok()); assert!(checker.can_commit(&snapshot, &neutral_actions).is_ok()); @@ -279,22 +282,28 @@ mod tests { #[test] fn test_versions() { let checker_1 = ProtocolChecker::new(HashSet::new(), HashSet::new()); - let actions = vec![Action::Protocol(Protocol { - min_reader_version: 1, - min_writer_version: 1, - ..Default::default() - })]; - let snapshot_1 = DeltaTableState::from_actions(actions, 1).unwrap(); + let actions = vec![ + Action::Protocol(Protocol { + min_reader_version: 1, + min_writer_version: 1, + ..Default::default() + }), + create_metadata_action(None, Some(HashMap::new())), + ]; + let snapshot_1 = DeltaTableState::from_actions(actions).unwrap(); assert!(checker_1.can_read_from(&snapshot_1).is_ok()); assert!(checker_1.can_write_to(&snapshot_1).is_ok()); let checker_2 = ProtocolChecker::new(READER_V2.clone(), HashSet::new()); - let actions = vec![Action::Protocol(Protocol { - min_reader_version: 2, - min_writer_version: 1, - ..Default::default() - })]; - let snapshot_2 = DeltaTableState::from_actions(actions, 1).unwrap(); + let actions = vec![ + Action::Protocol(Protocol { + min_reader_version: 2, + min_writer_version: 1, + ..Default::default() + }), + create_metadata_action(None, Some(HashMap::new())), + ]; + let snapshot_2 = DeltaTableState::from_actions(actions).unwrap(); assert!(checker_1.can_read_from(&snapshot_2).is_err()); assert!(checker_1.can_write_to(&snapshot_2).is_err()); assert!(checker_2.can_read_from(&snapshot_1).is_ok()); @@ -302,12 +311,15 @@ mod tests { assert!(checker_2.can_write_to(&snapshot_2).is_ok()); let checker_3 = ProtocolChecker::new(READER_V2.clone(), WRITER_V2.clone()); - let actions = vec![Action::Protocol(Protocol { - min_reader_version: 2, - min_writer_version: 2, - ..Default::default() - })]; - let snapshot_3 = DeltaTableState::from_actions(actions, 1).unwrap(); + let actions = vec![ + Action::Protocol(Protocol { + min_reader_version: 2, + min_writer_version: 2, + ..Default::default() + }), + create_metadata_action(None, Some(HashMap::new())), + ]; + let snapshot_3 = DeltaTableState::from_actions(actions).unwrap(); assert!(checker_1.can_read_from(&snapshot_3).is_err()); assert!(checker_1.can_write_to(&snapshot_3).is_err()); assert!(checker_2.can_read_from(&snapshot_3).is_ok()); @@ -318,12 +330,15 @@ mod tests { assert!(checker_3.can_write_to(&snapshot_3).is_ok()); let checker_4 = ProtocolChecker::new(READER_V2.clone(), WRITER_V3.clone()); - let actions = vec![Action::Protocol(Protocol { - min_reader_version: 2, - min_writer_version: 3, - ..Default::default() - })]; - let snapshot_4 = DeltaTableState::from_actions(actions, 1).unwrap(); + let actions = vec![ + Action::Protocol(Protocol { + min_reader_version: 2, + min_writer_version: 3, + ..Default::default() + }), + create_metadata_action(None, Some(HashMap::new())), + ]; + let snapshot_4 = DeltaTableState::from_actions(actions).unwrap(); assert!(checker_1.can_read_from(&snapshot_4).is_err()); assert!(checker_1.can_write_to(&snapshot_4).is_err()); assert!(checker_2.can_read_from(&snapshot_4).is_ok()); @@ -337,12 +352,15 @@ mod tests { assert!(checker_4.can_write_to(&snapshot_4).is_ok()); let checker_5 = ProtocolChecker::new(READER_V2.clone(), WRITER_V4.clone()); - let actions = vec![Action::Protocol(Protocol { - min_reader_version: 2, - min_writer_version: 4, - ..Default::default() - })]; - let snapshot_5 = DeltaTableState::from_actions(actions, 1).unwrap(); + let actions = vec![ + Action::Protocol(Protocol { + min_reader_version: 2, + min_writer_version: 4, + ..Default::default() + }), + create_metadata_action(None, Some(HashMap::new())), + ]; + let snapshot_5 = DeltaTableState::from_actions(actions).unwrap(); assert!(checker_1.can_read_from(&snapshot_5).is_err()); assert!(checker_1.can_write_to(&snapshot_5).is_err()); assert!(checker_2.can_read_from(&snapshot_5).is_ok()); @@ -359,12 +377,15 @@ mod tests { assert!(checker_5.can_write_to(&snapshot_5).is_ok()); let checker_6 = ProtocolChecker::new(READER_V2.clone(), WRITER_V5.clone()); - let actions = vec![Action::Protocol(Protocol { - min_reader_version: 2, - min_writer_version: 5, - ..Default::default() - })]; - let snapshot_6 = DeltaTableState::from_actions(actions, 1).unwrap(); + let actions = vec![ + Action::Protocol(Protocol { + min_reader_version: 2, + min_writer_version: 5, + ..Default::default() + }), + create_metadata_action(None, Some(HashMap::new())), + ]; + let snapshot_6 = DeltaTableState::from_actions(actions).unwrap(); assert!(checker_1.can_read_from(&snapshot_6).is_err()); assert!(checker_1.can_write_to(&snapshot_6).is_err()); assert!(checker_2.can_read_from(&snapshot_6).is_ok()); @@ -384,12 +405,15 @@ mod tests { assert!(checker_6.can_write_to(&snapshot_6).is_ok()); let checker_7 = ProtocolChecker::new(READER_V2.clone(), WRITER_V6.clone()); - let actions = vec![Action::Protocol(Protocol { - min_reader_version: 2, - min_writer_version: 6, - ..Default::default() - })]; - let snapshot_7 = DeltaTableState::from_actions(actions, 1).unwrap(); + let actions = vec![ + Action::Protocol(Protocol { + min_reader_version: 2, + min_writer_version: 6, + ..Default::default() + }), + create_metadata_action(None, Some(HashMap::new())), + ]; + let snapshot_7 = DeltaTableState::from_actions(actions).unwrap(); assert!(checker_1.can_read_from(&snapshot_7).is_err()); assert!(checker_1.can_write_to(&snapshot_7).is_err()); assert!(checker_2.can_read_from(&snapshot_7).is_ok()); diff --git a/crates/deltalake-core/src/operations/transaction/state.rs b/crates/deltalake-core/src/operations/transaction/state.rs index 9ebf41029b..199ac79912 100644 --- a/crates/deltalake-core/src/operations/transaction/state.rs +++ b/crates/deltalake-core/src/operations/transaction/state.rs @@ -29,7 +29,7 @@ impl DeltaTableState { } fn _arrow_schema(&self, wrap_partitions: bool) -> DeltaResult { - let meta = self.metadata()?; + let meta = self.metadata(); let fields = meta .schema()? .fields() @@ -74,15 +74,15 @@ impl DeltaTableState { pub fn files_matching_predicate( &self, filters: &[Expr], - ) -> DeltaResult> { + ) -> DeltaResult> { if let Some(Some(predicate)) = (!filters.is_empty()).then_some(conjunction(filters.iter().cloned())) { let expr = logical_expr_to_physical_expr(&predicate, self.arrow_schema()?.as_ref()); let pruning_predicate = PruningPredicate::try_new(expr, self.arrow_schema()?)?; Ok(Either::Left( - self.files() - .iter() + self.files()? + .into_iter() .zip(pruning_predicate.prune(self)?) .filter_map( |(action, keep_file)| { @@ -95,7 +95,7 @@ impl DeltaTableState { ), )) } else { - Ok(Either::Right(self.files().iter())) + Ok(Either::Right(self.files()?.into_iter())) } } @@ -117,7 +117,7 @@ impl DeltaTableState { &self, object_store: Arc, ) -> DeltaResult { - if let Some(add) = self.files().iter().max_by_key(|obj| obj.modification_time) { + if let Some(add) = self.files()?.iter().max_by_key(|obj| obj.modification_time) { let file_meta = add.try_into()?; let file_reader = ParquetObjectReader::new(object_store, file_meta); let file_schema = ParquetRecordBatchStreamBuilder::new(file_reader) @@ -298,25 +298,25 @@ impl PruningStatistics for DeltaTableState { /// return the minimum values for the named column, if known. /// Note: the returned array must contain `num_containers()` rows fn min_values(&self, column: &Column) -> Option { - let partition_columns = &self.metadata().ok()?.partition_columns; - let container = - AddContainer::new(self.files(), partition_columns, self.arrow_schema().ok()?); + let files = self.files().ok()?; + let partition_columns = &self.metadata().partition_columns; + let container = AddContainer::new(&files, partition_columns, self.arrow_schema().ok()?); container.min_values(column) } /// return the maximum values for the named column, if known. /// Note: the returned array must contain `num_containers()` rows. fn max_values(&self, column: &Column) -> Option { - let partition_columns = &self.metadata().ok()?.partition_columns; - let container = - AddContainer::new(self.files(), partition_columns, self.arrow_schema().ok()?); + let files = self.files().ok()?; + let partition_columns = &self.metadata().partition_columns; + let container = AddContainer::new(&files, partition_columns, self.arrow_schema().ok()?); container.max_values(column) } /// return the number of containers (e.g. row groups) being /// pruned with these statistics fn num_containers(&self) -> usize { - self.files().len() + self.files().unwrap().len() } /// return the number of null values for the named column as an @@ -324,9 +324,9 @@ impl PruningStatistics for DeltaTableState { /// /// Note: the returned array must contain `num_containers()` rows. fn null_counts(&self, column: &Column) -> Option { - let partition_columns = &self.metadata().ok()?.partition_columns; - let container = - AddContainer::new(self.files(), partition_columns, self.arrow_schema().ok()?); + let files = self.files().ok()?; + let partition_columns = &self.metadata().partition_columns; + let container = AddContainer::new(&files, partition_columns, self.arrow_schema().ok()?); container.null_counts(column) } } @@ -340,7 +340,7 @@ mod tests { #[test] fn test_parse_predicate_expression() { - let snapshot = DeltaTableState::from_actions(init_table_actions(None), 0).unwrap(); + let snapshot = DeltaTableState::from_actions(init_table_actions(None)).unwrap(); let session = SessionContext::new(); let state = session.state(); @@ -372,7 +372,7 @@ mod tests { actions.push(create_add_action("included-1", true, Some("{\"numRecords\":10,\"minValues\":{\"value\":1},\"maxValues\":{\"value\":100},\"nullCount\":{\"value\":0}}".into()))); actions.push(create_add_action("included-2", true, Some("{\"numRecords\":10,\"minValues\":{\"value\":-10},\"maxValues\":{\"value\":3},\"nullCount\":{\"value\":0}}".into()))); - let state = DeltaTableState::from_actions(actions, 0).unwrap(); + let state = DeltaTableState::from_actions(actions).unwrap(); let files = state .files_matching_predicate(&[]) .unwrap() diff --git a/crates/deltalake-core/src/operations/transaction/test_utils.rs b/crates/deltalake-core/src/operations/transaction/test_utils.rs index 6cc6c23bcf..7d178681f0 100644 --- a/crates/deltalake-core/src/operations/transaction/test_utils.rs +++ b/crates/deltalake-core/src/operations/transaction/test_utils.rs @@ -138,7 +138,7 @@ pub async fn create_initialized_table( true, ), ]); - let state = DeltaTableState::from_actions(init_table_actions(None), 0).unwrap(); + let state = DeltaTableState::from_actions(init_table_actions(None)).unwrap(); let operation = DeltaOperation::Create { mode: SaveMode::ErrorIfExists, location: "location".into(), diff --git a/crates/deltalake-core/src/operations/update.rs b/crates/deltalake-core/src/operations/update.rs index d3db975585..582a37da28 100644 --- a/crates/deltalake-core/src/operations/update.rs +++ b/crates/deltalake-core/src/operations/update.rs @@ -209,7 +209,7 @@ async fn execute( }) .collect::, _>>()?; - let current_metadata = snapshot.metadata()?; + let current_metadata = snapshot.metadata(); let table_partition_cols = current_metadata.partition_columns.clone(); let scan_start = Instant::now(); @@ -354,7 +354,7 @@ async fn execute( )?); let add_actions = write_execution_plan( - snapshot, + Some(snapshot), state.clone(), projection.clone(), table_partition_cols.clone(), @@ -424,7 +424,7 @@ async fn execute( log_store.as_ref(), &actions, operation.clone(), - snapshot, + Some(snapshot), Some(app_metadata), ) .await?; @@ -466,7 +466,7 @@ impl std::future::IntoFuture for UpdateBuilder { .await?; if let Some(op) = &operation { - this.snapshot.merge(actions, op, version, true, true)?; + this.snapshot.merge(actions, op, version)?; } let table = DeltaTable::new_with_state(this.log_store, this.snapshot); @@ -569,7 +569,7 @@ mod tests { let table = write_batch(table, batch).await; assert_eq!(table.version(), 1); - assert_eq!(table.get_file_uris().count(), 1); + assert_eq!(table.get_files_count(), 1); let (table, metrics) = DeltaOps(table) .update() @@ -578,7 +578,7 @@ mod tests { .unwrap(); assert_eq!(table.version(), 2); - assert_eq!(table.get_file_uris().count(), 1); + assert_eq!(table.get_files_count(), 1); assert_eq!(metrics.num_added_files, 1); assert_eq!(metrics.num_removed_files, 1); assert_eq!(metrics.num_updated_rows, 4); @@ -623,7 +623,7 @@ mod tests { let table = write_batch(table, batch).await; assert_eq!(table.version(), 1); - assert_eq!(table.get_file_uris().count(), 1); + assert_eq!(table.get_files_count(), 1); let (table, metrics) = DeltaOps(table) .update() @@ -633,14 +633,14 @@ mod tests { .unwrap(); assert_eq!(table.version(), 2); - assert_eq!(table.get_file_uris().count(), 1); + assert_eq!(table.get_files_count(), 1); assert_eq!(metrics.num_added_files, 1); assert_eq!(metrics.num_removed_files, 1); assert_eq!(metrics.num_updated_rows, 2); assert_eq!(metrics.num_copied_rows, 2); let commit_info = table.history(None).await.unwrap(); - let last_commit = &commit_info[commit_info.len() - 1]; + let last_commit = &commit_info[0]; let parameters = last_commit.operation_parameters.clone().unwrap(); assert_eq!(parameters["predicate"], json!("modified = '2021-02-03'")); @@ -680,7 +680,7 @@ mod tests { let table = write_batch(table, batch.clone()).await; assert_eq!(table.version(), 1); - assert_eq!(table.get_file_uris().count(), 2); + assert_eq!(table.get_files_count(), 2); let (table, metrics) = DeltaOps(table) .update() @@ -691,7 +691,7 @@ mod tests { .unwrap(); assert_eq!(table.version(), 2); - assert_eq!(table.get_file_uris().count(), 2); + assert_eq!(table.get_files_count(), 2); assert_eq!(metrics.num_added_files, 1); assert_eq!(metrics.num_removed_files, 1); assert_eq!(metrics.num_updated_rows, 2); @@ -715,7 +715,7 @@ mod tests { let table = setup_table(Some(vec!["modified"])).await; let table = write_batch(table, batch).await; assert_eq!(table.version(), 1); - assert_eq!(table.get_file_uris().count(), 2); + assert_eq!(table.get_files_count(), 2); let (table, metrics) = DeltaOps(table) .update() @@ -730,7 +730,7 @@ mod tests { .unwrap(); assert_eq!(table.version(), 2); - assert_eq!(table.get_file_uris().count(), 3); + assert_eq!(table.get_files_count(), 3); assert_eq!(metrics.num_added_files, 2); assert_eq!(metrics.num_removed_files, 1); assert_eq!(metrics.num_updated_rows, 1); @@ -826,7 +826,7 @@ mod tests { async fn test_update_null() { let table = prepare_values_table().await; assert_eq!(table.version(), 0); - assert_eq!(table.get_file_uris().count(), 1); + assert_eq!(table.get_files_count(), 1); let (table, metrics) = DeltaOps(table) .update() @@ -834,7 +834,7 @@ mod tests { .await .unwrap(); assert_eq!(table.version(), 1); - assert_eq!(table.get_file_uris().count(), 1); + assert_eq!(table.get_files_count(), 1); assert_eq!(metrics.num_added_files, 1); assert_eq!(metrics.num_removed_files, 1); assert_eq!(metrics.num_updated_rows, 5); @@ -864,14 +864,14 @@ mod tests { .await .unwrap(); assert_eq!(table.version(), 1); - assert_eq!(table.get_file_uris().count(), 1); + assert_eq!(table.get_files_count(), 1); assert_eq!(metrics.num_added_files, 1); assert_eq!(metrics.num_removed_files, 1); assert_eq!(metrics.num_updated_rows, 2); assert_eq!(metrics.num_copied_rows, 3); let commit_info = table.history(None).await.unwrap(); - let last_commit = &commit_info[commit_info.len() - 1]; + let last_commit = &commit_info[0]; let extra_info = last_commit.info.clone(); assert_eq!( extra_info["operationMetrics"], @@ -900,7 +900,7 @@ mod tests { .await .unwrap(); assert_eq!(table.version(), 1); - assert_eq!(table.get_file_uris().count(), 1); + assert_eq!(table.get_files_count(), 1); assert_eq!(metrics.num_added_files, 1); assert_eq!(metrics.num_removed_files, 1); assert_eq!(metrics.num_updated_rows, 2); diff --git a/crates/deltalake-core/src/operations/vacuum.rs b/crates/deltalake-core/src/operations/vacuum.rs index 850fa6b92c..b0f8a3193a 100644 --- a/crates/deltalake-core/src/operations/vacuum.rs +++ b/crates/deltalake-core/src/operations/vacuum.rs @@ -206,13 +206,13 @@ impl VacuumBuilder { self.log_store.object_store().clone(), ) .await?; - let valid_files = self.snapshot.file_paths_iter().collect::>(); + let valid_files = self.snapshot.file_paths_iter()?.collect::>(); let mut files_to_delete = vec![]; let mut file_sizes = vec![]; let object_store = self.log_store.object_store(); let mut all_files = object_store.list(None); - let partition_columns = &self.snapshot.metadata()?.partition_columns; + let partition_columns = &self.snapshot.metadata().partition_columns; while let Some(obj_meta) = all_files.next().await { // TODO should we allow NotFound here in case we have a temporary commit file in the list @@ -330,7 +330,7 @@ impl VacuumPlan { let start_actions = vec![Action::CommitInfo(commit_info)]; - commit(store, &start_actions, start_operation, snapshot, None).await?; + commit(store, &start_actions, start_operation, Some(snapshot), None).await?; // Finish VACUUM START COMMIT let locations = futures::stream::iter(self.files_to_delete) @@ -374,7 +374,7 @@ impl VacuumPlan { let end_actions = vec![Action::CommitInfo(commit_info)]; - commit(store, &end_actions, end_operation, snapshot, None).await?; + commit(store, &end_actions, end_operation, Some(snapshot), None).await?; // Finish VACUUM END COMMIT Ok(VacuumMetrics { @@ -432,7 +432,7 @@ mod tests { .await .unwrap(); - let result = VacuumBuilder::new(table.log_store, table.state.clone()) + let result = VacuumBuilder::new(table.log_store(), table.snapshot().unwrap().clone()) .with_retention_period(Duration::hours(1)) .with_dry_run(true) .await; @@ -442,23 +442,26 @@ mod tests { let table = open_table("../deltalake-test/tests/data/delta-0.8.0") .await .unwrap(); - let (table, result) = VacuumBuilder::new(table.log_store, table.state) - .with_retention_period(Duration::hours(0)) - .with_dry_run(true) - .with_enforce_retention_duration(false) - .await - .unwrap(); + + let (table, result) = + VacuumBuilder::new(table.log_store(), table.snapshot().unwrap().clone()) + .with_retention_period(Duration::hours(0)) + .with_dry_run(true) + .with_enforce_retention_duration(false) + .await + .unwrap(); // do not enforce retention duration check with 0 hour will purge all files assert_eq!( result.files_deleted, vec!["part-00001-911a94a2-43f6-4acb-8620-5e68c2654989-c000.snappy.parquet"] ); - let (table, result) = VacuumBuilder::new(table.log_store, table.state) - .with_retention_period(Duration::hours(169)) - .with_dry_run(true) - .await - .unwrap(); + let (table, result) = + VacuumBuilder::new(table.log_store(), table.snapshot().unwrap().clone()) + .with_retention_period(Duration::hours(169)) + .with_dry_run(true) + .await + .unwrap(); assert_eq!( result.files_deleted, @@ -471,11 +474,12 @@ mod tests { .as_secs() / 3600; let empty: Vec = Vec::new(); - let (_table, result) = VacuumBuilder::new(table.log_store, table.state) - .with_retention_period(Duration::hours(retention_hours as i64)) - .with_dry_run(true) - .await - .unwrap(); + let (_table, result) = + VacuumBuilder::new(table.log_store(), table.snapshot().unwrap().clone()) + .with_retention_period(Duration::hours(retention_hours as i64)) + .with_dry_run(true) + .await + .unwrap(); assert_eq!(result.files_deleted, empty); } diff --git a/crates/deltalake-core/src/operations/write.rs b/crates/deltalake-core/src/operations/write.rs index b94469a213..7ef271d43d 100644 --- a/crates/deltalake-core/src/operations/write.rs +++ b/crates/deltalake-core/src/operations/write.rs @@ -84,7 +84,7 @@ impl From for DeltaTableError { #[derive(Debug, Clone)] pub struct WriteBuilder { /// A snapshot of the to-be-loaded table's state - snapshot: DeltaTableState, + snapshot: Option, /// Delta object store for handling data files log_store: LogStoreRef, /// The input plan @@ -121,7 +121,7 @@ pub struct WriteBuilder { impl WriteBuilder { /// Create a new [`WriteBuilder`] - pub fn new(log_store: LogStoreRef, snapshot: DeltaTableState) -> Self { + pub fn new(log_store: LogStoreRef, snapshot: Option) -> Self { Self { snapshot, log_store, @@ -249,9 +249,9 @@ impl WriteBuilder { } async fn check_preconditions(&self) -> DeltaResult> { - match self.log_store.is_delta_table_location().await? { - true => { - PROTOCOL.can_write_to(&self.snapshot)?; + match &self.snapshot { + Some(snapshot) => { + PROTOCOL.can_write_to(snapshot)?; match self.mode { SaveMode::ErrorIfExists => { Err(WriteError::AlreadyExists(self.log_store.root_uri()).into()) @@ -259,7 +259,7 @@ impl WriteBuilder { _ => Ok(vec![]), } } - false => { + None => { let schema: StructType = if let Some(plan) = &self.input { Ok(plan.schema().try_into()?) } else if let Some(batches) = &self.batches { @@ -295,7 +295,7 @@ impl WriteBuilder { #[allow(clippy::too_many_arguments)] pub(crate) async fn write_execution_plan( - snapshot: &DeltaTableState, + snapshot: Option<&DeltaTableState>, state: SessionState, plan: Arc, partition_columns: Vec, @@ -310,10 +310,16 @@ pub(crate) async fn write_execution_plan( let schema: ArrowSchemaRef = if overwrite_schema { plan.schema() } else { - snapshot.input_schema().unwrap_or(plan.schema()) + snapshot + .and_then(|s| s.input_schema().ok()) + .unwrap_or(plan.schema()) }; - let checker = DeltaDataChecker::new(snapshot); + let checker = if let Some(snapshot) = snapshot { + DeltaDataChecker::new(snapshot) + } else { + DeltaDataChecker::empty() + }; // Write data to disk let mut tasks = vec![]; @@ -364,11 +370,13 @@ impl std::future::IntoFuture for WriteBuilder { type IntoFuture = BoxFuture<'static, Self::Output>; fn into_future(self) -> Self::IntoFuture { - let mut this = self; + let this = self; Box::pin(async move { if this.mode == SaveMode::Overwrite { - PROTOCOL.check_append_only(&this.snapshot)?; + if let Some(snapshot) = &this.snapshot { + PROTOCOL.check_append_only(snapshot)?; + } } // Create table actions to initialize table in case it does not yet exist and should be created @@ -376,9 +384,8 @@ impl std::future::IntoFuture for WriteBuilder { let active_partitions = this .snapshot - .metadata() - .map(|meta| meta.partition_columns.clone()) - .ok(); + .as_ref() + .map(|s| s.metadata().partition_columns.clone()); // validate partition columns let partition_columns = if let Some(active_part) = active_partitions { @@ -406,20 +413,22 @@ impl std::future::IntoFuture for WriteBuilder { Err(WriteError::MissingData) } else { schema = batches[0].schema(); - let table_schema = this - .snapshot - .physical_arrow_schema(this.log_store.object_store().clone()) - .await - .or_else(|_| this.snapshot.arrow_schema()) - .unwrap_or(schema.clone()); - if !can_cast_batch(schema.fields(), table_schema.fields()) - && !(this.overwrite_schema && matches!(this.mode, SaveMode::Overwrite)) - { - return Err(DeltaTableError::Generic( - "Schema of data does not match table schema".to_string(), - )); - }; + if let Some(snapshot) = &this.snapshot { + let table_schema = snapshot + .physical_arrow_schema(this.log_store.object_store().clone()) + .await + .or_else(|_| snapshot.arrow_schema()) + .unwrap_or(schema.clone()); + + if !can_cast_batch(schema.fields(), table_schema.fields()) + && !(this.overwrite_schema && matches!(this.mode, SaveMode::Overwrite)) + { + return Err(DeltaTableError::Generic( + "Schema of data does not match table schema".to_string(), + )); + }; + } let data = if !partition_columns.is_empty() { // TODO partitioning should probably happen in its own plan ... @@ -468,7 +477,7 @@ impl std::future::IntoFuture for WriteBuilder { }; let add_actions = write_execution_plan( - &this.snapshot, + this.snapshot.as_ref(), state, plan, partition_columns.clone(), @@ -483,61 +492,61 @@ impl std::future::IntoFuture for WriteBuilder { actions.extend(add_actions.into_iter().map(Action::Add)); // Collect remove actions if we are overwriting the table - if matches!(this.mode, SaveMode::Overwrite) { - // Update metadata with new schema - let table_schema = this - .snapshot - .physical_arrow_schema(this.log_store.object_store().clone()) - .await - .or_else(|_| this.snapshot.arrow_schema()) - .unwrap_or(schema.clone()); - - if schema != table_schema { - let mut metadata = this.snapshot.metadata()?.clone(); - let delta_schema: StructType = schema.as_ref().try_into()?; - metadata.schema_string = serde_json::to_string(&delta_schema)?; - actions.push(Action::Metadata(metadata)); - } - // This should never error, since now() will always be larger than UNIX_EPOCH - let deletion_timestamp = SystemTime::now() - .duration_since(UNIX_EPOCH) - .unwrap() - .as_millis() as i64; - - let to_remove_action = |add: &Add| { - Action::Remove(Remove { - path: add.path.clone(), - deletion_timestamp: Some(deletion_timestamp), - data_change: true, - extended_file_metadata: Some(false), - partition_values: Some(add.partition_values.clone()), - size: Some(add.size), - // TODO add file metadata to remove action (tags missing) - tags: None, - deletion_vector: add.deletion_vector.clone(), - base_row_id: add.base_row_id, - default_row_commit_version: add.default_row_commit_version, - }) - }; + if let Some(snapshot) = &this.snapshot { + if matches!(this.mode, SaveMode::Overwrite) { + // Update metadata with new schema + let table_schema = snapshot + .physical_arrow_schema(this.log_store.object_store().clone()) + .await + .or_else(|_| snapshot.arrow_schema()) + .unwrap_or(schema.clone()); - match this.predicate { - Some(_pred) => { - return Err(DeltaTableError::Generic( - "Overwriting data based on predicate is not yet implemented" - .to_string(), - )); + if schema != table_schema { + let mut metadata = snapshot.metadata().clone(); + let delta_schema: StructType = schema.as_ref().try_into()?; + metadata.schema_string = serde_json::to_string(&delta_schema)?; + actions.push(Action::Metadata(metadata)); } - _ => { - let remove_actions = this - .snapshot - .files() - .iter() - .map(to_remove_action) - .collect::>(); - actions.extend(remove_actions); + // This should never error, since now() will always be larger than UNIX_EPOCH + let deletion_timestamp = SystemTime::now() + .duration_since(UNIX_EPOCH) + .unwrap() + .as_millis() as i64; + + let to_remove_action = |add: &Add| { + Action::Remove(Remove { + path: add.path.clone(), + deletion_timestamp: Some(deletion_timestamp), + data_change: true, + extended_file_metadata: Some(false), + partition_values: Some(add.partition_values.clone()), + size: Some(add.size), + // TODO add file metadata to remove action (tags missing) + tags: None, + deletion_vector: add.deletion_vector.clone(), + base_row_id: add.base_row_id, + default_row_commit_version: add.default_row_commit_version, + }) + }; + + match this.predicate { + Some(_pred) => { + return Err(DeltaTableError::Generic( + "Overwriting data based on predicate is not yet implemented" + .to_string(), + )); + } + _ => { + let remove_actions = snapshot + .files()? + .iter() + .map(to_remove_action) + .collect::>(); + actions.extend(remove_actions); + } } - } - }; + }; + } let operation = DeltaOperation::Write { mode: this.mode, @@ -552,7 +561,7 @@ impl std::future::IntoFuture for WriteBuilder { this.log_store.as_ref(), &actions, operation.clone(), - &this.snapshot, + this.snapshot.as_ref(), this.app_metadata, ) .await?; @@ -560,12 +569,14 @@ impl std::future::IntoFuture for WriteBuilder { // TODO we do not have the table config available, but since we are merging only our newly // created actions, it may be safe to assume, that we want to include all actions. // then again, having only some tombstones may be misleading. - this.snapshot - .merge(actions, &operation, version, true, true)?; - - // TODO should we build checkpoints based on config? - - Ok(DeltaTable::new_with_state(this.log_store, this.snapshot)) + if let Some(mut snapshot) = this.snapshot { + snapshot.merge(actions, &operation, version)?; + Ok(DeltaTable::new_with_state(this.log_store, snapshot)) + } else { + let mut table = DeltaTable::new(this.log_store, Default::default()); + table.update().await?; + Ok(table) + } }) } } @@ -645,11 +656,11 @@ mod tests { .await .unwrap(); assert_eq!(table.version(), 1); - assert_eq!(table.get_file_uris().count(), 1); + assert_eq!(table.get_files_count(), 1); table.load().await.unwrap(); assert_eq!(table.history(None).await.unwrap().len(), 2); assert_eq!( - table.history(None).await.unwrap()[1] + table.history(None).await.unwrap()[0] .info .clone() .into_iter() @@ -668,11 +679,11 @@ mod tests { .await .unwrap(); assert_eq!(table.version(), 2); - assert_eq!(table.get_file_uris().count(), 2); + assert_eq!(table.get_files_count(), 2); table.load().await.unwrap(); assert_eq!(table.history(None).await.unwrap().len(), 3); assert_eq!( - table.history(None).await.unwrap()[2] + table.history(None).await.unwrap()[0] .info .clone() .into_iter() @@ -691,11 +702,11 @@ mod tests { .await .unwrap(); assert_eq!(table.version(), 3); - assert_eq!(table.get_file_uris().count(), 1); + assert_eq!(table.get_files_count(), 1); table.load().await.unwrap(); assert_eq!(table.history(None).await.unwrap().len(), 4); assert_eq!( - table.history(None).await.unwrap()[3] + table.history(None).await.unwrap()[0] .info .clone() .into_iter() @@ -812,7 +823,7 @@ mod tests { .await .unwrap(); assert_eq!(table.version(), 0); - assert_eq!(table.get_file_uris().count(), 1) + assert_eq!(table.get_files_count(), 1) } #[tokio::test] @@ -825,7 +836,7 @@ mod tests { .await .unwrap(); assert_eq!(table.version(), 0); - assert_eq!(table.get_file_uris().count(), 2); + assert_eq!(table.get_files_count(), 2); let table = DeltaOps::new_in_memory() .write(vec![batch]) @@ -834,7 +845,7 @@ mod tests { .await .unwrap(); assert_eq!(table.version(), 0); - assert_eq!(table.get_file_uris().count(), 4) + assert_eq!(table.get_files_count(), 4) } #[tokio::test] diff --git a/crates/deltalake-core/src/protocol/checkpoints.rs b/crates/deltalake-core/src/protocol/checkpoints.rs index 482dac797e..d4ed6a1f07 100644 --- a/crates/deltalake-core/src/protocol/checkpoints.rs +++ b/crates/deltalake-core/src/protocol/checkpoints.rs @@ -82,7 +82,12 @@ pub const CHECKPOINT_RECORD_BATCH_SIZE: usize = 5000; /// Creates checkpoint at current table version pub async fn create_checkpoint(table: &DeltaTable) -> Result<(), ProtocolError> { - create_checkpoint_for(table.version(), table.get_state(), table.log_store.as_ref()).await?; + create_checkpoint_for( + table.version(), + table.snapshot().map_err(|_| ProtocolError::NoMetaData)?, + table.log_store.as_ref(), + ) + .await?; Ok(()) } @@ -91,7 +96,8 @@ pub async fn create_checkpoint(table: &DeltaTable) -> Result<(), ProtocolError> pub async fn cleanup_metadata(table: &DeltaTable) -> Result { let log_retention_timestamp = Utc::now().timestamp_millis() - table - .get_state() + .snapshot() + .map_err(|_| ProtocolError::NoMetaData)? .table_config() .log_retention_duration() .as_millis() as i64; @@ -114,14 +120,11 @@ pub async fn create_checkpoint_from_table_uri_and_cleanup( let table = open_table_with_version(table_uri, version) .await .map_err(|err| ProtocolError::Generic(err.to_string()))?; - create_checkpoint_for(version, table.get_state(), table.log_store.as_ref()).await?; + let snapshot = table.snapshot().map_err(|_| ProtocolError::NoMetaData)?; + create_checkpoint_for(version, snapshot, table.log_store.as_ref()).await?; - let enable_expired_log_cleanup = cleanup.unwrap_or_else(|| { - table - .get_state() - .table_config() - .enable_expired_log_cleanup() - }); + let enable_expired_log_cleanup = + cleanup.unwrap_or_else(|| snapshot.table_config().enable_expired_log_cleanup()); if table.version() >= 0 && enable_expired_log_cleanup { let deleted_log_num = cleanup_metadata(&table).await?; @@ -246,7 +249,7 @@ fn parquet_bytes_from_state( state: &DeltaTableState, mut tombstones: Vec, ) -> Result<(CheckPoint, bytes::Bytes), ProtocolError> { - let current_metadata = state.metadata()?; + let current_metadata = state.metadata(); let schema = current_metadata.schema()?; let partition_col_data_types = get_partition_col_data_types(&schema, ¤t_metadata); @@ -273,7 +276,7 @@ fn parquet_bytes_from_state( remove.extended_file_metadata = Some(false); } } - + let files = state.files().unwrap(); // protocol let jsons = std::iter::once(Action::Protocol(Protocol { min_reader_version: state.protocol().min_reader_version, @@ -310,7 +313,7 @@ fn parquet_bytes_from_state( })) .map(|a| serde_json::to_value(a).map_err(ProtocolError::from)) // adds - .chain(state.files().iter().map(|f| { + .chain(files.iter().map(|f| { checkpoint_add_from_state(f, partition_col_data_types.as_slice(), &stats_conversions) })); @@ -544,7 +547,8 @@ mod tests { .unwrap(); assert_eq!(table.version(), 0); assert_eq!(table.get_schema().unwrap(), &table_schema); - let res = create_checkpoint_for(0, table.get_state(), table.log_store.as_ref()).await; + let res = + create_checkpoint_for(0, table.snapshot().unwrap(), table.log_store.as_ref()).await; assert!(res.is_ok()); // Look at the "files" and verify that the _last_checkpoint has the right version @@ -573,7 +577,7 @@ mod tests { .unwrap(); assert_eq!(table.version(), 0); assert_eq!(table.get_schema().unwrap(), &table_schema); - match create_checkpoint_for(1, table.get_state(), table.log_store.as_ref()).await { + match create_checkpoint_for(1, table.snapshot().unwrap(), table.log_store.as_ref()).await { Ok(_) => { /* * If a checkpoint is allowed to be created here, it will use the passed in diff --git a/crates/deltalake-core/src/protocol/mod.rs b/crates/deltalake-core/src/protocol/mod.rs index 0d320d0f6f..fe1803ef6e 100644 --- a/crates/deltalake-core/src/protocol/mod.rs +++ b/crates/deltalake-core/src/protocol/mod.rs @@ -882,7 +882,7 @@ mod tests { // test table with partitions let path = "../deltalake-test/tests/data/delta-0.8.0-null-partition"; let table = crate::open_table(path).await.unwrap(); - let actions = table.get_state().add_actions_table(true).unwrap(); + let actions = table.get_state().unwrap().add_actions_table(true).unwrap(); let actions = sort_batch_by(&actions, "path").unwrap(); let mut expected_columns: Vec<(&str, ArrayRef)> = vec![ @@ -901,7 +901,7 @@ mod tests { assert_eq!(expected, actions); - let actions = table.get_state().add_actions_table(false).unwrap(); + let actions = table.get_state().unwrap().add_actions_table(false).unwrap(); let actions = sort_batch_by(&actions, "path").unwrap(); expected_columns[4] = ( @@ -921,7 +921,7 @@ mod tests { // test table with partitions let path = "../deltalake-test/tests/data/table_with_deletion_logs"; let table = crate::open_table(path).await.unwrap(); - let actions = table.get_state().add_actions_table(true).unwrap(); + let actions = table.get_state().unwrap().add_actions_table(true).unwrap(); let actions = sort_batch_by(&actions, "path").unwrap(); let actions = actions .project(&[ @@ -979,7 +979,7 @@ mod tests { assert_eq!(expected, actions); - let actions = table.get_state().add_actions_table(false).unwrap(); + let actions = table.get_state().unwrap().add_actions_table(false).unwrap(); let actions = sort_batch_by(&actions, "path").unwrap(); let actions = actions .project(&[ @@ -1028,7 +1028,7 @@ mod tests { let path = "../deltalake-test/tests/data/simple_table"; let table = crate::open_table(path).await.unwrap(); - let actions = table.get_state().add_actions_table(true).unwrap(); + let actions = table.get_state().unwrap().add_actions_table(true).unwrap(); let actions = sort_batch_by(&actions, "path").unwrap(); let expected_columns: Vec<(&str, ArrayRef)> = vec![ @@ -1067,7 +1067,7 @@ mod tests { assert_eq!(expected, actions); - let actions = table.get_state().add_actions_table(false).unwrap(); + let actions = table.get_state().unwrap().add_actions_table(false).unwrap(); let actions = sort_batch_by(&actions, "path").unwrap(); // For now, this column is ignored. @@ -1085,7 +1085,7 @@ mod tests { // test table with column mapping and partitions let path = "../deltalake-test/tests/data/table_with_column_mapping"; let table = crate::open_table(path).await.unwrap(); - let actions = table.get_state().add_actions_table(true).unwrap(); + let actions = table.get_state().unwrap().add_actions_table(true).unwrap(); let expected_columns: Vec<(&str, ArrayRef)> = vec![ ( "path", @@ -1159,7 +1159,7 @@ mod tests { // test table with stats let path = "../deltalake-test/tests/data/delta-0.8.0"; let table = crate::open_table(path).await.unwrap(); - let actions = table.get_state().add_actions_table(true).unwrap(); + let actions = table.get_state().unwrap().add_actions_table(true).unwrap(); let actions = sort_batch_by(&actions, "path").unwrap(); let expected_columns: Vec<(&str, ArrayRef)> = vec![ @@ -1205,7 +1205,7 @@ mod tests { let mut table = crate::open_table(path).await.unwrap(); table.load_version(1).await.unwrap(); - let actions = table.get_state().add_actions_table(true).unwrap(); + let actions = table.get_state().unwrap().add_actions_table(true).unwrap(); let expected_columns: Vec<(&str, ArrayRef)> = vec![ ( @@ -1386,7 +1386,7 @@ mod tests { ); assert_eq!(expected, actions); - let actions = table.get_state().add_actions_table(false).unwrap(); + let actions = table.get_state().unwrap().add_actions_table(false).unwrap(); // For brevity, just checking a few nested columns in stats assert_eq!( diff --git a/crates/deltalake-core/src/table/builder.rs b/crates/deltalake-core/src/table/builder.rs index 8d543b8d39..c5e9e8e0a6 100644 --- a/crates/deltalake-core/src/table/builder.rs +++ b/crates/deltalake-core/src/table/builder.rs @@ -51,7 +51,7 @@ pub enum DeltaVersion { } /// Configuration options for delta table -#[derive(Debug, Serialize, Deserialize, Clone)] +#[derive(Debug, Serialize, Deserialize, Clone, PartialEq)] #[serde(rename_all = "camelCase")] pub struct DeltaTableConfig { /// Indicates whether our use case requires tracking tombstones. @@ -437,9 +437,6 @@ fn ensure_file_location_exists(path: PathBuf) -> DeltaResult<()> { #[cfg(test)] mod tests { - use itertools::Itertools; - use object_store::path::Path; - use super::*; use crate::storage::DefaultObjectStoreFactory; @@ -555,31 +552,4 @@ mod tests { let url = ensure_table_uri(&expected).unwrap(); assert_eq!(expected.as_str().trim_end_matches('/'), url.as_str()); } - - #[tokio::test] - async fn read_delta_table_ignoring_tombstones() { - let table = DeltaTableBuilder::from_uri("../deltalake-test/tests/data/delta-0.8.0") - .without_tombstones() - .load() - .await - .unwrap(); - assert!( - table - .get_state() - .all_tombstones(table.object_store().clone()) - .await - .unwrap() - .collect_vec() - .is_empty(), - "loading without tombstones should skip tombstones" - ); - - assert_eq!( - table.get_files_iter().collect_vec(), - vec![ - Path::from("part-00000-c9b90f86-73e6-46c8-93ba-ff6bfaf892a1-c000.snappy.parquet"), - Path::from("part-00000-04ec9591-0b73-459e-8d18-ba5711d6cbe1-c000.snappy.parquet") - ] - ); - } } diff --git a/crates/deltalake-core/src/table/mod.rs b/crates/deltalake-core/src/table/mod.rs index fbb868267d..2faa6b88c5 100644 --- a/crates/deltalake-core/src/table/mod.rs +++ b/crates/deltalake-core/src/table/mod.rs @@ -1,13 +1,13 @@ //! Delta Table read and write implementation +use std::cmp::Ordering; use std::collections::HashMap; use std::fmt; use std::fmt::Formatter; -use std::{cmp::Ordering, collections::HashSet}; use chrono::{DateTime, Utc}; -use futures::{StreamExt, TryStreamExt}; -use object_store::{path::Path, Error as ObjectStoreError, ObjectStore}; +use futures::TryStreamExt; +use object_store::{path::Path, ObjectStore}; use serde::de::{Error, SeqAccess, Visitor}; use serde::ser::SerializeSeq; use serde::{Deserialize, Deserializer, Serialize, Serializer}; @@ -16,18 +16,13 @@ use tracing::debug; use self::builder::DeltaTableConfig; use self::state::DeltaTableState; use crate::errors::DeltaTableError; -use crate::kernel::{ - Action, Add, CommitInfo, DataCheck, DataType, Metadata, Protocol, ReaderFeatures, StructType, - WriterFeatures, -}; +use crate::kernel::{Action, Add, CommitInfo, DataCheck, DataType, Metadata, Protocol, StructType}; use crate::logstore::LogStoreRef; use crate::logstore::{self, LogStoreConfig}; use crate::partitions::PartitionFilter; -use crate::protocol::{ - find_latest_check_point_for_version, get_last_checkpoint, DeltaOperation, ProtocolError, - SaveMode, Stats, -}; +use crate::protocol::Stats; use crate::storage::{commit_uri_from_version, ObjectStoreRef}; +use crate::DeltaResult; pub mod builder; pub mod config; @@ -199,15 +194,11 @@ pub enum PeekCommit { /// In memory representation of a Delta Table pub struct DeltaTable { /// The state of the table as of the most recent loaded Delta log entry. - pub state: DeltaTableState, + pub state: Option, /// the load options used during load pub config: DeltaTableConfig, /// log store pub(crate) log_store: LogStoreRef, - /// file metadata for latest checkpoint - last_check_point: Option, - /// table versions associated with timestamps - version_timestamp: HashMap, } impl Serialize for DeltaTable { @@ -219,8 +210,6 @@ impl Serialize for DeltaTable { seq.serialize_element(&self.state)?; seq.serialize_element(&self.config)?; seq.serialize_element(self.log_store.config())?; - seq.serialize_element(&self.last_check_point)?; - seq.serialize_element(&self.version_timestamp)?; seq.end() } } @@ -255,19 +244,11 @@ impl<'de> Deserialize<'de> for DeltaTable { let log_store = crate::logstore::logstore_for(storage_config.location, storage_config.options) .map_err(|_| A::Error::custom("Failed deserializing LogStore"))?; - let last_check_point = seq - .next_element()? - .ok_or_else(|| A::Error::invalid_length(0, &self))?; - let version_timestamp = seq - .next_element()? - .ok_or_else(|| A::Error::invalid_length(0, &self))?; let table = DeltaTable { state, config, log_store, - last_check_point, - version_timestamp, }; Ok(table) } @@ -284,11 +265,9 @@ impl DeltaTable { /// call one of the `open_table` helper methods instead. pub fn new(log_store: LogStoreRef, config: DeltaTableConfig) -> Self { Self { - state: DeltaTableState::with_version(-1), + state: None, log_store, config, - last_check_point: None, - version_timestamp: HashMap::new(), } } @@ -299,11 +278,9 @@ impl DeltaTable { /// please call one of the `open_table` helper methods instead. pub(crate) fn new_with_state(log_store: LogStoreRef, state: DeltaTableState) -> Self { Self { - state, + state: Some(state), log_store, config: Default::default(), - last_check_point: None, - version_timestamp: HashMap::new(), } } @@ -349,13 +326,6 @@ impl DeltaTable { checkpoint_data_paths } - #[cfg(feature = "parquet")] - async fn restore_checkpoint(&mut self, check_point: CheckPoint) -> Result<(), DeltaTableError> { - self.state = DeltaTableState::from_checkpoint(self, &check_point).await?; - - Ok(()) - } - /// returns the latest available version of the table pub async fn get_latest_version(&self) -> Result { self.log_store.get_latest_version(self.version()).await @@ -363,14 +333,19 @@ impl DeltaTable { /// Currently loaded version of the table pub fn version(&self) -> i64 { - self.state.version() + self.state.as_ref().map(|s| s.version()).unwrap_or(-1) } /// Load DeltaTable with data from latest checkpoint pub async fn load(&mut self) -> Result<(), DeltaTableError> { - self.last_check_point = None; - self.state = DeltaTableState::with_version(-1); - self.update().await + self.update_incremental(None).await + } + + /// Updates the DeltaTable to the most recent state committed to the transaction log by + /// loading the last checkpoint and incrementally applying each version since. + #[cfg(feature = "parquet")] + pub async fn update(&mut self) -> Result<(), DeltaTableError> { + self.update_incremental(None).await } /// Get the list of actions for the next commit @@ -389,38 +364,8 @@ impl DeltaTable { Ok(PeekCommit::New(next_version, actions.unwrap())) } - /// Updates the DeltaTable to the most recent state committed to the transaction log by - /// loading the last checkpoint and incrementally applying each version since. - #[cfg(feature = "parquet")] - pub async fn update(&mut self) -> Result<(), DeltaTableError> { - match get_last_checkpoint(self.log_store.as_ref()).await { - Ok(last_check_point) => { - debug!("update with latest checkpoint {last_check_point:?}"); - if Some(last_check_point) == self.last_check_point { - self.update_incremental(None).await - } else { - self.last_check_point = Some(last_check_point); - self.restore_checkpoint(last_check_point).await?; - self.update_incremental(None).await - } - } - Err(ProtocolError::CheckpointNotFound) => { - debug!("update without checkpoint"); - self.update_incremental(None).await - } - Err(err) => Err(DeltaTableError::from(err)), - } - } - - /// Updates the DeltaTable to the most recent state committed to the transaction log. - #[cfg(not(feature = "parquet"))] - pub async fn update(&mut self) -> Result<(), DeltaTableError> { - self.update_incremental(None).await - } - /// Updates the DeltaTable to the latest version by incrementally applying newer versions. /// It assumes that the table is already updated to the current version `self.version`. - #[deprecated(since = "0.17.0")] pub async fn update_incremental( &mut self, max_version: Option, @@ -429,111 +374,49 @@ impl DeltaTable { "incremental update with version({}) and max_version({max_version:?})", self.version(), ); - - // update to latest version if given max_version is not larger than current version - let max_version = max_version.filter(|x| x > &self.version()); - let max_version: i64 = match max_version { - Some(x) => x, - None => self.get_latest_version().await?, - }; - - let buf_size = self.config.log_buffer_size; - - let log_store = self.log_store.clone(); - let mut log_stream = futures::stream::iter(self.version() + 1..max_version + 1) - .map(|version| { - let log_store = log_store.clone(); - async move { - if let Some(data) = log_store.read_commit_entry(version).await? { - Ok(Some((version, logstore::get_actions(version, data).await?))) - } else { - Ok(None) - } - } - }) - .buffered(buf_size); - - while let Some(res) = log_stream.next().await { - let (new_version, actions) = match res { - Ok(Some((version, actions))) => (version, actions), - Ok(None) => break, - Err(err) => return Err(err), - }; - - debug!("merging table state with version: {new_version}"); - // NOTE just a dummy operation for API transitioning. - let op = DeltaOperation::Write { - mode: SaveMode::Append, - partition_by: None, - predicate: None, - }; - self.state.merge( - actions, - &op, - new_version, - self.config.require_tombstones, - self.config.require_files, - )?; - if self.version() == max_version { - return Ok(()); + match self.state.as_mut() { + Some(state) => { + state + .update(self.log_store.object_store(), max_version) + .await + } + _ => { + let state = DeltaTableState::try_new( + &Path::default(), + self.log_store.object_store(), + self.config.clone(), + max_version, + ) + .await?; + self.state = Some(state); + Ok(()) } } - - if self.version() == -1 { - return Err(DeltaTableError::not_a_table(self.table_uri())); - } - - Ok(()) } /// Loads the DeltaTable state for the given version. pub async fn load_version(&mut self, version: i64) -> Result<(), DeltaTableError> { - // check if version is valid - let commit_uri = commit_uri_from_version(version); - match self.object_store().head(&commit_uri).await { - Ok(_) => {} - Err(ObjectStoreError::NotFound { .. }) => { - return Err(DeltaTableError::InvalidVersion(version)); - } - Err(e) => { - return Err(DeltaTableError::from(e)); - } - } - - // 1. find latest checkpoint below version - #[cfg(feature = "parquet")] - match find_latest_check_point_for_version(self.log_store.as_ref(), version).await? { - Some(check_point) => { - self.restore_checkpoint(check_point).await?; - } - None => { - // no checkpoint found, clear table state and start from the beginning - self.state = DeltaTableState::with_version(-1); + if let Some(snapshot) = &self.state { + if snapshot.version() > version { + self.state = None; } } - - debug!("update incrementally from version {version}"); - // 2. apply all logs starting from checkpoint - self.update_incremental(Some(version)).await?; - - Ok(()) + self.update_incremental(Some(version)).await } - pub(crate) async fn get_version_timestamp( - &mut self, - version: i64, - ) -> Result { - match self.version_timestamp.get(&version) { - Some(ts) => Ok(*ts), + pub(crate) async fn get_version_timestamp(&self, version: i64) -> Result { + match self + .state + .as_ref() + .and_then(|s| s.version_timestamp(version)) + { + Some(ts) => Ok(ts), None => { let meta = self .object_store() .head(&commit_uri_from_version(version)) .await?; let ts = meta.last_modified.timestamp_millis(); - // also cache timestamp for version - self.version_timestamp.insert(version, ts); - Ok(ts) } } @@ -545,12 +428,11 @@ impl DeltaTable { /// it returns all commits from the earliest commit. pub async fn history(&self, limit: Option) -> Result, DeltaTableError> { let infos = self - .state + .snapshot()? .snapshot - .as_ref() - .unwrap() .snapshot() - .commit_infos(self.object_store()) + .commit_infos(self.object_store(), limit) + .await? .try_collect::>() .await?; Ok(infos.into_iter().filter_map(|b| b).collect()) @@ -560,8 +442,11 @@ impl DeltaTable { pub fn get_active_add_actions_by_partitions<'a>( &'a self, filters: &'a [PartitionFilter], - ) -> Result + '_, DeltaTableError> { - self.state.get_active_add_actions_by_partitions(filters) + ) -> Result + '_, DeltaTableError> { + self.state + .as_ref() + .ok_or(DeltaTableError::NoMetadata)? + .get_active_add_actions_by_partitions(filters) } /// Returns the file list tracked in current table state filtered by provided @@ -596,101 +481,100 @@ impl DeltaTable { /// Returns an iterator of file names present in the loaded state #[inline] - pub fn get_files_iter(&self) -> impl Iterator + '_ { - self.state.file_paths_iter() - } - - /// Returns a collection of file names present in the loaded state - #[deprecated(since = "0.17.0", note = "use get_files_iter() instead")] - #[inline] - pub fn get_files(&self) -> Vec { - self.state.file_paths_iter().collect() + pub fn get_files_iter(&self) -> DeltaResult + '_> { + Ok(self + .state + .as_ref() + .ok_or(DeltaTableError::NoMetadata)? + .file_paths_iter()?) } - /// Returns file names present in the loaded state in HashSet - #[deprecated(since = "0.17.0", note = "use get_files_iter() instead")] - pub fn get_file_set(&self) -> HashSet { - self.state.file_paths_iter().collect() + /// Returns a URIs for all active files present in the current table version. + pub fn get_file_uris(&self) -> DeltaResult + '_> { + Ok(self + .state + .as_ref() + .ok_or(DeltaTableError::NoMetadata)? + .file_paths_iter()? + .map(|path| self.log_store.to_uri(&path))) } - /// Returns a URIs for all active files present in the current table version. - pub fn get_file_uris(&self) -> impl Iterator + '_ { - self.state - .file_paths_iter() - .map(|path| self.log_store.to_uri(&path)) + /// Get the number of files in the table - retrn 0 if no metadata is loaded + pub fn get_files_count(&self) -> usize { + self.state.as_ref().map(|s| s.files_count()).unwrap_or(0) } /// Returns statistics for files, in order - pub fn get_stats(&self) -> impl Iterator, DeltaTableError>> + '_ { - self.state.files().iter().map(|add| { - add.get_stats() - .map_err(|e| DeltaTableError::InvalidStatsJson { json_err: e }) - }) + pub fn get_stats( + &self, + ) -> DeltaResult, DeltaTableError>> + '_> { + Ok(self + .state + .as_ref() + .ok_or(DeltaTableError::NoMetadata)? + .files()? + .into_iter() + .map(|add| { + add.get_stats() + .map_err(|e| DeltaTableError::InvalidStatsJson { json_err: e }) + })) } /// Returns partition values for files, in order pub fn get_partition_values( &self, - ) -> impl Iterator>> + '_ { - self.state.files().iter().map(|add| &add.partition_values) + ) -> DeltaResult>> + '_> { + Ok(self + .state + .as_ref() + .ok_or(DeltaTableError::NoMetadata)? + .files()? + .into_iter() + .map(|add| add.partition_values)) + } + + /// Returns the currently loaded state snapshot. + pub fn snapshot(&self) -> DeltaResult<&DeltaTableState> { + self.state.as_ref().ok_or(DeltaTableError::NotInitialized) } /// Returns the currently loaded state snapshot. - pub fn get_state(&self) -> &DeltaTableState { - &self.state + pub fn get_state(&self) -> Option<&DeltaTableState> { + self.state.as_ref() } /// Returns current table protocol - pub fn protocol(&self) -> &Protocol { - self.state.protocol() + pub fn protocol(&self) -> DeltaResult<&Protocol> { + Ok(self + .state + .as_ref() + .ok_or(DeltaTableError::NoMetadata)? + .protocol()) } /// Returns the metadata associated with the loaded state. pub fn metadata(&self) -> Result<&Metadata, DeltaTableError> { - Ok(self.state.metadata()?) + Ok(self.snapshot()?.metadata()) } /// Returns the current version of the DeltaTable based on the loaded metadata. - pub fn get_app_transaction_version(&self) -> &HashMap { - self.state.app_transaction_version() - } - - /// Returns the minimum reader version supported by the DeltaTable based on the loaded - /// metadata. - #[deprecated(since = "0.17.0", note = "use protocol().min_reader_version instead")] - pub fn get_min_reader_version(&self) -> i32 { - self.state.protocol().min_reader_version - } - - /// Returns the minimum writer version supported by the DeltaTable based on the loaded - /// metadata. - #[deprecated(since = "0.17.0", note = "use protocol().min_writer_version instead")] - pub fn get_min_writer_version(&self) -> i32 { - self.state.protocol().min_writer_version - } - - /// Returns current supported reader features by this table - #[deprecated(since = "0.17.0", note = "use protocol().reader_features instead")] - pub fn get_reader_features(&self) -> Option<&HashSet> { - self.state.protocol().reader_features.as_ref() - } - - /// Returns current supported writer features by this table - #[deprecated(since = "0.17.0", note = "use protocol().writer_features instead")] - pub fn get_writer_features(&self) -> Option<&HashSet> { - self.state.protocol().writer_features.as_ref() + pub fn get_app_transaction_version(&self) -> HashMap { + self.state + .as_ref() + .map(|s| s.app_transaction_version().clone()) + .unwrap_or_default() } /// Return table schema parsed from transaction log. Return None if table hasn't been loaded or /// no metadata was found in the log. pub fn schema(&self) -> Option<&StructType> { - self.state.schema() + Some(self.snapshot().ok()?.schema()) } /// Return table schema parsed from transaction log. Return `DeltaTableError` if table hasn't /// been loaded or no metadata was found in the log. pub fn get_schema(&self) -> Result<&StructType, DeltaTableError> { - self.schema().ok_or(DeltaTableError::NoSchema) + Ok(self.snapshot()?.schema()) } /// Time travel Delta table to the latest version that's created at or before provided @@ -736,22 +620,7 @@ impl DeltaTable { impl fmt::Display for DeltaTable { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { writeln!(f, "DeltaTable({})", self.table_uri())?; - writeln!(f, "\tversion: {}", self.version())?; - match self.state.metadata() { - Ok(metadata) => { - writeln!(f, "\tmetadata: {metadata:?}")?; - } - _ => { - writeln!(f, "\tmetadata: None")?; - } - } - writeln!( - f, - "\tmin_version: read={}, write={}", - self.state.protocol().min_reader_version, - self.state.protocol().min_writer_version - )?; - writeln!(f, "\tfiles count: {}", self.state.files().len()) + writeln!(f, "\tversion: {}", self.version()) } } diff --git a/crates/deltalake-core/src/table/state.rs b/crates/deltalake-core/src/table/state.rs index 90d7fd428b..2dffb4c408 100644 --- a/crates/deltalake-core/src/table/state.rs +++ b/crates/deltalake-core/src/table/state.rs @@ -1,142 +1,95 @@ //! The module for delta table state. use std::collections::HashMap; -use std::collections::HashSet; -use std::io::{BufRead, BufReader, Cursor}; use std::sync::Arc; use chrono::Utc; use futures::TryStreamExt; -use lazy_static::lazy_static; -use object_store::{path::Path, Error as ObjectStoreError, ObjectStore}; +use object_store::{path::Path, ObjectStore}; use serde::{Deserialize, Serialize}; use super::config::TableConfig; use super::get_partition_col_data_types; use crate::errors::DeltaTableError; use crate::kernel::EagerSnapshot; -use crate::kernel::{Action, Add, CommitInfo, DataType, DomainMetadata, Remove, StructType}; +use crate::kernel::{Action, Add, DataType, Remove, StructType}; use crate::kernel::{Metadata, Protocol}; use crate::partitions::{DeltaTablePartition, PartitionFilter}; use crate::protocol::DeltaOperation; -use crate::protocol::ProtocolError; -use crate::storage::commit_uri_from_version; use crate::DeltaResult; -use crate::DeltaTable; #[cfg(feature = "parquet")] -use super::{CheckPoint, DeltaTableConfig}; +use super::DeltaTableConfig; /// State snapshot currently held by the Delta Table instance. -#[derive(Debug, Clone, Serialize, Deserialize, Default)] +#[derive(Debug, Clone, Serialize, Deserialize)] #[serde(rename_all = "camelCase")] pub struct DeltaTableState { - // current table version represented by this table state - version: i64, - // A remove action should remain in the state of the table as a tombstone until it has expired. - // A tombstone expires when the creation timestamp of the delta file exceeds the expiration - tombstones: HashSet, - // active files for table state - files: Vec, - // Information added to individual commits - commit_infos: Vec, - // Domain metadatas provided by the system or user - domain_metadatas: Vec, app_transaction_version: HashMap, - protocol: Option, - metadata: Option, - pub(crate) snapshot: Option, + pub(crate) snapshot: EagerSnapshot, } impl DeltaTableState { - /// Create Table state with specified version - pub fn with_version(version: i64) -> Self { - Self { - version, - ..Self::default() - } + /// Create a new DeltaTableState + pub async fn try_new( + table_root: &Path, + store: Arc, + config: DeltaTableConfig, + version: Option, + ) -> DeltaResult { + let snapshot = EagerSnapshot::try_new(table_root, store.clone(), config, version).await?; + Ok(Self { + snapshot, + app_transaction_version: HashMap::new(), + }) } /// Return table version pub fn version(&self) -> i64 { - self.version + self.snapshot.version() } - /// Construct a delta table state object from commit version. - pub async fn from_commit(table: &DeltaTable, version: i64) -> Result { - let commit_uri = commit_uri_from_version(version); - let commit_log_bytes = match table.object_store().get(&commit_uri).await { - Ok(get) => Ok(get.bytes().await?), - Err(ObjectStoreError::NotFound { .. }) => Err(ProtocolError::EndOfLog), - Err(source) => Err(ProtocolError::ObjectStore { source }), - }?; - let reader = BufReader::new(Cursor::new(commit_log_bytes)); - - let mut new_state = DeltaTableState::with_version(version); - for line in reader.lines() { - let action: Action = serde_json::from_str(line?.as_str())?; - new_state.process_action( - action, - table.config.require_tombstones, - table.config.require_files, - )?; - } - - Ok(new_state) + /// Get the timestamp when a version commit was created. + /// This is the timestamp of the commit file. + /// If the commit file is not present, None is returned. + pub fn version_timestamp(&self, version: i64) -> Option { + self.snapshot.version_timestamp(version) } /// Construct a delta table state object from a list of actions #[cfg(test)] - pub fn from_actions(actions: Vec, version: i64) -> Result { - let mut new_state = DeltaTableState::with_version(version); - for action in actions { - new_state.process_action(action, true, true)?; - } - Ok(new_state) - } - - /// Update DeltaTableState with checkpoint data. - #[cfg(feature = "parquet")] - pub fn process_checkpoint_bytes( - &mut self, - data: bytes::Bytes, - table_config: &DeltaTableConfig, - ) -> Result<(), DeltaTableError> { - use parquet::file::reader::{FileReader, SerializedFileReader}; - - let preader = SerializedFileReader::new(data)?; - let schema = preader.metadata().file_metadata().schema(); - if !schema.is_group() { - return Err(DeltaTableError::from(ProtocolError::Generic( - "Action record in checkpoint should be a struct".to_string(), - ))); - } - for record in preader.get_row_iter(None)? { - self.process_action( - Action::from_parquet_record(schema, &record.unwrap())?, - table_config.require_tombstones, - table_config.require_files, - )?; - } - - Ok(()) - } - - /// Construct a delta table state object from checkpoint. - #[cfg(feature = "parquet")] - pub async fn from_checkpoint( - table: &DeltaTable, - check_point: &CheckPoint, - ) -> Result { - let checkpoint_data_paths = table.get_checkpoint_data_paths(check_point); - let mut new_state = Self::with_version(check_point.version); - - for f in &checkpoint_data_paths { - let obj = table.object_store().get(f).await?.bytes().await?; - new_state.process_checkpoint_bytes(obj, &table.config)?; - } - - Ok(new_state) + pub fn from_actions(actions: Vec) -> DeltaResult { + use crate::protocol::SaveMode; + let metadata = actions + .iter() + .find_map(|a| match a { + Action::Metadata(m) => Some(m.clone()), + _ => None, + }) + .ok_or(DeltaTableError::NotInitialized)?; + let protocol = actions + .iter() + .find_map(|a| match a { + Action::Protocol(p) => Some(p.clone()), + _ => None, + }) + .ok_or(DeltaTableError::NotInitialized)?; + + let commit_data = [( + actions, + DeltaOperation::Create { + mode: SaveMode::Append, + location: Path::default().to_string(), + protocol: protocol.clone(), + metadata: metadata.clone(), + }, + None, + )]; + let snapshot = EagerSnapshot::new_test(&commit_data).unwrap(); + Ok(Self { + app_transaction_version: Default::default(), + snapshot, + }) } /// Full list of tombstones (remove actions) representing files removed from table state). @@ -146,9 +99,8 @@ impl DeltaTableState { ) -> DeltaResult> { Ok(self .snapshot - .as_ref() - .map(|s| s.snapshot().tombstones(store)) - .unwrap_or_else(|| Ok(Box::pin(futures::stream::empty())))? + .snapshot() + .tombstones(store)? .try_collect::>() .await? .into_iter() @@ -174,17 +126,25 @@ impl DeltaTableState { /// Full list of add actions representing all parquet files that are part of the current /// delta table state. - pub fn files(&self) -> &Vec { - self.files.as_ref() + pub fn files(&self) -> DeltaResult> { + Ok(self.snapshot.file_actions()?.collect()) + } + + /// Get the number of files in the current table state + pub fn files_count(&self) -> usize { + self.snapshot.files_count() } /// Returns an iterator of file names present in the loaded state #[inline] - pub fn file_paths_iter(&self) -> impl Iterator + '_ { - self.files.iter().map(|add| match Path::parse(&add.path) { - Ok(path) => path, - Err(_) => Path::from(add.path.as_ref()), - }) + pub fn file_paths_iter(&self) -> DeltaResult + '_> { + Ok(self + .snapshot + .file_actions()? + .map(|add| match Path::parse(&add.path) { + Ok(path) => path, + Err(_) => Path::from(add.path.as_ref()), + })) } /// HashMap containing the last txn version stored for every app id writing txn @@ -195,31 +155,22 @@ impl DeltaTableState { /// The most recent protocol of the table. pub fn protocol(&self) -> &Protocol { - lazy_static! { - static ref DEFAULT_PROTOCOL: Protocol = Protocol::default(); - } - self.protocol.as_ref().unwrap_or(&DEFAULT_PROTOCOL) + self.snapshot.protocol() } /// The most recent metadata of the table. - pub fn metadata(&self) -> Result<&Metadata, ProtocolError> { - self.metadata.as_ref().ok_or(ProtocolError::NoMetaData) + pub fn metadata(&self) -> &Metadata { + self.snapshot.metadata() } /// The table schema - pub fn schema(&self) -> Option<&StructType> { - self.snapshot.as_ref().map(|s| s.schema()) + pub fn schema(&self) -> &StructType { + self.snapshot.schema() } /// Well known table configuration pub fn table_config(&self) -> TableConfig<'_> { - lazy_static! { - static ref DUMMY_CONF: HashMap> = HashMap::new(); - } - self.metadata - .as_ref() - .map(|meta| TableConfig(&meta.configuration)) - .unwrap_or_else(|| TableConfig(&DUMMY_CONF)) + self.snapshot.table_config() } /// Merges new state information into our state @@ -233,106 +184,24 @@ impl DeltaTableState { pub fn merge( &mut self, actions: Vec, - _operation: &DeltaOperation, + operation: &DeltaOperation, version: i64, - require_tombstones: bool, - require_files: bool, - ) -> Result<(), ProtocolError> { - let mut new_state = DeltaTableState::with_version(version); - for action in actions { - new_state.process_action(action, true, true)?; - } - - if !new_state.tombstones.is_empty() { - self.files - .retain(|a| !new_state.tombstones.contains(a.path.as_str())); - } - - if require_tombstones && require_files { - new_state.tombstones.into_iter().for_each(|r| { - self.tombstones.insert(r); - }); - - if !new_state.files.is_empty() { - new_state.files.iter().for_each(|s| { - self.tombstones.remove(s.path.as_str()); - }); - } - } - - if require_files { - self.files.append(&mut new_state.files); - } - - if new_state.metadata.is_some() { - self.metadata = new_state.metadata.take(); - } - - if new_state.protocol.is_some() { - self.protocol = new_state.protocol.take(); - } - - new_state - .app_transaction_version - .drain() - .for_each(|(app_id, version)| { - *self - .app_transaction_version - .entry(app_id) - .or_insert(version) = version - }); - - if !new_state.commit_infos.is_empty() { - self.commit_infos.append(&mut new_state.commit_infos); - } - - if self.version < new_state.version { - self.version = new_state.version + ) -> Result<(), DeltaTableError> { + let commit_infos = vec![(actions, operation.clone(), None)]; + let new_version = self.snapshot.advance(&commit_infos)?; + if new_version != version { + return Err(DeltaTableError::Generic("Version mismatch".to_string())); } - Ok(()) } - /// Process given action by updating current state. - fn process_action( + /// Update the state of the table to the given version. + pub async fn update( &mut self, - action: Action, - require_tombstones: bool, - require_files: bool, - ) -> Result<(), ProtocolError> { - match action { - // TODO: optionally load CDC into TableState - Action::Cdc(_v) => {} - Action::Add(v) => { - if require_files { - self.files.push(v); - } - } - Action::Remove(v) => { - if require_tombstones && require_files { - self.tombstones.insert(v); - } - } - Action::Protocol(v) => { - self.protocol = Some(v); - } - Action::Metadata(v) => { - self.metadata = Some(v.clone()); - } - Action::Txn(v) => { - *self - .app_transaction_version - .entry(v.app_id) - .or_insert(v.version) = v.version; - } - Action::CommitInfo(v) => { - self.commit_infos.push(v); - } - Action::DomainMetadata(v) => { - self.domain_metadatas.push(v); - } - } - + store: Arc, + version: Option, + ) -> Result<(), DeltaTableError> { + self.snapshot.update(store, version).await?; Ok(()) } @@ -340,8 +209,8 @@ impl DeltaTableState { pub fn get_active_add_actions_by_partitions<'a>( &'a self, filters: &'a [PartitionFilter], - ) -> Result + '_, DeltaTableError> { - let current_metadata = self.metadata()?; + ) -> Result + '_, DeltaTableError> { + let current_metadata = self.metadata(); let nonpartitioned_columns: Vec = filters .iter() @@ -355,14 +224,12 @@ impl DeltaTableState { }); } - let partition_col_data_types: HashMap<&String, &DataType> = get_partition_col_data_types( - self.schema().ok_or(DeltaTableError::NoMetadata)?, - current_metadata, - ) - .into_iter() - .collect(); + let partition_col_data_types: HashMap<&String, &DataType> = + get_partition_col_data_types(self.schema(), current_metadata) + .into_iter() + .collect(); - let actions = self.files().iter().filter(move |add| { + let actions = self.files()?.into_iter().filter(move |add| { let partitions = add .partition_values .iter() @@ -375,58 +242,3 @@ impl DeltaTableState { Ok(actions) } } - -#[cfg(test)] -mod tests { - use super::*; - use crate::kernel::Txn; - use pretty_assertions::assert_eq; - - #[test] - fn state_round_trip() { - let expected = DeltaTableState { - version: 0, - tombstones: Default::default(), - files: vec![], - commit_infos: vec![], - domain_metadatas: vec![], - app_transaction_version: Default::default(), - metadata: None, - protocol: None, - snapshot: None, - }; - let bytes = serde_json::to_vec(&expected).unwrap(); - let actual: DeltaTableState = serde_json::from_slice(&bytes).unwrap(); - assert_eq!(actual.version, expected.version); - } - - #[test] - fn state_records_new_txn_version() { - let mut app_transaction_version = HashMap::new(); - app_transaction_version.insert("abc".to_string(), 1); - app_transaction_version.insert("xyz".to_string(), 1); - - let mut state = DeltaTableState { - version: -1, - files: vec![], - commit_infos: vec![], - domain_metadatas: vec![], - tombstones: HashSet::new(), - protocol: None, - metadata: None, - app_transaction_version, - snapshot: None, - }; - - let txn_action = Action::Txn(Txn { - app_id: "abc".to_string(), - version: 2, - last_updated: Some(0), - }); - - state.process_action(txn_action, false, true).unwrap(); - - assert_eq!(2, *state.app_transaction_version().get("abc").unwrap()); - assert_eq!(1, *state.app_transaction_version().get("xyz").unwrap()); - } -} diff --git a/crates/deltalake-core/src/table/state_arrow.rs b/crates/deltalake-core/src/table/state_arrow.rs index 9526784915..69ebd4a816 100644 --- a/crates/deltalake-core/src/table/state_arrow.rs +++ b/crates/deltalake-core/src/table/state_arrow.rs @@ -19,7 +19,7 @@ use itertools::Itertools; use super::config::ColumnMappingMode; use super::state::DeltaTableState; use crate::errors::DeltaTableError; -use crate::kernel::{DataType as DeltaDataType, StructType}; +use crate::kernel::{Add, DataType as DeltaDataType, StructType}; use crate::protocol::{ColumnCountStat, ColumnValueStat, Stats}; impl DeltaTableState { @@ -54,27 +54,22 @@ impl DeltaTableState { &self, flatten: bool, ) -> Result { + let files = self.files()?; let mut paths = arrow::array::StringBuilder::with_capacity( - self.files().len(), - self.files().iter().map(|add| add.path.len()).sum(), + files.len(), + files.iter().map(|add| add.path.len()).sum(), ); - for action in self.files() { + for action in &files { paths.append_value(&action.path); } - let size = self - .files() - .iter() - .map(|file| file.size) - .collect::(); - let mod_time: TimestampMillisecondArray = self - .files() + let size = files.iter().map(|file| file.size).collect::(); + let mod_time: TimestampMillisecondArray = files .iter() .map(|file| file.modification_time) .collect::>() .into(); - let data_change = self - .files() + let data_change = files .iter() .map(|file| Some(file.data_change)) .collect::(); @@ -86,10 +81,10 @@ impl DeltaTableState { (Cow::Borrowed("data_change"), Arc::new(data_change)), ]; - let metadata = self.metadata()?; + let metadata = self.metadata(); if !metadata.partition_columns.is_empty() { - let partition_cols_batch = self.partition_columns_as_batch(flatten)?; + let partition_cols_batch = self.partition_columns_as_batch(flatten, &files)?; arrays.extend( partition_cols_batch .schema() @@ -100,7 +95,7 @@ impl DeltaTableState { ) } - if self.files().iter().any(|add| add.stats.is_some()) { + if files.iter().any(|add| add.stats.is_some()) { let stats = self.stats_as_batch(flatten)?; arrays.extend( stats @@ -111,8 +106,8 @@ impl DeltaTableState { .zip(stats.columns().iter().map(Arc::clone)), ); } - if self.files().iter().any(|add| add.deletion_vector.is_some()) { - let delvs = self.deletion_vectors_as_batch(flatten)?; + if files.iter().any(|add| add.deletion_vector.is_some()) { + let delvs = self.deletion_vectors_as_batch(flatten, &files)?; arrays.extend( delvs .schema() @@ -122,13 +117,13 @@ impl DeltaTableState { .zip(delvs.columns().iter().map(Arc::clone)), ); } - if self.files().iter().any(|add| { + if files.iter().any(|add| { add.tags .as_ref() .map(|tags| !tags.is_empty()) .unwrap_or(false) }) { - let tags = self.tags_as_batch(flatten)?; + let tags = self.tags_as_batch(flatten, &files)?; arrays.extend( tags.schema() .fields @@ -144,8 +139,9 @@ impl DeltaTableState { fn partition_columns_as_batch( &self, flatten: bool, + files: &Vec, ) -> Result { - let metadata = self.metadata()?; + let metadata = self.metadata(); let column_mapping_mode = self.table_config().column_mapping_mode(); let partition_column_types: Vec = metadata .partition_columns @@ -190,7 +186,7 @@ impl DeltaTableState { .collect::, DeltaTableError>>()?, }; // Append values - for action in self.files() { + for action in files { for (name, maybe_value) in action.partition_values.iter() { let logical_name = match column_mapping_mode { ColumnMappingMode::None => name.as_str(), @@ -262,9 +258,9 @@ impl DeltaTableState { fn tags_as_batch( &self, flatten: bool, + files: &Vec, ) -> Result { - let tag_keys: HashSet<&str> = self - .files() + let tag_keys: HashSet<&str> = files .iter() .flat_map(|add| add.tags.as_ref().map(|tags| tags.keys())) .flatten() @@ -275,12 +271,12 @@ impl DeltaTableState { .map(|&key| { ( key, - arrow::array::StringBuilder::with_capacity(self.files().len(), 64), + arrow::array::StringBuilder::with_capacity(files.len(), 64), ) }) .collect(); - for add in self.files() { + for add in files { for &key in &tag_keys { if let Some(value) = add .tags @@ -322,15 +318,16 @@ impl DeltaTableState { fn deletion_vectors_as_batch( &self, flatten: bool, + files: &Vec, ) -> Result { - let capacity = self.files().len(); + let capacity = files.len(); let mut storage_type = arrow::array::StringBuilder::with_capacity(capacity, 1); let mut path_or_inline_div = arrow::array::StringBuilder::with_capacity(capacity, 64); let mut offset = arrow::array::Int32Builder::with_capacity(capacity); let mut size_in_bytes = arrow::array::Int32Builder::with_capacity(capacity); let mut cardinality = arrow::array::Int64Builder::with_capacity(capacity); - for add in self.files() { + for add in files { if let Some(value) = &add.deletion_vector { storage_type.append_value(&value.storage_type); path_or_inline_div.append_value(value.path_or_inline_dv.clone()); @@ -401,7 +398,7 @@ impl DeltaTableState { flatten: bool, ) -> Result { let stats: Vec> = self - .files() + .files()? .iter() .map(|f| { f.get_stats() @@ -415,8 +412,7 @@ impl DeltaTableState { .map(|maybe_stat| maybe_stat.as_ref().map(|stat| stat.num_records)) .collect::>>(), ); - let metadata = self.metadata()?; - let schema = &metadata.schema()?; + let schema = self.schema(); #[derive(Debug)] struct ColStats<'a> { diff --git a/crates/deltalake-core/src/writer/mod.rs b/crates/deltalake-core/src/writer/mod.rs index fd3d2ed4e7..3efeef91ab 100644 --- a/crates/deltalake-core/src/writer/mod.rs +++ b/crates/deltalake-core/src/writer/mod.rs @@ -135,7 +135,8 @@ pub trait DeltaWriter { /// and commit the changes to the Delta log, creating a new table version. async fn flush_and_commit(&mut self, table: &mut DeltaTable) -> Result { let adds: Vec<_> = self.flush().await?.drain(..).map(Action::Add).collect(); - let partition_cols = table.metadata()?.partition_columns.clone(); + let snapshot = table.snapshot()?; + let partition_cols = snapshot.metadata().partition_columns.clone(); let partition_by = if !partition_cols.is_empty() { Some(partition_cols) } else { @@ -150,7 +151,7 @@ pub trait DeltaWriter { table.log_store.as_ref(), &adds, operation, - &table.state, + Some(snapshot), None, ) .await?; diff --git a/crates/deltalake-core/src/writer/test_utils.rs b/crates/deltalake-core/src/writer/test_utils.rs index 964e60c1c6..9ccc136592 100644 --- a/crates/deltalake-core/src/writer/test_utils.rs +++ b/crates/deltalake-core/src/writer/test_utils.rs @@ -315,7 +315,8 @@ pub mod datafusion { use std::sync::Arc; pub async fn get_data(table: &DeltaTable) -> Vec { - let table = DeltaTable::new_with_state(table.log_store.clone(), table.state.clone()); + let table = + DeltaTable::new_with_state(table.log_store.clone(), table.snapshot().unwrap().clone()); let ctx = SessionContext::new(); ctx.register_table("test", Arc::new(table)).unwrap(); ctx.sql("select * from test") diff --git a/crates/deltalake-core/tests/checkpoint_writer.rs b/crates/deltalake-core/tests/checkpoint_writer.rs index 3efdba5e69..10474ceffd 100644 --- a/crates/deltalake-core/tests/checkpoint_writer.rs +++ b/crates/deltalake-core/tests/checkpoint_writer.rs @@ -51,7 +51,7 @@ mod simple_checkpoint { // delta table should load just fine with the checkpoint in place let table_result = deltalake_core::open_table(table_location).await.unwrap(); let table = table_result; - let files = table.get_files_iter(); + let files = table.get_files_iter().unwrap(); assert_eq!(12, files.count()); } @@ -138,7 +138,7 @@ mod delete_expired_delta_log_in_checkpoint { table.update().await.unwrap(); // make table to read the checkpoint assert_eq!( - table.get_files_iter().collect::>(), + table.get_files_iter().unwrap().collect::>(), vec![ ObjectStorePath::from(a1.path.as_ref()), ObjectStorePath::from(a2.path.as_ref()) @@ -186,7 +186,7 @@ mod delete_expired_delta_log_in_checkpoint { .unwrap(); table.update().await.unwrap(); // make table to read the checkpoint assert_eq!( - table.get_files_iter().collect::>(), + table.get_files_iter().unwrap().collect::>(), vec![ ObjectStorePath::from(a1.path.as_ref()), ObjectStorePath::from(a2.path.as_ref()) @@ -249,7 +249,7 @@ mod checkpoints_with_tombstones { checkpoints::create_checkpoint(&table).await.unwrap(); table.update().await.unwrap(); // make table to read the checkpoint assert_eq!( - table.get_files_iter().collect::>(), + table.get_files_iter().unwrap().collect::>(), vec![ ObjectStorePath::from(a1.path.as_ref()), ObjectStorePath::from(a2.path.as_ref()) @@ -258,12 +258,13 @@ mod checkpoints_with_tombstones { let (removes1, opt1) = pseudo_optimize(&mut table, 5 * 59 * 1000).await; assert_eq!( - table.get_files_iter().collect::>(), + table.get_files_iter().unwrap().collect::>(), vec![ObjectStorePath::from(opt1.path.as_ref())] ); assert_eq!( table - .get_state() + .snapshot() + .unwrap() .all_tombstones(table.object_store().clone()) .await .unwrap() @@ -274,12 +275,13 @@ mod checkpoints_with_tombstones { checkpoints::create_checkpoint(&table).await.unwrap(); table.update().await.unwrap(); // make table to read the checkpoint assert_eq!( - table.get_files_iter().collect::>(), + table.get_files_iter().unwrap().collect::>(), vec![ObjectStorePath::from(opt1.path.as_ref())] ); assert_eq!( table - .get_state() + .snapshot() + .unwrap() .all_tombstones(table.object_store().clone()) .await .unwrap() @@ -352,6 +354,7 @@ mod checkpoints_with_tombstones { async fn pseudo_optimize(table: &mut DeltaTable, offset_millis: i64) -> (HashSet, Add) { let removes: HashSet = table .get_files_iter() + .unwrap() .map(|p| Remove { path: p.to_string(), deletion_timestamp: Some(Utc::now().timestamp_millis() - offset_millis), diff --git a/crates/deltalake-core/tests/command_filesystem_check.rs b/crates/deltalake-core/tests/command_filesystem_check.rs index 6636fa05f2..e05f088d16 100644 --- a/crates/deltalake-core/tests/command_filesystem_check.rs +++ b/crates/deltalake-core/tests/command_filesystem_check.rs @@ -22,25 +22,25 @@ async fn test_filesystem_check(context: &IntegrationContext) -> TestResult { context.object_store().delete(&path).await?; let table = context.table_builder(TestTables::Simple).load().await?; - let version = table.state.version(); - let active = table.state.files().len(); + let version = table.snapshot()?.version(); + let active = table.snapshot()?.files()?.len(); // Validate a Dry run does not mutate the table log and indentifies orphaned add actions let op = DeltaOps::from(table); let (table, metrics) = op.filesystem_check().with_dry_run(true).await?; - assert_eq!(version, table.state.version()); - assert_eq!(active, table.state.files().len()); + assert_eq!(version, table.snapshot()?.version()); + assert_eq!(active, table.snapshot()?.files()?.len()); assert_eq!(vec![file.to_string()], metrics.files_removed); // Validate a run updates the table version with proper remove actions let op = DeltaOps::from(table); let (table, metrics) = op.filesystem_check().await?; - assert_eq!(version + 1, table.state.version()); - assert_eq!(active - 1, table.state.files().len()); + assert_eq!(version + 1, table.snapshot()?.version()); + assert_eq!(active - 1, table.snapshot()?.files()?.len()); assert_eq!(vec![file.to_string()], metrics.files_removed); let remove = table - .state + .snapshot()? .all_tombstones(table.object_store().clone()) .await? .collect::>(); @@ -50,8 +50,8 @@ async fn test_filesystem_check(context: &IntegrationContext) -> TestResult { // An additional run should return an empty list of orphaned actions let op = DeltaOps::from(table); let (table, metrics) = op.filesystem_check().await?; - assert_eq!(version + 1, table.state.version()); - assert_eq!(active - 1, table.state.files().len()); + assert_eq!(version + 1, table.snapshot()?.version()); + assert_eq!(active - 1, table.snapshot()?.files()?.len()); assert!(metrics.files_removed.is_empty()); Ok(()) @@ -75,18 +75,19 @@ async fn test_filesystem_check_partitioned() -> TestResult { .table_builder(TestTables::Delta0_8_0Partitioned) .load() .await?; - let version = table.state.version(); - let active = table.state.files().len(); + + let version = table.snapshot()?.version(); + let active = table.snapshot()?.files()?.len(); // Validate a run updates the table version with proper remove actions let op = DeltaOps::from(table); let (table, metrics) = op.filesystem_check().await?; - assert_eq!(version + 1, table.state.version()); - assert_eq!(active - 1, table.state.files().len()); + assert_eq!(version + 1, table.snapshot()?.version()); + assert_eq!(active - 1, table.snapshot()?.files()?.len()); assert_eq!(vec![file.to_string()], metrics.files_removed); let remove = table - .state + .snapshot()? .all_tombstones(table.object_store().clone()) .await? .collect::>(); diff --git a/crates/deltalake-core/tests/command_optimize.rs b/crates/deltalake-core/tests/command_optimize.rs index 7b1444cf4b..5b85137271 100644 --- a/crates/deltalake-core/tests/command_optimize.rs +++ b/crates/deltalake-core/tests/command_optimize.rs @@ -170,7 +170,7 @@ async fn test_optimize_non_partitioned_table() -> Result<(), Box> { .await?; let version = dt.version(); - assert_eq!(dt.get_state().files().len(), 5); + assert_eq!(dt.get_files_count(), 5); let optimize = DeltaOps(dt).optimize().with_target_size(2_000_000); let (dt, metrics) = optimize.await?; @@ -180,7 +180,7 @@ async fn test_optimize_non_partitioned_table() -> Result<(), Box> { assert_eq!(metrics.num_files_removed, 4); assert_eq!(metrics.total_considered_files, 5); assert_eq!(metrics.partitions_optimized, 1); - assert_eq!(dt.get_state().files().len(), 2); + assert_eq!(dt.get_files_count(), 2); Ok(()) } @@ -236,7 +236,7 @@ async fn test_optimize_with_partitions() -> Result<(), Box> { assert_eq!(version + 1, dt.version()); assert_eq!(metrics.num_files_added, 1); assert_eq!(metrics.num_files_removed, 2); - assert_eq!(dt.get_state().files().len(), 3); + assert_eq!(dt.get_files_count(), 3); Ok(()) } @@ -269,7 +269,7 @@ async fn test_conflict_for_remove_actions() -> Result<(), Box> { let filter = vec![PartitionFilter::try_from(("date", "=", "2022-05-22"))?]; let plan = create_merge_plan( OptimizeType::Compact, - &dt.state, + dt.snapshot()?, &filter, None, WriterProperties::builder().build(), @@ -277,7 +277,7 @@ async fn test_conflict_for_remove_actions() -> Result<(), Box> { let uri = context.tmp_dir.path().to_str().to_owned().unwrap(); let other_dt = deltalake_core::open_table(uri).await?; - let add = &other_dt.get_state().files()[0]; + let add = &other_dt.snapshot()?.files()?[0]; let remove = Remove { path: add.path.clone(), deletion_timestamp: Some( @@ -301,13 +301,13 @@ async fn test_conflict_for_remove_actions() -> Result<(), Box> { other_dt.log_store().as_ref(), &vec![Action::Remove(remove)], operation, - &other_dt.state, + Some(other_dt.snapshot()?), None, ) .await?; let maybe_metrics = plan - .execute(dt.log_store(), &dt.state, 1, 20, None, None) + .execute(dt.log_store(), dt.snapshot()?, 1, 20, None, None) .await; assert!(maybe_metrics.is_err()); @@ -341,7 +341,7 @@ async fn test_no_conflict_for_append_actions() -> Result<(), Box> { let filter = vec![PartitionFilter::try_from(("date", "=", "2022-05-22"))?]; let plan = create_merge_plan( OptimizeType::Compact, - &dt.state, + dt.snapshot()?, &filter, None, WriterProperties::builder().build(), @@ -358,7 +358,7 @@ async fn test_no_conflict_for_append_actions() -> Result<(), Box> { .await?; let metrics = plan - .execute(dt.log_store(), &dt.state, 1, 20, None, None) + .execute(dt.log_store(), dt.snapshot()?, 1, 20, None, None) .await?; assert_eq!(metrics.num_files_added, 1); assert_eq!(metrics.num_files_removed, 2); @@ -391,7 +391,7 @@ async fn test_commit_interval() -> Result<(), Box> { let plan = create_merge_plan( OptimizeType::Compact, - &dt.state, + dt.snapshot()?, &[], None, WriterProperties::builder().build(), @@ -400,7 +400,7 @@ async fn test_commit_interval() -> Result<(), Box> { let metrics = plan .execute( dt.log_store(), - &dt.state, + dt.snapshot()?, 1, 20, Some(Duration::from_secs(0)), // this will cause as many commits as num_files_added @@ -610,7 +610,7 @@ async fn test_commit_info() -> Result<(), Box> { let (dt, metrics) = optimize.await?; let commit_info = dt.history(None).await?; - let last_commit = &commit_info[commit_info.len() - 1]; + let last_commit = &commit_info[0]; let commit_metrics = serde_json::from_value::(last_commit.info["operationMetrics"].clone())?; @@ -719,7 +719,7 @@ async fn test_zorder_unpartitioned() -> Result<(), Box> { assert_eq!(metrics.total_considered_files, 2); // Check data - let files = dt.get_files_iter().collect::>(); + let files = dt.get_files_iter()?.collect::>(); assert_eq!(files.len(), 1); let actual = read_parquet_file(&files[0], dt.object_store()).await?; diff --git a/crates/deltalake-core/tests/command_restore.rs b/crates/deltalake-core/tests/command_restore.rs index ee234f9f8c..8ef0c02e5d 100644 --- a/crates/deltalake-core/tests/command_restore.rs +++ b/crates/deltalake-core/tests/command_restore.rs @@ -101,17 +101,17 @@ async fn test_restore_by_version() -> Result<(), Box> { let result = DeltaOps(table).restore().with_version_to_restore(1).await?; assert_eq!(result.1.num_restored_file, 1); assert_eq!(result.1.num_removed_file, 2); - assert_eq!(result.0.state.version(), 4); + assert_eq!(result.0.snapshot()?.version(), 4); let table_uri = context.tmp_dir.path().to_str().to_owned().unwrap(); let mut table = DeltaOps::try_from_uri(table_uri).await?; table.0.load_version(1).await?; - assert_eq!(table.0.state.files(), result.0.state.files()); + assert_eq!(table.0.snapshot()?.files()?, result.0.snapshot()?.files()?); let result = DeltaOps(result.0) .restore() .with_version_to_restore(0) .await?; - assert_eq!(result.0.state.files().len(), 0); + assert_eq!(result.0.get_files_count(), 0); Ok(()) } @@ -136,7 +136,7 @@ async fn test_restore_by_datetime() -> Result<(), Box> { .await?; assert_eq!(result.1.num_restored_file, 1); assert_eq!(result.1.num_removed_file, 2); - assert_eq!(result.0.state.version(), 4); + assert_eq!(result.0.snapshot()?.version(), 4); Ok(()) } @@ -169,14 +169,14 @@ async fn test_restore_with_error_params() -> Result<(), Box> { async fn test_restore_file_missing() -> Result<(), Box> { let context = setup_test().await?; - for file in context.table.state.files().iter() { + for file in context.table.snapshot()?.files()?.iter() { let p = context.tmp_dir.path().join(file.clone().path); fs::remove_file(p).unwrap(); } for file in context .table - .state + .snapshot()? .all_tombstones(context.table.object_store().clone()) .await? { @@ -196,14 +196,14 @@ async fn test_restore_file_missing() -> Result<(), Box> { async fn test_restore_allow_file_missing() -> Result<(), Box> { let context = setup_test().await?; - for file in context.table.state.files().iter() { + for file in context.table.snapshot()?.files()?.iter() { let p = context.tmp_dir.path().join(file.clone().path); fs::remove_file(p).unwrap(); } for file in context .table - .state + .snapshot()? .all_tombstones(context.table.object_store().clone()) .await? { diff --git a/crates/deltalake-core/tests/commit_info_format.rs b/crates/deltalake-core/tests/commit_info_format.rs index b0e9d2d324..b47850ae30 100644 --- a/crates/deltalake-core/tests/commit_info_format.rs +++ b/crates/deltalake-core/tests/commit_info_format.rs @@ -24,14 +24,14 @@ async fn test_operational_parameters() -> Result<(), Box> { table.log_store().as_ref(), &actions, operation, - &table.state, + Some(table.snapshot()?), None, ) .await?; table.update().await?; let commit_info = table.history(None).await?; - let last_commit = &commit_info[commit_info.len() - 1]; + let last_commit = &commit_info[0]; let parameters = last_commit.operation_parameters.clone().unwrap(); assert_eq!(parameters["mode"], json!("Append")); assert_eq!(parameters["partitionBy"], json!("[\"some_partition\"]")); diff --git a/crates/deltalake-core/tests/fs_common/mod.rs b/crates/deltalake-core/tests/fs_common/mod.rs index 17a9619118..2cd67d1eef 100644 --- a/crates/deltalake-core/tests/fs_common/mod.rs +++ b/crates/deltalake-core/tests/fs_common/mod.rs @@ -124,7 +124,7 @@ pub async fn commit_actions( table.log_store().as_ref(), &actions, operation, - &table.state, + Some(table.snapshot().unwrap()), None, ) .await diff --git a/crates/deltalake-core/tests/integration.rs b/crates/deltalake-core/tests/integration.rs index 95e5d90815..4776d9253a 100644 --- a/crates/deltalake-core/tests/integration.rs +++ b/crates/deltalake-core/tests/integration.rs @@ -42,7 +42,7 @@ async fn test_action_reconciliation() { let a = fs_common::add(3 * 60 * 1000); assert_eq!(1, fs_common::commit_add(&mut table, &a).await); assert_eq!( - table.get_files_iter().collect::>(), + table.get_files_iter().unwrap().collect::>(), vec![Path::from(a.path.clone())] ); @@ -61,10 +61,11 @@ async fn test_action_reconciliation() { }; assert_eq!(2, fs_common::commit_removes(&mut table, vec![&r]).await); - assert_eq!(table.get_files_iter().count(), 0); + assert_eq!(table.get_files_iter().unwrap().count(), 0); assert_eq!( table - .get_state() + .snapshot() + .unwrap() .all_tombstones(table.object_store().clone()) .await .unwrap() @@ -76,13 +77,14 @@ async fn test_action_reconciliation() { // Add removed file back. assert_eq!(3, fs_common::commit_add(&mut table, &a).await); assert_eq!( - table.get_files_iter().collect::>(), + table.get_files_iter().unwrap().collect::>(), vec![Path::from(a.path)] ); // tombstone is removed. assert_eq!( table - .get_state() + .snapshot() + .unwrap() .all_tombstones(table.object_store().clone()) .await .unwrap() diff --git a/crates/deltalake-core/tests/integration_datafusion.rs b/crates/deltalake-core/tests/integration_datafusion.rs index a7ef7943f2..2307c1ba5f 100644 --- a/crates/deltalake-core/tests/integration_datafusion.rs +++ b/crates/deltalake-core/tests/integration_datafusion.rs @@ -211,14 +211,15 @@ mod local { // Trying to execute the write from the input plan without providing Datafusion with a session // state containing the referenced object store in the registry results in an error. - assert!( - WriteBuilder::new(target_table.log_store(), target_table.state.clone()) - .with_input_execution_plan(source_scan.clone()) - .await - .unwrap_err() - .to_string() - .contains("No suitable object store found for delta-rs://") - ); + assert!(WriteBuilder::new( + target_table.log_store(), + target_table.snapshot().ok().cloned() + ) + .with_input_execution_plan(source_scan.clone()) + .await + .unwrap_err() + .to_string() + .contains("No suitable object store found for delta-rs://")); // Register the missing source table object store let source_uri = Url::parse( @@ -238,10 +239,13 @@ mod local { .register_object_store(source_store_url, source_store.object_store()); // Execute write to the target table with the proper state - let target_table = WriteBuilder::new(target_table.log_store(), target_table.state.clone()) - .with_input_execution_plan(source_scan) - .with_input_session_state(state) - .await?; + let target_table = WriteBuilder::new( + target_table.log_store(), + target_table.snapshot().ok().cloned(), + ) + .with_input_execution_plan(source_scan) + .with_input_session_state(state) + .await?; ctx.register_table("target", Arc::new(target_table))?; // Check results @@ -290,7 +294,7 @@ mod local { let table = open_table("../deltalake-test/tests/data/delta-0.8.0") .await .unwrap(); - let statistics = table.state.datafusion_table_statistics()?; + let statistics = table.snapshot()?.datafusion_table_statistics()?; assert_eq!(statistics.num_rows, Precision::Exact(4_usize),); @@ -331,7 +335,7 @@ mod local { let table = open_table("../deltalake-test/tests/data/delta-0.2.0") .await .unwrap(); - let statistics = table.state.datafusion_table_statistics()?; + let statistics = table.snapshot()?.datafusion_table_statistics()?; assert_eq!(statistics.num_rows, Precision::Absent); @@ -370,7 +374,7 @@ mod local { .await .unwrap(); let schema = table.get_schema().unwrap(); - let statistics = table.state.datafusion_table_statistics()?; + let statistics = table.snapshot()?.datafusion_table_statistics()?; assert_eq!(statistics.num_rows, Precision::Exact(12)); // `new_column` statistics diff --git a/crates/deltalake-core/tests/read_delta_log_test.rs b/crates/deltalake-core/tests/read_delta_log_test.rs index 1d60b562d4..445e4384f3 100644 --- a/crates/deltalake-core/tests/read_delta_log_test.rs +++ b/crates/deltalake-core/tests/read_delta_log_test.rs @@ -154,8 +154,8 @@ async fn test_read_liquid_table() -> DeltaResult<()> { async fn test_read_table_features() -> DeltaResult<()> { let mut _table = deltalake_core::open_table("../deltalake-test/tests/data/simple_table_features").await?; - let rf = _table.protocol().reader_features.clone(); - let wf = _table.protocol().writer_features.clone(); + let rf = _table.protocol()?.reader_features.clone(); + let wf = _table.protocol()?.writer_features.clone(); assert!(rf.is_some()); assert!(wf.is_some()); diff --git a/crates/deltalake-test/src/concurrent.rs b/crates/deltalake-test/src/concurrent.rs index 5fdd8e000f..452965e486 100644 --- a/crates/deltalake-test/src/concurrent.rs +++ b/crates/deltalake-test/src/concurrent.rs @@ -38,9 +38,9 @@ async fn prepare_table( .await?; assert_eq!(0, table.version()); - assert_eq!(1, table.protocol().min_reader_version); - assert_eq!(2, table.protocol().min_writer_version); - assert_eq!(0, table.get_files_iter().count()); + assert_eq!(1, table.protocol()?.min_reader_version); + assert_eq!(2, table.protocol()?.min_writer_version); + // assert_eq!(0, table.get_files_iter().count()); Ok((table, table_uri)) } @@ -142,7 +142,7 @@ impl Worker { self.table.log_store().as_ref(), &actions, operation, - &self.table.state, + Some(self.table.snapshot().unwrap()), None, ) .await diff --git a/crates/deltalake-test/src/lib.rs b/crates/deltalake-test/src/lib.rs index 459b93973a..31f378e86f 100644 --- a/crates/deltalake-test/src/lib.rs +++ b/crates/deltalake-test/src/lib.rs @@ -152,7 +152,7 @@ pub async fn add_file( table.log_store().as_ref(), &actions, operation, - &table.state, + table.state.as_ref(), None, ) .await @@ -190,7 +190,7 @@ pub async fn remove_file( table.log_store().as_ref(), &actions, operation, - &table.state, + table.state.as_ref(), None, ) .await diff --git a/crates/deltalake-test/src/read.rs b/crates/deltalake-test/src/read.rs index c40e0b8ee7..3300544ff9 100644 --- a/crates/deltalake-test/src/read.rs +++ b/crates/deltalake-test/src/read.rs @@ -41,10 +41,10 @@ async fn read_simple_table(integration: &IntegrationContext) -> TestResult { .await?; assert_eq!(table.version(), 4); - assert_eq!(table.protocol().min_writer_version, 2); - assert_eq!(table.protocol().min_reader_version, 1); + assert_eq!(table.protocol()?.min_writer_version, 2); + assert_eq!(table.protocol()?.min_reader_version, 1); assert_eq!( - table.get_files_iter().collect::>(), + table.get_files_iter()?.collect::>(), vec![ Path::from("part-00000-c1777d7d-89d9-4790-b38a-6ee7e24456b1-c000.snappy.parquet"), Path::from("part-00001-7891c33d-cedc-47c3-88a6-abcfb049d3b4-c000.snappy.parquet"), @@ -54,7 +54,7 @@ async fn read_simple_table(integration: &IntegrationContext) -> TestResult { ] ); let tombstones = table - .get_state() + .snapshot()? .all_tombstones(table.object_store().clone()) .await? .collect::>(); @@ -85,10 +85,10 @@ async fn read_simple_table_with_version(integration: &IntegrationContext) -> Tes .await?; assert_eq!(table.version(), 3); - assert_eq!(table.protocol().min_writer_version, 2); - assert_eq!(table.protocol().min_reader_version, 1); + assert_eq!(table.protocol()?.min_writer_version, 2); + assert_eq!(table.protocol()?.min_reader_version, 1); assert_eq!( - table.get_files_iter().collect::>(), + table.get_files_iter()?.collect::>(), vec![ Path::from("part-00000-c1777d7d-89d9-4790-b38a-6ee7e24456b1-c000.snappy.parquet"), Path::from("part-00001-7891c33d-cedc-47c3-88a6-abcfb049d3b4-c000.snappy.parquet"), @@ -99,7 +99,7 @@ async fn read_simple_table_with_version(integration: &IntegrationContext) -> Tes ] ); let tombstones = table - .get_state() + .snapshot()? .all_tombstones(table.object_store().clone()) .await? .collect::>(); @@ -130,8 +130,8 @@ async fn read_golden(integration: &IntegrationContext) -> TestResult { .unwrap(); assert_eq!(table.version(), 0); - assert_eq!(table.protocol().min_writer_version, 2); - assert_eq!(table.protocol().min_reader_version, 1); + assert_eq!(table.protocol()?.min_writer_version, 2); + assert_eq!(table.protocol()?.min_reader_version, 1); Ok(()) } @@ -166,7 +166,7 @@ async fn read_encoded_table(integration: &IntegrationContext, root_path: &str) - .await?; assert_eq!(table.version(), 0); - assert_eq!(table.get_files_iter().count(), 2); + assert_eq!(table.get_files_iter()?.count(), 2); Ok(()) } diff --git a/crates/deltalake-test/tests/data/delta-0.2.0/.part-00000-512e1537-8aaa-4193-b8b4-bef3de0de409-c000.snappy.parquet.crc b/crates/deltalake-test/tests/data/delta-0.2.0/.part-00000-512e1537-8aaa-4193-b8b4-bef3de0de409-c000.snappy.parquet.crc deleted file mode 100644 index c29a858708eb4c45095445cc8e22cc00c70dfd4e..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 12 TcmYc;N@ieSU}Ctj>y#$|6zc=v diff --git a/crates/deltalake-test/tests/data/delta-0.2.0/.part-00000-7c2deba3-1994-4fb8-bc07-d46c948aa415-c000.snappy.parquet.crc b/crates/deltalake-test/tests/data/delta-0.2.0/.part-00000-7c2deba3-1994-4fb8-bc07-d46c948aa415-c000.snappy.parquet.crc deleted file mode 100644 index c29a858708eb4c45095445cc8e22cc00c70dfd4e..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 12 TcmYc;N@ieSU}Ctj>y#$|6zc=v diff --git a/crates/deltalake-test/tests/data/delta-0.2.0/.part-00000-b44fcdb0-8b06-4f3a-8606-f8311a96f6dc-c000.snappy.parquet.crc b/crates/deltalake-test/tests/data/delta-0.2.0/.part-00000-b44fcdb0-8b06-4f3a-8606-f8311a96f6dc-c000.snappy.parquet.crc deleted file mode 100644 index c29a858708eb4c45095445cc8e22cc00c70dfd4e..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 12 TcmYc;N@ieSU}Ctj>y#$|6zc=v diff --git a/crates/deltalake-test/tests/data/delta-0.2.0/.part-00000-cb6b150b-30b8-4662-ad28-ff32ddab96d2-c000.snappy.parquet.crc b/crates/deltalake-test/tests/data/delta-0.2.0/.part-00000-cb6b150b-30b8-4662-ad28-ff32ddab96d2-c000.snappy.parquet.crc deleted file mode 100644 index ccbde4e02b3c957ed6db9b6a7d62f35286fd0f52..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 12 TcmYc;N@ieSU}9Kz&@T}H6LEW diff --git a/crates/deltalake-test/tests/data/simple_table_with_checkpoint/.part-00000-f0e955c5-a1e3-4eec-834e-dcc098fc9005-c000.snappy.parquet.crc b/crates/deltalake-test/tests/data/simple_table_with_checkpoint/.part-00000-f0e955c5-a1e3-4eec-834e-dcc098fc9005-c000.snappy.parquet.crc deleted file mode 100644 index dd1d93b5d4725a78e2b29ac463b110c6a44ef18d..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 12 TcmYc;N@ieSU}EUx;9Lg)5PSlI diff --git a/crates/deltalake-test/tests/data/simple_table_with_checkpoint/_delta_log/.00000000000000000010.checkpoint.parquet.crc b/crates/deltalake-test/tests/data/simple_table_with_checkpoint/_delta_log/.00000000000000000010.checkpoint.parquet.crc deleted file mode 100644 index fd993bf8ca6801ce1c313c83ab523810ff2d479f..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 96 zcmV-m0H6P3a$^7h00ICVS1{H0j&uKhxZ%<4NiKfwU};fC)TY*pVcIHW;>ICT8id&% zx!i#xi90Fn4*Cns+;>)5-o anyhow::Result<()> { }; if files_matches.is_present("full_uri") { - table.get_file_uris().for_each(|f| println!("{f}")); + table.get_file_uris()?.for_each(|f| println!("{f}")); } else { - table.get_files_iter().for_each(|f| println!("{f}")); + table.get_files_iter()?.for_each(|f| println!("{f}")); }; } Some(("info", info_matches)) => { diff --git a/python/src/lib.rs b/python/src/lib.rs index d98d07d7e8..0e1f39151b 100644 --- a/python/src/lib.rs +++ b/python/src/lib.rs @@ -148,8 +148,14 @@ impl RawDeltaTable { pub fn protocol_versions(&self) -> PyResult<(i32, i32)> { Ok(( - self._table.protocol().min_reader_version, - self._table.protocol().min_writer_version, + self._table + .protocol() + .map_err(PythonError::from)? + .min_reader_version, + self._table + .protocol() + .map_err(PythonError::from)? + .min_writer_version, )) } @@ -219,6 +225,7 @@ impl RawDeltaTable { Ok(self ._table .get_files_iter() + .map_err(PythonError::from)? .map(|f| f.to_string()) .collect()) } @@ -235,7 +242,11 @@ impl RawDeltaTable { .get_file_uris_by_partitions(&filters) .map_err(PythonError::from)?) } else { - Ok(self._table.get_file_uris().collect()) + Ok(self + ._table + .get_file_uris() + .map_err(PythonError::from)? + .collect()) } } @@ -255,9 +266,12 @@ impl RawDeltaTable { enforce_retention_duration: bool, custom_metadata: Option>, ) -> PyResult> { - let mut cmd = VacuumBuilder::new(self._table.log_store(), self._table.state.clone()) - .with_enforce_retention_duration(enforce_retention_duration) - .with_dry_run(dry_run); + let mut cmd = VacuumBuilder::new( + self._table.log_store(), + self._table.snapshot().map_err(PythonError::from)?.clone(), + ) + .with_enforce_retention_duration(enforce_retention_duration) + .with_dry_run(dry_run); if let Some(retention_period) = retention_hours { cmd = cmd.with_retention_period(Duration::hours(retention_period as i64)); } @@ -285,8 +299,11 @@ impl RawDeltaTable { safe_cast: bool, custom_metadata: Option>, ) -> PyResult { - let mut cmd = UpdateBuilder::new(self._table.log_store(), self._table.state.clone()) - .with_safe_cast(safe_cast); + let mut cmd = UpdateBuilder::new( + self._table.log_store(), + self._table.snapshot().map_err(PythonError::from)?.clone(), + ) + .with_safe_cast(safe_cast); if let Some(writer_props) = writer_properties { cmd = cmd.with_writer_properties( @@ -333,8 +350,11 @@ impl RawDeltaTable { writer_properties: Option>>, custom_metadata: Option>, ) -> PyResult { - let mut cmd = OptimizeBuilder::new(self._table.log_store(), self._table.state.clone()) - .with_max_concurrent_tasks(max_concurrent_tasks.unwrap_or_else(num_cpus::get)); + let mut cmd = OptimizeBuilder::new( + self._table.log_store(), + self._table.snapshot().map_err(PythonError::from)?.clone(), + ) + .with_max_concurrent_tasks(max_concurrent_tasks.unwrap_or_else(num_cpus::get)); if let Some(size) = target_size { cmd = cmd.with_target_size(size); } @@ -386,10 +406,13 @@ impl RawDeltaTable { writer_properties: Option>>, custom_metadata: Option>, ) -> PyResult { - let mut cmd = OptimizeBuilder::new(self._table.log_store(), self._table.state.clone()) - .with_max_concurrent_tasks(max_concurrent_tasks.unwrap_or_else(num_cpus::get)) - .with_max_spill_size(max_spill_size) - .with_type(OptimizeType::ZOrder(z_order_columns)); + let mut cmd = OptimizeBuilder::new( + self._table.log_store(), + self._table.snapshot().map_err(PythonError::from)?.clone(), + ) + .with_max_concurrent_tasks(max_concurrent_tasks.unwrap_or_else(num_cpus::get)) + .with_max_spill_size(max_spill_size) + .with_type(OptimizeType::ZOrder(z_order_columns)); if let Some(size) = target_size { cmd = cmd.with_target_size(size); } @@ -426,8 +449,10 @@ impl RawDeltaTable { constraints: HashMap, custom_metadata: Option>, ) -> PyResult<()> { - let mut cmd = - ConstraintBuilder::new(self._table.log_store(), self._table.get_state().clone()); + let mut cmd = ConstraintBuilder::new( + self._table.log_store(), + self._table.snapshot().map_err(PythonError::from)?.clone(), + ); for (col_name, expression) in constraints { cmd = cmd.with_constraint(col_name.clone(), expression.clone()); @@ -494,7 +519,7 @@ impl RawDeltaTable { let mut cmd = MergeBuilder::new( self._table.log_store(), - self._table.state.clone(), + self._table.snapshot().map_err(PythonError::from)?.clone(), predicate, source_df, ) @@ -642,7 +667,10 @@ impl RawDeltaTable { protocol_downgrade_allowed: bool, custom_metadata: Option>, ) -> PyResult { - let mut cmd = RestoreBuilder::new(self._table.log_store(), self._table.state.clone()); + let mut cmd = RestoreBuilder::new( + self._table.log_store(), + self._table.snapshot().map_err(PythonError::from)?.clone(), + ); if let Some(val) = target { if let Ok(version) = val.extract::() { cmd = cmd.with_version_to_restore(version) @@ -705,16 +733,21 @@ impl RawDeltaTable { self._table .get_files_iter() + .map_err(PythonError::from)? .map(|p| p.to_string()) - .zip(self._table.get_partition_values()) - .zip(self._table.get_stats()) + .zip( + self._table + .get_partition_values() + .map_err(PythonError::from)?, + ) + .zip(self._table.get_stats().map_err(PythonError::from)?) .filter(|((path, _), _)| match &path_set { Some(path_set) => path_set.contains(path), None => true, }) .map(|((path, partition_values), stats)| { let stats = stats.map_err(PythonError::from)?; - let expression = filestats_to_expression(py, &schema, partition_values, stats)?; + let expression = filestats_to_expression(py, &schema, &partition_values, stats)?; Ok((path, expression)) }) .collect() @@ -772,11 +805,15 @@ impl RawDeltaTable { let partition_columns: Vec<&str> = partition_columns.into_iter().collect(); - let active_partitions: HashSet)>> = self + let adds = self ._table - .get_state() + .snapshot() + .map_err(PythonError::from)? .get_active_add_actions_by_partitions(&converted_filters) .map_err(PythonError::from)? + .collect::>(); + let active_partitions: HashSet)>> = adds + .iter() .map(|add| { partition_columns .iter() @@ -820,7 +857,8 @@ impl RawDeltaTable { let add_actions = self ._table - .get_state() + .snapshot() + .map_err(PythonError::from)? .get_active_add_actions_by_partitions(&converted_filters) .map_err(PythonError::from)?; @@ -873,7 +911,7 @@ impl RawDeltaTable { &*store, &actions, operation, - self._table.get_state(), + Some(self._table.snapshot().map_err(PythonError::from)?), app_metadata, )) .map_err(PythonError::from)?; @@ -909,7 +947,8 @@ impl RawDeltaTable { pub fn get_add_actions(&self, flatten: bool) -> PyResult> { Ok(PyArrowType( self._table - .get_state() + .snapshot() + .map_err(PythonError::from)? .add_actions_table(flatten) .map_err(PythonError::from)?, )) @@ -923,7 +962,10 @@ impl RawDeltaTable { writer_properties: Option>>, custom_metadata: Option>, ) -> PyResult { - let mut cmd = DeleteBuilder::new(self._table.log_store(), self._table.state.clone()); + let mut cmd = DeleteBuilder::new( + self._table.log_store(), + self._table.snapshot().map_err(PythonError::from)?.clone(), + ); if let Some(predicate) = predicate { cmd = cmd.with_predicate(predicate); } @@ -955,9 +997,11 @@ impl RawDeltaTable { dry_run: bool, custom_metadata: Option>, ) -> PyResult { - let mut cmd = - FileSystemCheckBuilder::new(self._table.log_store(), self._table.state.clone()) - .with_dry_run(dry_run); + let mut cmd = FileSystemCheckBuilder::new( + self._table.log_store(), + self._table.snapshot().map_err(PythonError::from)?.clone(), + ) + .with_dry_run(dry_run); if let Some(metadata) = custom_metadata { let json_metadata: Map =