From f40221a0f5ad2a72732902362327e2a4a6e535e1 Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Tue, 1 Oct 2024 10:46:49 -0700 Subject: [PATCH 01/66] new Transaction API, write_json. empty commit for now --- kernel/src/actions/mod.rs | 7 ++ kernel/src/actions/transaction.rs | 120 +++++++++++++++++++ kernel/src/engine/default/filesystem.rs | 1 + kernel/src/engine/default/json.rs | 56 +++++++++ kernel/src/engine/default/mod.rs | 26 +++-- kernel/src/engine/sync/json.rs | 9 ++ kernel/src/lib.rs | 30 +++++ kernel/src/table.rs | 7 ++ kernel/src/transaction.rs | 82 +++++++++++++ kernel/tests/write.rs | 148 ++++++++++++++++++++++++ 10 files changed, 479 insertions(+), 7 deletions(-) create mode 100644 kernel/src/actions/transaction.rs create mode 100644 kernel/src/transaction.rs create mode 100644 kernel/tests/write.rs diff --git a/kernel/src/actions/mod.rs b/kernel/src/actions/mod.rs index 3d14eaa02..a186c5402 100644 --- a/kernel/src/actions/mod.rs +++ b/kernel/src/actions/mod.rs @@ -136,6 +136,13 @@ impl Protocol { } } + +#[derive(Debug, Clone, PartialEq, Eq, Schema)] +pub struct CommitInfo { + pub kernel_version: Option, + // pub engine_info: HashMap +} + #[derive(Debug, Clone, PartialEq, Eq, Schema)] pub struct CommitInfo { pub kernel_version: Option, diff --git a/kernel/src/actions/transaction.rs b/kernel/src/actions/transaction.rs new file mode 100644 index 000000000..7555d1e4d --- /dev/null +++ b/kernel/src/actions/transaction.rs @@ -0,0 +1,120 @@ +use std::sync::Arc; + +use crate::actions::visitors::TransactionVisitor; +use crate::actions::{Transaction, get_log_schema, TRANSACTION_NAME}; +use crate::snapshot::Snapshot; +use crate::{DeltaResult, Engine}; + +pub use crate::actions::visitors::TransactionMap; + +pub struct TransactionScanner { + snapshot: Arc, +} + +impl TransactionScanner { + pub fn new(snapshot: Arc) -> Self { + TransactionScanner { snapshot } + } + + /// Scan the entire log for all application ids but terminate early if a specific application id is provided + fn scan_application_transactions( + &self, + engine: &dyn Engine, + application_id: Option<&str>, + ) -> DeltaResult { + let schema = get_log_schema().project(&[TRANSACTION_NAME])?; + + let mut visitor = TransactionVisitor::new(application_id.map(|s| s.to_owned())); + + // when all ids are requested then a full scan of the log to the latest checkpoint is required + let iter = + self.snapshot + .log_segment + .replay(engine, schema.clone(), schema.clone(), None)?; + + for maybe_data in iter { + let (txns, _) = maybe_data?; + txns.extract(schema.clone(), &mut visitor)?; + // if a specific id is requested and a transaction was found, then return + if application_id.is_some() && !visitor.transactions.is_empty() { + break; + } + } + + Ok(visitor.transactions) + } + + /// Scan the Delta Log for the latest transaction entry of an application + pub fn application_transaction( + &self, + engine: &dyn Engine, + application_id: &str, + ) -> DeltaResult> { + let mut transactions = self.scan_application_transactions(engine, Some(application_id))?; + Ok(transactions.remove(application_id)) + } + + /// Scan the Delta Log to obtain the latest transaction for all applications + pub fn application_transactions(&self, engine: &dyn Engine) -> DeltaResult { + self.scan_application_transactions(engine, None) + } +} + +#[cfg(all(test, feature = "default-engine"))] +mod tests { + use std::path::PathBuf; + + use super::*; + use crate::engine::sync::SyncEngine; + use crate::Table; + + fn get_latest_transactions(path: &str, app_id: &str) -> (TransactionMap, Option) { + let path = std::fs::canonicalize(PathBuf::from(path)).unwrap(); + let url = url::Url::from_directory_path(path).unwrap(); + let engine = SyncEngine::new(); + + let table = Table::new(url); + let snapshot = table.snapshot(&engine, None).unwrap(); + let txn_scan = TransactionScanner::new(snapshot.into()); + + ( + txn_scan.application_transactions(&engine).unwrap(), + txn_scan.application_transaction(&engine, app_id).unwrap(), + ) + } + + #[test] + fn test_txn() { + let (txns, txn) = get_latest_transactions("./tests/data/basic_partitioned/", "test"); + assert!(txn.is_none()); + assert_eq!(txns.len(), 0); + + let (txns, txn) = get_latest_transactions("./tests/data/app-txn-no-checkpoint/", "my-app"); + assert!(txn.is_some()); + assert_eq!(txns.len(), 2); + assert_eq!(txns.get("my-app"), txn.as_ref()); + assert_eq!( + txns.get("my-app2"), + Some(Transaction { + app_id: "my-app2".to_owned(), + version: 2, + last_updated: None + }) + .as_ref() + ); + + let (txns, txn) = get_latest_transactions("./tests/data/app-txn-checkpoint/", "my-app"); + assert!(txn.is_some()); + assert_eq!(txns.len(), 2); + assert_eq!(txns.get("my-app"), txn.as_ref()); + assert_eq!( + txns.get("my-app2"), + Some(Transaction { + app_id: "my-app2".to_owned(), + version: 2, + last_updated: None + }) + .as_ref() + ); + } +} diff --git a/kernel/src/engine/default/filesystem.rs b/kernel/src/engine/default/filesystem.rs index b9036d04e..669a0c8ef 100644 --- a/kernel/src/engine/default/filesystem.rs +++ b/kernel/src/engine/default/filesystem.rs @@ -56,6 +56,7 @@ impl FileSystemClient for ObjectStoreFileSystemClient { match meta { Ok(meta) => { let mut location = url.clone(); + println!("listed location: {:?}", meta.location); location.set_path(&format!("/{}", meta.location.as_ref())); sender .send(Ok(FileMeta { diff --git a/kernel/src/engine/default/json.rs b/kernel/src/engine/default/json.rs index da5ad1baa..1b27b2f53 100644 --- a/kernel/src/engine/default/json.rs +++ b/kernel/src/engine/default/json.rs @@ -88,6 +88,62 @@ impl JsonHandler for DefaultJsonHandler { self.readahead, ) } + + // note: for now we just buffer all the data and write it out all at once + fn write_json_file<'a>( + &self, + path: &url::Url, + data: Box> + Send + 'a>, + _overwrite: bool, + ) -> DeltaResult<()> { + // Initialize schema and batches + let mut schema: Option = None; + let mut batches: Vec = Vec::new(); + + for chunk in data { + let arrow_data = ArrowEngineData::try_from_engine_data(chunk)?; + let record_batch = arrow_data.record_batch(); + + if schema.is_none() { + schema = Some(record_batch.schema()); + } + + println!("[Engine put_json] schema: {:#?}", record_batch.schema()); + batches.push(record_batch.clone()); + } + + for batch in batches.iter() { + println!("==============================================================="); + println!("[Engine put_json] batch: {:#?}", batch); + println!("[Engine put_json] schema: {:#?}", batch.schema()); + println!("==============================================================="); + } + + // collect all batches + let batches: Vec<&RecordBatch> = batches.iter().collect(); + + // Write the concatenated batch to JSON + let mut writer = arrow_json::LineDelimitedWriter::new(Vec::new()); + writer.write_batches(&batches)?; + writer.finish()?; + + let buffer = writer.into_inner(); + + println!("[Engine put_json] commit path: {:?}", path.path()); + + // Put if absent + futures::executor::block_on(async { + self.store + .put_opts( + &Path::from(path.path()), + buffer.into(), + object_store::PutMode::Create.into(), + ) + .await + })?; + + Ok(()) + } } /// A [`FileOpener`] that opens a JSON file and yields a [`FileOpenFuture`] diff --git a/kernel/src/engine/default/mod.rs b/kernel/src/engine/default/mod.rs index 5a8ef7af6..5abb0b9c3 100644 --- a/kernel/src/engine/default/mod.rs +++ b/kernel/src/engine/default/mod.rs @@ -40,21 +40,25 @@ pub struct DefaultEngine { impl DefaultEngine { /// Create a new [`DefaultEngine`] instance /// - /// The `path` parameter is used to determine the type of storage used. + /// # Parameters /// - /// The `task_executor` is used to spawn async IO tasks. See [executor::TaskExecutor]. - pub fn try_new(path: &Url, options: I, task_executor: Arc) -> DeltaResult + /// - `table_root`: The URL of the table within storage. + /// - `options`: key/value pairs of options to pass to the object store. + /// - `task_executor`: Used to spawn async IO tasks. See [executor::TaskExecutor]. + pub fn try_new(table_root: &Url, options: I, task_executor: Arc) -> DeltaResult where I: IntoIterator, K: AsRef, V: Into, { - let (store, prefix) = parse_url_opts(path, options)?; + // table root is the path of the table in the ObjectStore + let (store, table_root) = parse_url_opts(table_root, options)?; + println!("DEFAULT ENGINE INIT try_new table root: {:?}", table_root); let store = Arc::new(store); Ok(Self { file_system: Arc::new(ObjectStoreFileSystemClient::new( store.clone(), - prefix, + table_root, task_executor.clone(), )), json: Arc::new(DefaultJsonHandler::new( @@ -67,11 +71,19 @@ impl DefaultEngine { }) } - pub fn new(store: Arc, prefix: Path, task_executor: Arc) -> Self { + /// Create a new [`DefaultEngine`] instance + /// + /// # Parameters + /// + /// - `store`: The object store to use. + /// - `table_root_path`: The root path of the table within storage. + /// - `task_executor`: Used to spawn async IO tasks. See [executor::TaskExecutor]. + pub fn new(store: Arc, table_root_path: Path, task_executor: Arc) -> Self { + println!("DEFAULT ENGINE INIT new table root: {:?}", table_root_path); Self { file_system: Arc::new(ObjectStoreFileSystemClient::new( store.clone(), - prefix, + table_root_path, task_executor.clone(), )), json: Arc::new(DefaultJsonHandler::new( diff --git a/kernel/src/engine/sync/json.rs b/kernel/src/engine/sync/json.rs index 4cc884933..e43fd5290 100644 --- a/kernel/src/engine/sync/json.rs +++ b/kernel/src/engine/sync/json.rs @@ -40,4 +40,13 @@ impl JsonHandler for SyncJsonHandler { ) -> DeltaResult> { arrow_parse_json(json_strings, output_schema) } + + fn write_json_file<'a>( + &self, + _path: &Url, + _data: Box> + Send + 'a>, + _overwrite: bool, + ) -> DeltaResult<()> { + unimplemented!() + } } diff --git a/kernel/src/lib.rs b/kernel/src/lib.rs index 72725e824..69837525f 100644 --- a/kernel/src/lib.rs +++ b/kernel/src/lib.rs @@ -196,6 +196,36 @@ pub trait JsonHandler: Send + Sync { // clone the (potentially large) expression every time we call this function. predicate: Option, ) -> DeltaResult; + + /// Atomically (!) write a single JSON file. Each row of the input data represents an action + /// in the delta log. this PUT must: + /// (1) serialize the data to newline-delimited json (each row is a json object) + /// (2) write the data to the object store atomically (i.e. if the file already exists, fail + /// unless the overwrite flag is set) + /// + /// The JSON data should be written as { "column1": "value1", "column2": "value2", ... } + /// with each row on a new line. + /// + /// Null columns should not be written to the JSON file. For example, if a row has columns + /// ["a", "b"] and the value of "b" is null, the JSON object should be written as { "a": "..." } + /// + /// # Parameters + /// + /// - `path` - URL to write the JSON file to + /// - `data` - Iterator of EngineData to write to the JSON file. each row should be written as + /// a new JSON object appended to the file. (that is, the file is newline-delimeted JSON, and + /// each row is a JSON object on a single line) + /// - `overwrite` - If true, overwrite the file if it exists. If false, the call must fail if + /// the file exists. + /// + /// NOTE: the `overwrite` flag isn't used for the existing commit flow, but will be used in the + /// future to write `_last_checkpoint` files. + fn write_json_file<'a>( + &self, + path: &Url, + data: Box> + Send + 'a>, + overwrite: bool, + ) -> DeltaResult<()>; } /// Provides Parquet file related functionalities to Delta Kernel. diff --git a/kernel/src/table.rs b/kernel/src/table.rs index 590490410..087214851 100644 --- a/kernel/src/table.rs +++ b/kernel/src/table.rs @@ -6,6 +6,7 @@ use std::path::PathBuf; use url::Url; use crate::snapshot::Snapshot; +use crate::transaction::Transaction; use crate::{DeltaResult, Engine, Error, Version}; /// In-memory representation of a Delta table, which acts as an immutable root entity for reading @@ -76,6 +77,12 @@ impl Table { pub fn snapshot(&self, engine: &dyn Engine, version: Option) -> DeltaResult { Snapshot::try_new(self.location.clone(), engine, version) } + + /// Create a new write transaction builder for the table. + pub fn new_transaction(&self, engine: &dyn Engine) -> DeltaResult { + let latest_snapshot = Snapshot::try_new(self.location.clone(), engine, None)?; + Ok(Transaction::new(latest_snapshot)) + } } #[derive(Debug)] diff --git a/kernel/src/transaction.rs b/kernel/src/transaction.rs new file mode 100644 index 000000000..965e8ca9a --- /dev/null +++ b/kernel/src/transaction.rs @@ -0,0 +1,82 @@ +use std::sync::{Arc, LazyLock}; + +use crate::actions::get_log_schema; +use crate::schema::{Schema, StructType}; +use crate::snapshot::Snapshot; +use crate::{DataType, Expression}; +use crate::{DeltaResult, Engine, EngineData}; + +pub struct Transaction { + read_snapshot: Arc, + commit_info: Option, +} + +pub struct CommitInfoData { + data: Box, + schema: Schema, +} + +impl std::fmt::Debug for Transaction { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + f.write_str(&format!( + "Transaction {{ read_snapshot version: {}, commit_info: {} }}", + self.read_snapshot.version(), + self.commit_info.is_some() + )) + } +} + +impl Transaction { + pub fn new(snapshot: impl Into>) -> Self { + Transaction { + read_snapshot: snapshot.into(), + commit_info: None, + } + } + + pub fn commit(self, engine: &dyn Engine) -> DeltaResult { + // lazy lock for the expression + static COMMIT_INFO_EXPR: LazyLock = + LazyLock::new(|| Expression::column("commitInfo")); + + // step one: construct the iterator of actions we want to commit + let action_schema = get_log_schema(); + + let actions = self.commit_info.into_iter().map(|commit_info| { + // commit info has arbitrary schema ex: {engineInfo: string, operation: string} + // we want to bundle it up and put it in the commit_info field of the actions. + let commit_info_evaluator = engine.get_expression_handler().get_evaluator( + commit_info.schema.into(), + COMMIT_INFO_EXPR.clone(), // TODO remove clone? + >::into(action_schema.clone()), + ); + commit_info_evaluator.evaluate(commit_info.data.as_ref()).unwrap() + }); + + // step two: figure out the commit version and path to write + let commit_version = &self.read_snapshot.version() + 1; + let commit_file_name = format!("{:020}", commit_version) + ".json"; + let commit_path = &self + .read_snapshot + .table_root + .join("_delta_log/")? + .join(&commit_file_name)?; + + // step three: commit the actions as a json file in the log + let json_handler = engine.get_json_handler(); + + json_handler.write_json_file(commit_path, Box::new(actions), false)?; + Ok(CommitResult::Committed(commit_version)) + } + + /// Add commit info to the transaction. This is commit-wide metadata that is written as the + /// first action in the commit. Note it is required in order to commit. If the engine does not + /// require any commit info, pass an empty `EngineData`. + pub fn commit_info(&mut self, commit_info: Box, schema: Schema) { + self.commit_info = Some(CommitInfoData { data: commit_info, schema }); + } +} + +pub enum CommitResult { + Committed(crate::Version), +} diff --git a/kernel/tests/write.rs b/kernel/tests/write.rs new file mode 100644 index 000000000..81a731224 --- /dev/null +++ b/kernel/tests/write.rs @@ -0,0 +1,148 @@ +use std::sync::Arc; + +use arrow::array::{ArrayRef, BooleanArray, Int64Array, StringArray}; +use arrow::record_batch::RecordBatch; +use arrow_schema::Field; +use object_store::memory::InMemory; +use object_store::path::Path; +use object_store::ObjectStore; +use url::Url; + +use delta_kernel::engine::arrow_data::ArrowEngineData; +use delta_kernel::engine::default::executor::tokio::TokioBackgroundExecutor; +use delta_kernel::engine::default::DefaultEngine; +use delta_kernel::schema::{DataType, SchemaRef, StructField, StructType}; +use delta_kernel::{DeltaResult, Engine, EngineData, Table}; + +// fixme use in macro below +// const PROTOCOL_METADATA_TEMPLATE: &'static str = r#"{{"protocol":{{"minReaderVersion":3,"minWriterVersion":7,"readerFeatures":[],"writerFeatures":[]}}}} +// {{"metaData":{{"id":"{}","format":{{"provider":"parquet","options":{{}}}},"schemaString":"{}","partitionColumns":[],"configuration":{{}},"createdTime":1677811175819}}}}"#; + +// setup default engine with in-memory object store. +fn setup( + table_name: &str, +) -> ( + Arc, + DefaultEngine, + Url, +) { + let table_root_path = Path::from(format!("/{table_name}")); + let url = Url::parse(&format!("memory:///{}/", table_root_path)).unwrap(); + let storage = Arc::new(InMemory::new()); + ( + storage.clone(), + DefaultEngine::new( + storage, + table_root_path, + Arc::new(TokioBackgroundExecutor::new()), + ), + url, + ) +} + +// we provide this table creation function since we only do appends to existing tables for now. +// this will just create an empty table with the given schema. (just protocol + metadata actions) +async fn create_table( + store: Arc, + table_path: Url, + // fixme use this schema + _schema: SchemaRef, + schema_string: &str, + partition_columns: &str, +) -> Result> { + // put 0.json with protocol + metadata + let table_id = "test_id"; + let data = format!( + r#"{{"protocol":{{"minReaderVersion":3,"minWriterVersion":7,"readerFeatures":[],"writerFeatures":[]}}}} +{{"metaData":{{"id":"{}","format":{{"provider":"parquet","options":{{}}}},"schemaString":"{}","partitionColumns":[{}],"configuration":{{}},"createdTime":1677811175819}}}}"#, + table_id, schema_string, partition_columns + ).into_bytes(); + let path = table_path.path(); + let path = format!("{path}_delta_log/00000000000000000000.json"); + println!("putting to path: {}", path); + store.put(&Path::from(path), data.into()).await?; + Ok(Table::new(table_path)) +} + +// FIXME delete/unify from default/parquet.rs +fn get_metadata_schema() -> Arc { + let path_field = Field::new("path", arrow_schema::DataType::Utf8, false); + let size_field = Field::new("size", arrow_schema::DataType::Int64, false); + let partition_field = Field::new( + "partitionValues", + arrow_schema::DataType::Map( + Arc::new(Field::new( + "entries", + arrow_schema::DataType::Struct( + vec![ + Field::new("keys", arrow_schema::DataType::Utf8, false), + Field::new("values", arrow_schema::DataType::Utf8, true), + ] + .into(), + ), + false, + )), + false, + ), + false, + ); + let data_change_field = Field::new("dataChange", arrow_schema::DataType::Boolean, false); + let modification_time_field = + Field::new("modificationTime", arrow_schema::DataType::Int64, false); + + Arc::new(arrow_schema::Schema::new(vec![Field::new( + "add", + arrow_schema::DataType::Struct( + vec![ + path_field.clone(), + size_field.clone(), + partition_field.clone(), + data_change_field.clone(), + modification_time_field.clone(), + ] + .into(), + ), + false, + )])) +} + +#[tokio::test] +async fn test_commit_info() -> Result<(), Box> { + // setup tracing + let _ = tracing_subscriber::fmt::try_init(); + // setup in-memory object store and default engine + let (store, engine, table_location) = setup("test_table"); + // create a simple table: one int column named 'number' + let schema = Arc::new(StructType::new(vec![StructField::new( + "number", + DataType::INTEGER, + true, + )])); + let schema_string = r#"{\"type\":\"struct\",\"fields\":[{\"name\":\"number\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}}]}"#; + let table = create_table( + store.clone(), + table_location, + schema.clone(), + schema_string, + "", + ) + .await?; + // println!( + // "{:?}", + // store + // .get(&Path::from( + // "/test_table/_delta_log/00000000000000000000.json" + // )) + // .await + // ); + // append an arrow record batch (vec of record batches??) + let mut txn = table.new_transaction(&engine)?; + txn.commit(&engine)?; + let commit1 = store + .get(&Path::from( + "/test_table/_delta_log/00000000000000000001.json", + )) + .await?; + println!("commit1: {}", String::from_utf8(commit1.bytes().await?.to_vec())?); + Ok(()) +} From b16491cd449dc7cb0f48712f6e3f32c824fa9f38 Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Tue, 1 Oct 2024 22:36:19 -0700 Subject: [PATCH 02/66] commit info working --- kernel/src/transaction.rs | 57 ++++++++++++++++++++++++---- kernel/tests/write.rs | 79 ++++++++++++--------------------------- 2 files changed, 74 insertions(+), 62 deletions(-) diff --git a/kernel/src/transaction.rs b/kernel/src/transaction.rs index 965e8ca9a..f5cac58db 100644 --- a/kernel/src/transaction.rs +++ b/kernel/src/transaction.rs @@ -1,6 +1,7 @@ use std::sync::{Arc, LazyLock}; use crate::actions::get_log_schema; +use crate::expressions::Scalar; use crate::schema::{Schema, StructType}; use crate::snapshot::Snapshot; use crate::{DataType, Expression}; @@ -35,22 +36,61 @@ impl Transaction { } pub fn commit(self, engine: &dyn Engine) -> DeltaResult { - // lazy lock for the expression - static COMMIT_INFO_EXPR: LazyLock = - LazyLock::new(|| Expression::column("commitInfo")); - // step one: construct the iterator of actions we want to commit let action_schema = get_log_schema(); let actions = self.commit_info.into_iter().map(|commit_info| { + // expression to select all the columns + let commit_info_expr = Expression::Struct(vec![ + //Expression::Literal(Scalar::Null( + // action_schema + // .project(&[crate::actions::ADD_NAME]) + // .unwrap() + // .into(), + //)), + //Expression::Literal(Scalar::Null( + // action_schema + // .project(&[crate::actions::REMOVE_NAME]) + // .unwrap() + // .into(), + //)), + //Expression::Literal(Scalar::Null( + // action_schema + // .project(&[crate::actions::METADATA_NAME]) + // .unwrap() + // .into(), + //)), + //Expression::Literal(Scalar::Null( + // action_schema + // .project(&[crate::actions::PROTOCOL_NAME]) + // .unwrap() + // .into(), + //)), + //Expression::Literal(Scalar::Null( + // action_schema + // .project(&[crate::actions::TRANSACTION_NAME]) + // .unwrap() + // .into(), + //)), + Expression::Struct( + commit_info + .schema + .fields() + .map(|f| Expression::column(f.name())) + .collect(), + ), + ]); + // commit info has arbitrary schema ex: {engineInfo: string, operation: string} // we want to bundle it up and put it in the commit_info field of the actions. let commit_info_evaluator = engine.get_expression_handler().get_evaluator( commit_info.schema.into(), - COMMIT_INFO_EXPR.clone(), // TODO remove clone? + commit_info_expr, >::into(action_schema.clone()), ); - commit_info_evaluator.evaluate(commit_info.data.as_ref()).unwrap() + commit_info_evaluator + .evaluate(commit_info.data.as_ref()) + .unwrap() }); // step two: figure out the commit version and path to write @@ -73,7 +113,10 @@ impl Transaction { /// first action in the commit. Note it is required in order to commit. If the engine does not /// require any commit info, pass an empty `EngineData`. pub fn commit_info(&mut self, commit_info: Box, schema: Schema) { - self.commit_info = Some(CommitInfoData { data: commit_info, schema }); + self.commit_info = Some(CommitInfoData { + data: commit_info, + schema, + }); } } diff --git a/kernel/tests/write.rs b/kernel/tests/write.rs index 81a731224..9896a5936 100644 --- a/kernel/tests/write.rs +++ b/kernel/tests/write.rs @@ -1,8 +1,7 @@ use std::sync::Arc; -use arrow::array::{ArrayRef, BooleanArray, Int64Array, StringArray}; +use arrow::array::StringArray; use arrow::record_batch::RecordBatch; -use arrow_schema::Field; use object_store::memory::InMemory; use object_store::path::Path; use object_store::ObjectStore; @@ -12,7 +11,7 @@ use delta_kernel::engine::arrow_data::ArrowEngineData; use delta_kernel::engine::default::executor::tokio::TokioBackgroundExecutor; use delta_kernel::engine::default::DefaultEngine; use delta_kernel::schema::{DataType, SchemaRef, StructField, StructType}; -use delta_kernel::{DeltaResult, Engine, EngineData, Table}; +use delta_kernel::Table; // fixme use in macro below // const PROTOCOL_METADATA_TEMPLATE: &'static str = r#"{{"protocol":{{"minReaderVersion":3,"minWriterVersion":7,"readerFeatures":[],"writerFeatures":[]}}}} @@ -64,48 +63,6 @@ async fn create_table( Ok(Table::new(table_path)) } -// FIXME delete/unify from default/parquet.rs -fn get_metadata_schema() -> Arc { - let path_field = Field::new("path", arrow_schema::DataType::Utf8, false); - let size_field = Field::new("size", arrow_schema::DataType::Int64, false); - let partition_field = Field::new( - "partitionValues", - arrow_schema::DataType::Map( - Arc::new(Field::new( - "entries", - arrow_schema::DataType::Struct( - vec![ - Field::new("keys", arrow_schema::DataType::Utf8, false), - Field::new("values", arrow_schema::DataType::Utf8, true), - ] - .into(), - ), - false, - )), - false, - ), - false, - ); - let data_change_field = Field::new("dataChange", arrow_schema::DataType::Boolean, false); - let modification_time_field = - Field::new("modificationTime", arrow_schema::DataType::Int64, false); - - Arc::new(arrow_schema::Schema::new(vec![Field::new( - "add", - arrow_schema::DataType::Struct( - vec![ - path_field.clone(), - size_field.clone(), - partition_field.clone(), - data_change_field.clone(), - modification_time_field.clone(), - ] - .into(), - ), - false, - )])) -} - #[tokio::test] async fn test_commit_info() -> Result<(), Box> { // setup tracing @@ -127,22 +84,34 @@ async fn test_commit_info() -> Result<(), Box> { "", ) .await?; - // println!( - // "{:?}", - // store - // .get(&Path::from( - // "/test_table/_delta_log/00000000000000000000.json" - // )) - // .await - // ); - // append an arrow record batch (vec of record batches??) let mut txn = table.new_transaction(&engine)?; + + use arrow_schema::Schema as ArrowSchema; + use arrow_schema::{DataType as ArrowDataType, Field}; + + // add commit info of the form {engineInfo: "default engine"} + let commit_info_schema = Arc::new(ArrowSchema::new(vec![Field::new( + "engineInfo", + ArrowDataType::Utf8, + true, + )])); + let commit_info_batch = RecordBatch::try_new( + commit_info_schema.clone(), + vec![Arc::new(StringArray::from(vec!["default engine"]))], + )?; + txn.commit_info( + Box::new(ArrowEngineData::new(commit_info_batch)), + commit_info_schema.try_into()?, + ); txn.commit(&engine)?; let commit1 = store .get(&Path::from( "/test_table/_delta_log/00000000000000000001.json", )) .await?; - println!("commit1: {}", String::from_utf8(commit1.bytes().await?.to_vec())?); + assert_eq!( + String::from_utf8(commit1.bytes().await?.to_vec())?, + "{\"commitInfo\":{\"kernelVersion\":\"default engine\"}}\n" + ); Ok(()) } From 432a33973307c4b7f96954afe14d6aba327a5084 Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Tue, 1 Oct 2024 22:47:02 -0700 Subject: [PATCH 03/66] fix commit info --- kernel/src/transaction.rs | 67 +++++++++++++++++++++------------------ 1 file changed, 36 insertions(+), 31 deletions(-) diff --git a/kernel/src/transaction.rs b/kernel/src/transaction.rs index f5cac58db..4e259540e 100644 --- a/kernel/src/transaction.rs +++ b/kernel/src/transaction.rs @@ -1,4 +1,4 @@ -use std::sync::{Arc, LazyLock}; +use std::sync::Arc; use crate::actions::get_log_schema; use crate::expressions::Scalar; @@ -42,36 +42,41 @@ impl Transaction { let actions = self.commit_info.into_iter().map(|commit_info| { // expression to select all the columns let commit_info_expr = Expression::Struct(vec![ - //Expression::Literal(Scalar::Null( - // action_schema - // .project(&[crate::actions::ADD_NAME]) - // .unwrap() - // .into(), - //)), - //Expression::Literal(Scalar::Null( - // action_schema - // .project(&[crate::actions::REMOVE_NAME]) - // .unwrap() - // .into(), - //)), - //Expression::Literal(Scalar::Null( - // action_schema - // .project(&[crate::actions::METADATA_NAME]) - // .unwrap() - // .into(), - //)), - //Expression::Literal(Scalar::Null( - // action_schema - // .project(&[crate::actions::PROTOCOL_NAME]) - // .unwrap() - // .into(), - //)), - //Expression::Literal(Scalar::Null( - // action_schema - // .project(&[crate::actions::TRANSACTION_NAME]) - // .unwrap() - // .into(), - //)), + Expression::Literal(Scalar::Null( + action_schema + .field(crate::actions::ADD_NAME) + .unwrap() + .data_type() + .clone(), + )), + Expression::Literal(Scalar::Null( + action_schema + .field(crate::actions::REMOVE_NAME) + .unwrap() + .data_type() + .clone(), + )), + Expression::Literal(Scalar::Null( + action_schema + .field(crate::actions::METADATA_NAME) + .unwrap() + .data_type() + .clone(), + )), + Expression::Literal(Scalar::Null( + action_schema + .field(crate::actions::PROTOCOL_NAME) + .unwrap() + .data_type() + .clone(), + )), + Expression::Literal(Scalar::Null( + action_schema + .field(crate::actions::TRANSACTION_NAME) + .unwrap() + .data_type() + .clone(), + )), Expression::Struct( commit_info .schema From 93fab4df705602ac49a26c4208a91146f7249ba4 Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Thu, 3 Oct 2024 16:21:35 -0700 Subject: [PATCH 04/66] well that was a mess --- kernel/src/actions/mod.rs | 1 - kernel/src/transaction.rs | 87 +++++++++++++++++++++++++++++++-------- kernel/tests/write.rs | 2 +- 3 files changed, 71 insertions(+), 19 deletions(-) diff --git a/kernel/src/actions/mod.rs b/kernel/src/actions/mod.rs index a186c5402..b5b3f3bc4 100644 --- a/kernel/src/actions/mod.rs +++ b/kernel/src/actions/mod.rs @@ -140,7 +140,6 @@ impl Protocol { #[derive(Debug, Clone, PartialEq, Eq, Schema)] pub struct CommitInfo { pub kernel_version: Option, - // pub engine_info: HashMap } #[derive(Debug, Clone, PartialEq, Eq, Schema)] diff --git a/kernel/src/transaction.rs b/kernel/src/transaction.rs index 4e259540e..c8732b4ed 100644 --- a/kernel/src/transaction.rs +++ b/kernel/src/transaction.rs @@ -36,62 +36,115 @@ impl Transaction { } pub fn commit(self, engine: &dyn Engine) -> DeltaResult { + use crate::actions::{ + ADD_NAME, COMMIT_INFO_NAME, METADATA_NAME, PROTOCOL_NAME, REMOVE_NAME, TRANSACTION_NAME, + }; + // step one: construct the iterator of actions we want to commit let action_schema = get_log_schema(); let actions = self.commit_info.into_iter().map(|commit_info| { // expression to select all the columns + let mut commit_info_expr = vec![Expression::literal("v0.3.1")]; + commit_info_expr.extend( + commit_info + .schema + .fields() + .map(|f| Expression::column(f.name())) + .collect::>(), + ); let commit_info_expr = Expression::Struct(vec![ Expression::Literal(Scalar::Null( - action_schema - .field(crate::actions::ADD_NAME) - .unwrap() - .data_type() - .clone(), + action_schema.field(ADD_NAME).unwrap().data_type().clone(), )), Expression::Literal(Scalar::Null( action_schema - .field(crate::actions::REMOVE_NAME) + .field(REMOVE_NAME) .unwrap() .data_type() .clone(), )), Expression::Literal(Scalar::Null( action_schema - .field(crate::actions::METADATA_NAME) + .field(METADATA_NAME) .unwrap() .data_type() .clone(), )), Expression::Literal(Scalar::Null( action_schema - .field(crate::actions::PROTOCOL_NAME) + .field(PROTOCOL_NAME) .unwrap() .data_type() .clone(), )), Expression::Literal(Scalar::Null( action_schema - .field(crate::actions::TRANSACTION_NAME) + .field(TRANSACTION_NAME) .unwrap() .data_type() .clone(), )), - Expression::Struct( - commit_info - .schema - .fields() - .map(|f| Expression::column(f.name())) - .collect(), - ), + Expression::Struct(commit_info_expr), ]); + // add the commit info fields to the action schema. + // e.g. if engine's commit info is {engineInfo: string, operation: string} + // then the 'commit_info' field in the actions will be: + // {kernelVersion: string, engineInfo: string, operation: string} + // let action_fields = action_schema + // .project_as_struct(&[ + // ADD_NAME, + // REMOVE_NAME, + // METADATA_NAME, + // PROTOCOL_NAME, + // TRANSACTION_NAME, + // ]) + // .unwrap() + // .fields(); + // let kernel_commit_info_fields = action_schema + // .project_as_struct(&[COMMIT_INFO_NAME]) + // .unwrap() + // .fields(); + // let engine_commit_info_fields = commit_info.schema.fields(); + // let commit_info_fields = kernel_commit_info_fields.chain(engine_commit_info_fields); + // let action_schema = StructType::new( + // std::iter::once(commit_info_fields) + // .chain(action_fields) + // .collect::Vec<_>() + // ); + + let mut action_fields = action_schema.fields().collect::>(); + let commit_info_field = action_fields.pop().unwrap(); + let mut commit_info_fields = + if let DataType::Struct(commit_info_schema) = commit_info_field.data_type() { + commit_info_schema.fields().collect::>() + } else { + unreachable!() + }; + commit_info_fields.extend(commit_info.schema.fields()); + let commit_info_schema = + StructType::new(commit_info_fields.into_iter().map(|f| f.clone()).collect()); + let mut action_fields = action_fields + .into_iter() + .map(|f| f.clone()) + .collect::>(); + action_fields.push(crate::schema::StructField::new( + COMMIT_INFO_NAME, + commit_info_schema, + true, + )); + let action_schema = StructType::new(action_fields); + + println!("commit_info schema: {:#?}", commit_info.schema); + println!("action_schema: {:#?}", action_schema); + // commit info has arbitrary schema ex: {engineInfo: string, operation: string} // we want to bundle it up and put it in the commit_info field of the actions. let commit_info_evaluator = engine.get_expression_handler().get_evaluator( commit_info.schema.into(), commit_info_expr, - >::into(action_schema.clone()), + action_schema.into(), ); commit_info_evaluator .evaluate(commit_info.data.as_ref()) diff --git a/kernel/tests/write.rs b/kernel/tests/write.rs index 9896a5936..2ac241538 100644 --- a/kernel/tests/write.rs +++ b/kernel/tests/write.rs @@ -111,7 +111,7 @@ async fn test_commit_info() -> Result<(), Box> { .await?; assert_eq!( String::from_utf8(commit1.bytes().await?.to_vec())?, - "{\"commitInfo\":{\"kernelVersion\":\"default engine\"}}\n" + "{\"commitInfo\":{\"kernelVersion\":\"v0.3.1\",\"engineInfo\":\"default engine\"}}\n" ); Ok(()) } From 64d7eaf7e0b0c677637ff5f3bb863cf25e1a1f76 Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Thu, 3 Oct 2024 21:24:05 -0700 Subject: [PATCH 05/66] better --- kernel/src/engine/default/filesystem.rs | 1 - kernel/src/engine/default/json.rs | 13 +- kernel/src/engine/default/mod.rs | 2 - kernel/src/table.rs | 2 +- kernel/src/transaction.rs | 226 ++++++++++++------------ 5 files changed, 111 insertions(+), 133 deletions(-) diff --git a/kernel/src/engine/default/filesystem.rs b/kernel/src/engine/default/filesystem.rs index 669a0c8ef..b9036d04e 100644 --- a/kernel/src/engine/default/filesystem.rs +++ b/kernel/src/engine/default/filesystem.rs @@ -56,7 +56,6 @@ impl FileSystemClient for ObjectStoreFileSystemClient { match meta { Ok(meta) => { let mut location = url.clone(); - println!("listed location: {:?}", meta.location); location.set_path(&format!("/{}", meta.location.as_ref())); sender .send(Ok(FileMeta { diff --git a/kernel/src/engine/default/json.rs b/kernel/src/engine/default/json.rs index 1b27b2f53..1914f315f 100644 --- a/kernel/src/engine/default/json.rs +++ b/kernel/src/engine/default/json.rs @@ -96,7 +96,6 @@ impl JsonHandler for DefaultJsonHandler { data: Box> + Send + 'a>, _overwrite: bool, ) -> DeltaResult<()> { - // Initialize schema and batches let mut schema: Option = None; let mut batches: Vec = Vec::new(); @@ -108,29 +107,19 @@ impl JsonHandler for DefaultJsonHandler { schema = Some(record_batch.schema()); } - println!("[Engine put_json] schema: {:#?}", record_batch.schema()); batches.push(record_batch.clone()); } - for batch in batches.iter() { - println!("==============================================================="); - println!("[Engine put_json] batch: {:#?}", batch); - println!("[Engine put_json] schema: {:#?}", batch.schema()); - println!("==============================================================="); - } - // collect all batches let batches: Vec<&RecordBatch> = batches.iter().collect(); - // Write the concatenated batch to JSON + // write the batches to JSON let mut writer = arrow_json::LineDelimitedWriter::new(Vec::new()); writer.write_batches(&batches)?; writer.finish()?; let buffer = writer.into_inner(); - println!("[Engine put_json] commit path: {:?}", path.path()); - // Put if absent futures::executor::block_on(async { self.store diff --git a/kernel/src/engine/default/mod.rs b/kernel/src/engine/default/mod.rs index 5abb0b9c3..12c89ca2e 100644 --- a/kernel/src/engine/default/mod.rs +++ b/kernel/src/engine/default/mod.rs @@ -53,7 +53,6 @@ impl DefaultEngine { { // table root is the path of the table in the ObjectStore let (store, table_root) = parse_url_opts(table_root, options)?; - println!("DEFAULT ENGINE INIT try_new table root: {:?}", table_root); let store = Arc::new(store); Ok(Self { file_system: Arc::new(ObjectStoreFileSystemClient::new( @@ -79,7 +78,6 @@ impl DefaultEngine { /// - `table_root_path`: The root path of the table within storage. /// - `task_executor`: Used to spawn async IO tasks. See [executor::TaskExecutor]. pub fn new(store: Arc, table_root_path: Path, task_executor: Arc) -> Self { - println!("DEFAULT ENGINE INIT new table root: {:?}", table_root_path); Self { file_system: Arc::new(ObjectStoreFileSystemClient::new( store.clone(), diff --git a/kernel/src/table.rs b/kernel/src/table.rs index 087214851..8b1cf1d20 100644 --- a/kernel/src/table.rs +++ b/kernel/src/table.rs @@ -78,7 +78,7 @@ impl Table { Snapshot::try_new(self.location.clone(), engine, version) } - /// Create a new write transaction builder for the table. + /// Create a new write transaction for this table. pub fn new_transaction(&self, engine: &dyn Engine) -> DeltaResult { let latest_snapshot = Snapshot::try_new(self.location.clone(), engine, None)?; Ok(Transaction::new(latest_snapshot)) diff --git a/kernel/src/transaction.rs b/kernel/src/transaction.rs index c8732b4ed..8e184a69a 100644 --- a/kernel/src/transaction.rs +++ b/kernel/src/transaction.rs @@ -2,17 +2,17 @@ use std::sync::Arc; use crate::actions::get_log_schema; use crate::expressions::Scalar; -use crate::schema::{Schema, StructType}; +use crate::schema::{Schema, SchemaRef, StructType}; use crate::snapshot::Snapshot; use crate::{DataType, Expression}; use crate::{DeltaResult, Engine, EngineData}; pub struct Transaction { read_snapshot: Arc, - commit_info: Option, + commit_info: Option, } -pub struct CommitInfoData { +struct EngineCommitInfo { data: Box, schema: Schema, } @@ -36,120 +36,11 @@ impl Transaction { } pub fn commit(self, engine: &dyn Engine) -> DeltaResult { - use crate::actions::{ - ADD_NAME, COMMIT_INFO_NAME, METADATA_NAME, PROTOCOL_NAME, REMOVE_NAME, TRANSACTION_NAME, - }; - // step one: construct the iterator of actions we want to commit - let action_schema = get_log_schema(); - - let actions = self.commit_info.into_iter().map(|commit_info| { - // expression to select all the columns - let mut commit_info_expr = vec![Expression::literal("v0.3.1")]; - commit_info_expr.extend( - commit_info - .schema - .fields() - .map(|f| Expression::column(f.name())) - .collect::>(), - ); - let commit_info_expr = Expression::Struct(vec![ - Expression::Literal(Scalar::Null( - action_schema.field(ADD_NAME).unwrap().data_type().clone(), - )), - Expression::Literal(Scalar::Null( - action_schema - .field(REMOVE_NAME) - .unwrap() - .data_type() - .clone(), - )), - Expression::Literal(Scalar::Null( - action_schema - .field(METADATA_NAME) - .unwrap() - .data_type() - .clone(), - )), - Expression::Literal(Scalar::Null( - action_schema - .field(PROTOCOL_NAME) - .unwrap() - .data_type() - .clone(), - )), - Expression::Literal(Scalar::Null( - action_schema - .field(TRANSACTION_NAME) - .unwrap() - .data_type() - .clone(), - )), - Expression::Struct(commit_info_expr), - ]); - - // add the commit info fields to the action schema. - // e.g. if engine's commit info is {engineInfo: string, operation: string} - // then the 'commit_info' field in the actions will be: - // {kernelVersion: string, engineInfo: string, operation: string} - // let action_fields = action_schema - // .project_as_struct(&[ - // ADD_NAME, - // REMOVE_NAME, - // METADATA_NAME, - // PROTOCOL_NAME, - // TRANSACTION_NAME, - // ]) - // .unwrap() - // .fields(); - // let kernel_commit_info_fields = action_schema - // .project_as_struct(&[COMMIT_INFO_NAME]) - // .unwrap() - // .fields(); - // let engine_commit_info_fields = commit_info.schema.fields(); - // let commit_info_fields = kernel_commit_info_fields.chain(engine_commit_info_fields); - // let action_schema = StructType::new( - // std::iter::once(commit_info_fields) - // .chain(action_fields) - // .collect::Vec<_>() - // ); - - let mut action_fields = action_schema.fields().collect::>(); - let commit_info_field = action_fields.pop().unwrap(); - let mut commit_info_fields = - if let DataType::Struct(commit_info_schema) = commit_info_field.data_type() { - commit_info_schema.fields().collect::>() - } else { - unreachable!() - }; - commit_info_fields.extend(commit_info.schema.fields()); - let commit_info_schema = - StructType::new(commit_info_fields.into_iter().map(|f| f.clone()).collect()); - let mut action_fields = action_fields - .into_iter() - .map(|f| f.clone()) - .collect::>(); - action_fields.push(crate::schema::StructField::new( - COMMIT_INFO_NAME, - commit_info_schema, - true, - )); - let action_schema = StructType::new(action_fields); - - println!("commit_info schema: {:#?}", commit_info.schema); - println!("action_schema: {:#?}", action_schema); - - // commit info has arbitrary schema ex: {engineInfo: string, operation: string} - // we want to bundle it up and put it in the commit_info field of the actions. - let commit_info_evaluator = engine.get_expression_handler().get_evaluator( - commit_info.schema.into(), - commit_info_expr, - action_schema.into(), - ); - commit_info_evaluator - .evaluate(commit_info.data.as_ref()) - .unwrap() - }); + // + // TODO: enforce single row commit info + // TODO: for now we always require commit info + let (actions, _actions_schema) = generate_commit_info(engine, self.commit_info)?; // step two: figure out the commit version and path to write let commit_version = &self.read_snapshot.version() + 1; @@ -171,7 +62,7 @@ impl Transaction { /// first action in the commit. Note it is required in order to commit. If the engine does not /// require any commit info, pass an empty `EngineData`. pub fn commit_info(&mut self, commit_info: Box, schema: Schema) { - self.commit_info = Some(CommitInfoData { + self.commit_info = Some(EngineCommitInfo { data: commit_info, schema, }); @@ -181,3 +72,104 @@ impl Transaction { pub enum CommitResult { Committed(crate::Version), } + +fn generate_commit_info<'a>( + engine: &'a dyn Engine, + engine_commit_info: Option, +) -> DeltaResult<( + Box> + Send + 'a>, + SchemaRef, +)> { + use crate::actions::{ + ADD_NAME, COMMIT_INFO_NAME, METADATA_NAME, PROTOCOL_NAME, REMOVE_NAME, TRANSACTION_NAME, + }; + + let action_schema = + Arc::new(engine_commit_info + .as_ref() + .map_or(get_log_schema().clone(), |commit_info| { + let mut action_fields = get_log_schema().fields().collect::>(); + let commit_info_field = action_fields.pop().unwrap(); + let mut commit_info_fields = + if let DataType::Struct(commit_info_schema) = commit_info_field.data_type() { + commit_info_schema.fields().collect::>() + } else { + unreachable!() + }; + commit_info_fields.extend(commit_info.schema.fields()); + let commit_info_schema = + StructType::new(commit_info_fields.into_iter().map(|f| f.clone()).collect()); + let mut action_fields = action_fields + .into_iter() + .map(|f| f.clone()) + .collect::>(); + action_fields.push(crate::schema::StructField::new( + COMMIT_INFO_NAME, + commit_info_schema, + true, + )); + StructType::new(action_fields) + })); + + let action_schema_ref = Arc::clone(&action_schema); + let actions = engine_commit_info.into_iter().map(move |commit_info| { + // TODO RENAME + let engine_commit_info_data = commit_info.data; + let engine_commit_info_schema = commit_info.schema; + // expression to select all the columns + let mut commit_info_expr = vec![Expression::literal("v0.3.1")]; + commit_info_expr.extend( + engine_commit_info_schema + .fields() + .map(|f| Expression::column(f.name())) + .collect::>(), + ); + let commit_info_expr = Expression::Struct(vec![ + Expression::Literal(Scalar::Null( + action_schema_ref.field(ADD_NAME).unwrap().data_type().clone(), + )), + Expression::Literal(Scalar::Null( + action_schema_ref + .field(REMOVE_NAME) + .unwrap() + .data_type() + .clone(), + )), + Expression::Literal(Scalar::Null( + action_schema_ref + .field(METADATA_NAME) + .unwrap() + .data_type() + .clone(), + )), + Expression::Literal(Scalar::Null( + action_schema_ref + .field(PROTOCOL_NAME) + .unwrap() + .data_type() + .clone(), + )), + Expression::Literal(Scalar::Null( + action_schema_ref + .field(TRANSACTION_NAME) + .unwrap() + .data_type() + .clone(), + )), + Expression::Struct(commit_info_expr), + ]); + + // commit info has arbitrary schema ex: {engineInfo: string, operation: string} + // we want to bundle it up and put it in the commit_info field of the actions. + let commit_info_evaluator = engine.get_expression_handler().get_evaluator( + engine_commit_info_schema.into(), + commit_info_expr, + action_schema_ref.clone().into(), + ); + + commit_info_evaluator + .evaluate(engine_commit_info_data.as_ref()) + .unwrap() + }); + Ok((Box::new(actions), action_schema)) +} From 05e9488db03a83cf798ff3b015d8a6621c4f280e Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Mon, 7 Oct 2024 16:20:27 -0700 Subject: [PATCH 06/66] cleanup --- kernel/src/transaction.rs | 157 ++++++++++++++++++++------------------ 1 file changed, 83 insertions(+), 74 deletions(-) diff --git a/kernel/src/transaction.rs b/kernel/src/transaction.rs index 8e184a69a..6d58e0d31 100644 --- a/kernel/src/transaction.rs +++ b/kernel/src/transaction.rs @@ -7,14 +7,20 @@ use crate::snapshot::Snapshot; use crate::{DataType, Expression}; use crate::{DeltaResult, Engine, EngineData}; +const KERNEL_VERSION: &str = env!("CARGO_PKG_VERSION"); + +/// A transaction represents an in-progress write to a table. pub struct Transaction { read_snapshot: Arc, commit_info: Option, } +// Since the engine can include any commit info it likes, we unify the data/schema pair as a single +// struct with Arc semantics. +#[derive(Clone)] struct EngineCommitInfo { - data: Box, - schema: Schema, + data: Arc, + schema: SchemaRef, } impl std::fmt::Debug for Transaction { @@ -28,6 +34,12 @@ impl std::fmt::Debug for Transaction { } impl Transaction { + /// Create a new transaction from a snapshot. The snapshot will be used to read the current + /// state of the table (e.g. to read the current version). + /// + /// Instead of using this API, the more typical API is + /// [Table::new_transaction](crate::table::Table::new_transaction) to create a transaction from + /// a table automatically backed by the latest snapshot. pub fn new(snapshot: impl Into>) -> Self { Transaction { read_snapshot: snapshot.into(), @@ -35,14 +47,13 @@ impl Transaction { } } + /// Consume the transaction and commit the in-progress write to the table. pub fn commit(self, engine: &dyn Engine) -> DeltaResult { // step one: construct the iterator of actions we want to commit - // - // TODO: enforce single row commit info - // TODO: for now we always require commit info - let (actions, _actions_schema) = generate_commit_info(engine, self.commit_info)?; + // note: only support commit_info right now. + let (actions, _actions_schema) = generate_commit_info(engine, self.commit_info.clone())?; - // step two: figure out the commit version and path to write + // step two: set new commit version (current_version + 1) and path to write let commit_version = &self.read_snapshot.version() + 1; let commit_file_name = format!("{:020}", commit_version) + ".json"; let commit_path = &self @@ -53,9 +64,14 @@ impl Transaction { // step three: commit the actions as a json file in the log let json_handler = engine.get_json_handler(); - - json_handler.write_json_file(commit_path, Box::new(actions), false)?; - Ok(CommitResult::Committed(commit_version)) + match json_handler.write_json_file(commit_path, Box::new(actions), false) { + Ok(()) => Ok(CommitResult::Committed(commit_version)), + Err(crate::error::Error::ObjectStore(object_store::Error::AlreadyExists { + path: _, + source: _, + })) => Ok(CommitResult::Conflict(self, commit_version)), + Err(e) => Err(e), + } } /// Add commit info to the transaction. This is commit-wide metadata that is written as the @@ -63,16 +79,25 @@ impl Transaction { /// require any commit info, pass an empty `EngineData`. pub fn commit_info(&mut self, commit_info: Box, schema: Schema) { self.commit_info = Some(EngineCommitInfo { - data: commit_info, - schema, + data: commit_info.into(), + schema: schema.into(), }); } } +/// Result after committing a transaction. If 'committed', the version is the new version written +/// to the log. If 'conflict', the transaction is returned so the caller can resolve the conflict +/// (along with the version which conflicted). pub enum CommitResult { + /// The transaction was successfully committed at the version. Committed(crate::Version), + /// The transaction conflicted with an existing version (at the version given). + Conflict(Transaction, crate::Version), } +// given the engine's commit info (data and schema as [EngineCommitInfo]) we want to create both: +// (1) the commitInfo action to commit (and append more actions to) and +// (2) the schema for the actions to commit (this is determined by the engine's commit info schema) fn generate_commit_info<'a>( engine: &'a dyn Engine, engine_commit_info: Option, @@ -84,80 +109,64 @@ fn generate_commit_info<'a>( ADD_NAME, COMMIT_INFO_NAME, METADATA_NAME, PROTOCOL_NAME, REMOVE_NAME, TRANSACTION_NAME, }; - let action_schema = - Arc::new(engine_commit_info - .as_ref() - .map_or(get_log_schema().clone(), |commit_info| { - let mut action_fields = get_log_schema().fields().collect::>(); - let commit_info_field = action_fields.pop().unwrap(); - let mut commit_info_fields = - if let DataType::Struct(commit_info_schema) = commit_info_field.data_type() { - commit_info_schema.fields().collect::>() - } else { - unreachable!() - }; - commit_info_fields.extend(commit_info.schema.fields()); - let commit_info_schema = - StructType::new(commit_info_fields.into_iter().map(|f| f.clone()).collect()); - let mut action_fields = action_fields - .into_iter() - .map(|f| f.clone()) - .collect::>(); - action_fields.push(crate::schema::StructField::new( - COMMIT_INFO_NAME, - commit_info_schema, - true, - )); - StructType::new(action_fields) - })); + // TODO: enforce single row commit info + // TODO: for now we always require commit info + let action_schema = Arc::new(engine_commit_info.as_ref().map_or( + get_log_schema().clone(), + |commit_info| { + let mut action_fields = get_log_schema().fields().collect::>(); + let commit_info_field = action_fields.pop().unwrap(); + let mut commit_info_fields = + if let DataType::Struct(commit_info_schema) = commit_info_field.data_type() { + commit_info_schema.fields().collect::>() + } else { + unreachable!() + }; + commit_info_fields.extend(commit_info.schema.fields()); + let commit_info_schema = + StructType::new(commit_info_fields.into_iter().map(|f| f.clone()).collect()); + let mut action_fields = action_fields + .into_iter() + .map(|f| f.clone()) + .collect::>(); + action_fields.push(crate::schema::StructField::new( + COMMIT_INFO_NAME, + commit_info_schema, + true, + )); + StructType::new(action_fields) + }, + )); let action_schema_ref = Arc::clone(&action_schema); let actions = engine_commit_info.into_iter().map(move |commit_info| { - // TODO RENAME let engine_commit_info_data = commit_info.data; let engine_commit_info_schema = commit_info.schema; - // expression to select all the columns - let mut commit_info_expr = vec![Expression::literal("v0.3.1")]; + let mut commit_info_expr = vec![Expression::literal(format!("v{}", KERNEL_VERSION))]; commit_info_expr.extend( engine_commit_info_schema .fields() .map(|f| Expression::column(f.name())) .collect::>(), ); - let commit_info_expr = Expression::Struct(vec![ - Expression::Literal(Scalar::Null( - action_schema_ref.field(ADD_NAME).unwrap().data_type().clone(), - )), - Expression::Literal(Scalar::Null( - action_schema_ref - .field(REMOVE_NAME) - .unwrap() - .data_type() - .clone(), - )), - Expression::Literal(Scalar::Null( - action_schema_ref - .field(METADATA_NAME) - .unwrap() - .data_type() - .clone(), - )), - Expression::Literal(Scalar::Null( - action_schema_ref - .field(PROTOCOL_NAME) - .unwrap() - .data_type() - .clone(), - )), + // generate expression with null for all the fields except the commit_info field, and + // append the commit_info to the end. + let commit_info_expr_fields = [ + ADD_NAME, + REMOVE_NAME, + METADATA_NAME, + PROTOCOL_NAME, + TRANSACTION_NAME, + ] + .iter() + .map(|name| { Expression::Literal(Scalar::Null( - action_schema_ref - .field(TRANSACTION_NAME) - .unwrap() - .data_type() - .clone(), - )), - Expression::Struct(commit_info_expr), - ]); + action_schema_ref.field(name).unwrap().data_type().clone(), + )) + }) + .chain(std::iter::once(Expression::Struct(commit_info_expr))) + .collect::>(); + let commit_info_expr = Expression::Struct(commit_info_expr_fields); // commit info has arbitrary schema ex: {engineInfo: string, operation: string} // we want to bundle it up and put it in the commit_info field of the actions. From 2282cd8012f77939a5c8e0067a2b28ea9e250b2a Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Mon, 7 Oct 2024 16:21:28 -0700 Subject: [PATCH 07/66] fmt --- kernel/src/actions/mod.rs | 1 - kernel/src/actions/transaction.rs | 2 +- kernel/src/engine/default/mod.rs | 6 +++++- 3 files changed, 6 insertions(+), 3 deletions(-) diff --git a/kernel/src/actions/mod.rs b/kernel/src/actions/mod.rs index b5b3f3bc4..093e42e09 100644 --- a/kernel/src/actions/mod.rs +++ b/kernel/src/actions/mod.rs @@ -136,7 +136,6 @@ impl Protocol { } } - #[derive(Debug, Clone, PartialEq, Eq, Schema)] pub struct CommitInfo { pub kernel_version: Option, diff --git a/kernel/src/actions/transaction.rs b/kernel/src/actions/transaction.rs index 7555d1e4d..1242f754e 100644 --- a/kernel/src/actions/transaction.rs +++ b/kernel/src/actions/transaction.rs @@ -1,7 +1,7 @@ use std::sync::Arc; use crate::actions::visitors::TransactionVisitor; -use crate::actions::{Transaction, get_log_schema, TRANSACTION_NAME}; +use crate::actions::{get_log_schema, Transaction, TRANSACTION_NAME}; use crate::snapshot::Snapshot; use crate::{DeltaResult, Engine}; diff --git a/kernel/src/engine/default/mod.rs b/kernel/src/engine/default/mod.rs index 12c89ca2e..e169674e1 100644 --- a/kernel/src/engine/default/mod.rs +++ b/kernel/src/engine/default/mod.rs @@ -45,7 +45,11 @@ impl DefaultEngine { /// - `table_root`: The URL of the table within storage. /// - `options`: key/value pairs of options to pass to the object store. /// - `task_executor`: Used to spawn async IO tasks. See [executor::TaskExecutor]. - pub fn try_new(table_root: &Url, options: I, task_executor: Arc) -> DeltaResult + pub fn try_new( + table_root: &Url, + options: I, + task_executor: Arc, + ) -> DeltaResult where I: IntoIterator, K: AsRef, From 21928b8bb3127c9ba3b11a48d6b5d40c72021176 Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Mon, 7 Oct 2024 16:46:44 -0700 Subject: [PATCH 08/66] test cleanup --- kernel/src/engine/default/json.rs | 2 + kernel/src/engine/sync/json.rs | 11 ++--- kernel/tests/write.rs | 71 +++++++++++++++++++------------ 3 files changed, 52 insertions(+), 32 deletions(-) diff --git a/kernel/src/engine/default/json.rs b/kernel/src/engine/default/json.rs index 1914f315f..100254384 100644 --- a/kernel/src/engine/default/json.rs +++ b/kernel/src/engine/default/json.rs @@ -5,6 +5,7 @@ use std::ops::Range; use std::sync::Arc; use std::task::{ready, Poll}; +use arrow_array::RecordBatch; use arrow_json::ReaderBuilder; use arrow_schema::SchemaRef as ArrowSchemaRef; use bytes::{Buf, Bytes}; @@ -14,6 +15,7 @@ use object_store::{DynObjectStore, GetResultPayload}; use super::executor::TaskExecutor; use super::file_stream::{FileOpenFuture, FileOpener, FileStream}; +use crate::engine::arrow_data::ArrowEngineData; use crate::engine::arrow_utils::parse_json as arrow_parse_json; use crate::schema::SchemaRef; use crate::{ diff --git a/kernel/src/engine/sync/json.rs b/kernel/src/engine/sync/json.rs index e43fd5290..5b38d17df 100644 --- a/kernel/src/engine/sync/json.rs +++ b/kernel/src/engine/sync/json.rs @@ -1,13 +1,14 @@ use std::{fs::File, io::BufReader}; -use crate::{ - engine::arrow_utils::parse_json as arrow_parse_json, schema::SchemaRef, DeltaResult, - EngineData, Expression, FileDataReadResultIterator, FileMeta, JsonHandler, -}; use arrow_schema::SchemaRef as ArrowSchemaRef; +use url::Url; use super::read_files; -use crate::engine::arrow_data::ArrowEngineData; +use crate::{ + engine::arrow_data::ArrowEngineData, engine::arrow_utils::parse_json as arrow_parse_json, + schema::SchemaRef, DeltaResult, EngineData, Expression, FileDataReadResultIterator, FileMeta, + JsonHandler, +}; pub(crate) struct SyncJsonHandler; diff --git a/kernel/tests/write.rs b/kernel/tests/write.rs index 2ac241538..c20a13309 100644 --- a/kernel/tests/write.rs +++ b/kernel/tests/write.rs @@ -2,9 +2,12 @@ use std::sync::Arc; use arrow::array::StringArray; use arrow::record_batch::RecordBatch; +use arrow_schema::Schema as ArrowSchema; +use arrow_schema::{DataType as ArrowDataType, Field}; use object_store::memory::InMemory; use object_store::path::Path; use object_store::ObjectStore; +use serde_json::{json, to_vec}; use url::Url; use delta_kernel::engine::arrow_data::ArrowEngineData; @@ -13,10 +16,6 @@ use delta_kernel::engine::default::DefaultEngine; use delta_kernel::schema::{DataType, SchemaRef, StructField, StructType}; use delta_kernel::Table; -// fixme use in macro below -// const PROTOCOL_METADATA_TEMPLATE: &'static str = r#"{{"protocol":{{"minReaderVersion":3,"minWriterVersion":7,"readerFeatures":[],"writerFeatures":[]}}}} -// {{"metaData":{{"id":"{}","format":{{"provider":"parquet","options":{{}}}},"schemaString":"{}","partitionColumns":[],"configuration":{{}},"createdTime":1677811175819}}}}"#; - // setup default engine with in-memory object store. fn setup( table_name: &str, @@ -44,21 +43,44 @@ fn setup( async fn create_table( store: Arc, table_path: Url, - // fixme use this schema - _schema: SchemaRef, - schema_string: &str, - partition_columns: &str, + schema: SchemaRef, + partition_columns: &[&str], ) -> Result> { - // put 0.json with protocol + metadata let table_id = "test_id"; - let data = format!( - r#"{{"protocol":{{"minReaderVersion":3,"minWriterVersion":7,"readerFeatures":[],"writerFeatures":[]}}}} -{{"metaData":{{"id":"{}","format":{{"provider":"parquet","options":{{}}}},"schemaString":"{}","partitionColumns":[{}],"configuration":{{}},"createdTime":1677811175819}}}}"#, - table_id, schema_string, partition_columns - ).into_bytes(); + let schema = serde_json::to_string(&schema)?; + + let protocol = json!({ + "protocol": { + "minReaderVersion": 3, + "minWriterVersion": 7, + "readerFeatures": [], + "writerFeatures": [] + } + }); + let metadata = json!({ + "metaData": { + "id": table_id, + "format": { + "provider": "parquet", + "options": {} + }, + "schemaString": schema, + "partitionColumns": partition_columns, + "configuration": {}, + "createdTime": 1677811175819u64 + } + }); + + let data = [ + to_vec(&protocol).unwrap(), + b"\n".to_vec(), + to_vec(&metadata).unwrap(), + ] + .concat(); + + // put 0.json with protocol + metadata let path = table_path.path(); let path = format!("{path}_delta_log/00000000000000000000.json"); - println!("putting to path: {}", path); store.put(&Path::from(path), data.into()).await?; Ok(Table::new(table_path)) } @@ -69,25 +91,17 @@ async fn test_commit_info() -> Result<(), Box> { let _ = tracing_subscriber::fmt::try_init(); // setup in-memory object store and default engine let (store, engine, table_location) = setup("test_table"); + // create a simple table: one int column named 'number' let schema = Arc::new(StructType::new(vec![StructField::new( "number", DataType::INTEGER, true, )])); - let schema_string = r#"{\"type\":\"struct\",\"fields\":[{\"name\":\"number\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}}]}"#; - let table = create_table( - store.clone(), - table_location, - schema.clone(), - schema_string, - "", - ) - .await?; - let mut txn = table.new_transaction(&engine)?; + let table = create_table(store.clone(), table_location, schema, &[]).await?; - use arrow_schema::Schema as ArrowSchema; - use arrow_schema::{DataType as ArrowDataType, Field}; + // create a transaction + let mut txn = table.new_transaction(&engine)?; // add commit info of the form {engineInfo: "default engine"} let commit_info_schema = Arc::new(ArrowSchema::new(vec![Field::new( @@ -103,7 +117,10 @@ async fn test_commit_info() -> Result<(), Box> { Box::new(ArrowEngineData::new(commit_info_batch)), commit_info_schema.try_into()?, ); + + // commit! txn.commit(&engine)?; + let commit1 = store .get(&Path::from( "/test_table/_delta_log/00000000000000000001.json", From 532ea8c0a31cb2d95161ecb555cf1a4240729401 Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Mon, 7 Oct 2024 19:24:04 -0700 Subject: [PATCH 09/66] appease clippy --- kernel/src/lib.rs | 6 +++--- kernel/src/transaction.rs | 8 ++++---- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/kernel/src/lib.rs b/kernel/src/lib.rs index 69837525f..2d0726a40 100644 --- a/kernel/src/lib.rs +++ b/kernel/src/lib.rs @@ -213,10 +213,10 @@ pub trait JsonHandler: Send + Sync { /// /// - `path` - URL to write the JSON file to /// - `data` - Iterator of EngineData to write to the JSON file. each row should be written as - /// a new JSON object appended to the file. (that is, the file is newline-delimeted JSON, and - /// each row is a JSON object on a single line) + /// a new JSON object appended to the file. (that is, the file is newline-delimeted JSON, and + /// each row is a JSON object on a single line) /// - `overwrite` - If true, overwrite the file if it exists. If false, the call must fail if - /// the file exists. + /// the file exists. /// /// NOTE: the `overwrite` flag isn't used for the existing commit flow, but will be used in the /// future to write `_last_checkpoint` files. diff --git a/kernel/src/transaction.rs b/kernel/src/transaction.rs index 6d58e0d31..593d27c96 100644 --- a/kernel/src/transaction.rs +++ b/kernel/src/transaction.rs @@ -98,6 +98,7 @@ pub enum CommitResult { // given the engine's commit info (data and schema as [EngineCommitInfo]) we want to create both: // (1) the commitInfo action to commit (and append more actions to) and // (2) the schema for the actions to commit (this is determined by the engine's commit info schema) +#[allow(clippy::type_complexity)] fn generate_commit_info<'a>( engine: &'a dyn Engine, engine_commit_info: Option, @@ -124,10 +125,9 @@ fn generate_commit_info<'a>( }; commit_info_fields.extend(commit_info.schema.fields()); let commit_info_schema = - StructType::new(commit_info_fields.into_iter().map(|f| f.clone()).collect()); + StructType::new(commit_info_fields.into_iter().cloned().collect()); let mut action_fields = action_fields - .into_iter() - .map(|f| f.clone()) + .into_iter().cloned() .collect::>(); action_fields.push(crate::schema::StructField::new( COMMIT_INFO_NAME, @@ -171,7 +171,7 @@ fn generate_commit_info<'a>( // commit info has arbitrary schema ex: {engineInfo: string, operation: string} // we want to bundle it up and put it in the commit_info field of the actions. let commit_info_evaluator = engine.get_expression_handler().get_evaluator( - engine_commit_info_schema.into(), + engine_commit_info_schema, commit_info_expr, action_schema_ref.clone().into(), ); From 215ed4edb222c01ee3707b1d7010ce712de27665 Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Mon, 7 Oct 2024 21:53:21 -0700 Subject: [PATCH 10/66] fmt --- kernel/src/transaction.rs | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/kernel/src/transaction.rs b/kernel/src/transaction.rs index 593d27c96..eaa8e3431 100644 --- a/kernel/src/transaction.rs +++ b/kernel/src/transaction.rs @@ -126,9 +126,7 @@ fn generate_commit_info<'a>( commit_info_fields.extend(commit_info.schema.fields()); let commit_info_schema = StructType::new(commit_info_fields.into_iter().cloned().collect()); - let mut action_fields = action_fields - .into_iter().cloned() - .collect::>(); + let mut action_fields = action_fields.into_iter().cloned().collect::>(); action_fields.push(crate::schema::StructField::new( COMMIT_INFO_NAME, commit_info_schema, From 78c846421d39787afe7d1a76f8b18fac47715ac9 Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Mon, 7 Oct 2024 22:50:42 -0700 Subject: [PATCH 11/66] lil cleanup --- kernel/src/transaction.rs | 19 ++++++++----------- 1 file changed, 8 insertions(+), 11 deletions(-) diff --git a/kernel/src/transaction.rs b/kernel/src/transaction.rs index eaa8e3431..08d86d0e9 100644 --- a/kernel/src/transaction.rs +++ b/kernel/src/transaction.rs @@ -1,7 +1,6 @@ use std::sync::Arc; use crate::actions::get_log_schema; -use crate::expressions::Scalar; use crate::schema::{Schema, SchemaRef, StructType}; use crate::snapshot::Snapshot; use crate::{DataType, Expression}; @@ -116,13 +115,13 @@ fn generate_commit_info<'a>( get_log_schema().clone(), |commit_info| { let mut action_fields = get_log_schema().fields().collect::>(); - let commit_info_field = action_fields.pop().unwrap(); - let mut commit_info_fields = - if let DataType::Struct(commit_info_schema) = commit_info_field.data_type() { - commit_info_schema.fields().collect::>() - } else { - unreachable!() - }; + let commit_info_field = action_fields + .pop() + .expect("last field is commit_info in action schema"); + let DataType::Struct(commit_info_schema) = commit_info_field.data_type() else { + unreachable!("commit_info_field is a struct"); + }; + let mut commit_info_fields = commit_info_schema.fields().collect::>(); commit_info_fields.extend(commit_info.schema.fields()); let commit_info_schema = StructType::new(commit_info_fields.into_iter().cloned().collect()); @@ -158,9 +157,7 @@ fn generate_commit_info<'a>( ] .iter() .map(|name| { - Expression::Literal(Scalar::Null( - action_schema_ref.field(name).unwrap().data_type().clone(), - )) + Expression::null_literal(action_schema_ref.field(name).unwrap().data_type().clone()) }) .chain(std::iter::once(Expression::Struct(commit_info_expr))) .collect::>(); From 0f1f9555ed7cff50d1153e01eea92a8e79aed9cf Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Mon, 7 Oct 2024 23:19:06 -0700 Subject: [PATCH 12/66] add a test --- kernel/src/transaction.rs | 66 +++++++++++++++++++++++++++++++++++++++ 1 file changed, 66 insertions(+) diff --git a/kernel/src/transaction.rs b/kernel/src/transaction.rs index 08d86d0e9..45f681b54 100644 --- a/kernel/src/transaction.rs +++ b/kernel/src/transaction.rs @@ -177,3 +177,69 @@ fn generate_commit_info<'a>( }); Ok((Box::new(actions), action_schema)) } + +#[cfg(test)] +mod tests { + use super::*; + use crate::engine::arrow_data::ArrowEngineData; + use arrow::array::{Int32Array, StringArray}; + use arrow::record_batch::RecordBatch; + + struct ExprEngine(Arc); + + impl ExprEngine { + fn new() -> Self { + ExprEngine(Arc::new( + crate::engine::arrow_expression::ArrowExpressionHandler, + )) + } + } + + impl Engine for ExprEngine { + fn get_expression_handler(&self) -> Arc { + self.0.clone() + } + + fn get_json_handler(&self) -> Arc { + unimplemented!() + } + + fn get_parquet_handler(&self) -> Arc { + unimplemented!() + } + + fn get_file_system_client(&self) -> Arc { + unimplemented!() + } + } + + // simple test for generating commit info + #[test] + fn test_generate_commit_info() { + let engine = ExprEngine::new(); + let schema = Arc::new(arrow_schema::Schema::new(vec![ + arrow_schema::Field::new("engine_info", arrow_schema::DataType::Utf8, true), + arrow_schema::Field::new("operation", arrow_schema::DataType::Utf8, true), + arrow_schema::Field::new("int", arrow_schema::DataType::Int32, true), + ])); + let data = RecordBatch::try_new( + schema.clone(), + vec![ + Arc::new(StringArray::from(vec!["test engine info"])), + Arc::new(StringArray::from(vec!["append"])), + Arc::new(Int32Array::from(vec![42])), + ], + ); + let engine_commit_info = EngineCommitInfo { + data: Arc::new(ArrowEngineData::new(data.unwrap())), + schema: Arc::new(schema.try_into().unwrap()), + }; + let (actions, actions_schema) = + generate_commit_info(&engine, Some(engine_commit_info)).unwrap(); + let actions: Vec<_> = actions.collect(); + + // FIXME actual assertions + assert_eq!(actions_schema.fields().collect::>().len(), 6); + assert_eq!(actions.len(), 1); + } +} From 8cc9cc9932d87adce7a3ea02e76e5967e48b651d Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Mon, 7 Oct 2024 23:23:34 -0700 Subject: [PATCH 13/66] better assert --- kernel/src/transaction.rs | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/kernel/src/transaction.rs b/kernel/src/transaction.rs index 45f681b54..943b33d7c 100644 --- a/kernel/src/transaction.rs +++ b/kernel/src/transaction.rs @@ -240,6 +240,11 @@ mod tests { // FIXME actual assertions assert_eq!(actions_schema.fields().collect::>().len(), 6); + let DataType::Struct(struct_type) = actions_schema.field("commitInfo").unwrap().data_type() + else { + unreachable!("commitInfo is a struct"); + }; + assert_eq!(struct_type.fields().collect::>().len(), 4); assert_eq!(actions.len(), 1); } } From 114c16f303f642a1ffa58acd1b02480197bab490 Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Tue, 8 Oct 2024 08:30:31 -0700 Subject: [PATCH 14/66] address feedback --- kernel/src/engine/default/json.rs | 4 ++-- kernel/src/engine/sync/json.rs | 4 ++-- kernel/src/lib.rs | 4 ++-- kernel/src/transaction.rs | 9 ++++----- 4 files changed, 10 insertions(+), 11 deletions(-) diff --git a/kernel/src/engine/default/json.rs b/kernel/src/engine/default/json.rs index 100254384..f16b52413 100644 --- a/kernel/src/engine/default/json.rs +++ b/kernel/src/engine/default/json.rs @@ -92,10 +92,10 @@ impl JsonHandler for DefaultJsonHandler { } // note: for now we just buffer all the data and write it out all at once - fn write_json_file<'a>( + fn write_json_file( &self, path: &url::Url, - data: Box> + Send + 'a>, + data: Box> + Send + '_>, _overwrite: bool, ) -> DeltaResult<()> { let mut schema: Option = None; diff --git a/kernel/src/engine/sync/json.rs b/kernel/src/engine/sync/json.rs index 5b38d17df..dc3f8f89e 100644 --- a/kernel/src/engine/sync/json.rs +++ b/kernel/src/engine/sync/json.rs @@ -42,10 +42,10 @@ impl JsonHandler for SyncJsonHandler { arrow_parse_json(json_strings, output_schema) } - fn write_json_file<'a>( + fn write_json_file( &self, _path: &Url, - _data: Box> + Send + 'a>, + _data: Box> + Send + '_>, _overwrite: bool, ) -> DeltaResult<()> { unimplemented!() diff --git a/kernel/src/lib.rs b/kernel/src/lib.rs index 2d0726a40..0999ddaae 100644 --- a/kernel/src/lib.rs +++ b/kernel/src/lib.rs @@ -220,10 +220,10 @@ pub trait JsonHandler: Send + Sync { /// /// NOTE: the `overwrite` flag isn't used for the existing commit flow, but will be used in the /// future to write `_last_checkpoint` files. - fn write_json_file<'a>( + fn write_json_file( &self, path: &Url, - data: Box> + Send + 'a>, + data: Box> + Send + '_>, overwrite: bool, ) -> DeltaResult<()>; } diff --git a/kernel/src/transaction.rs b/kernel/src/transaction.rs index 943b33d7c..bae54dde7 100644 --- a/kernel/src/transaction.rs +++ b/kernel/src/transaction.rs @@ -66,8 +66,7 @@ impl Transaction { match json_handler.write_json_file(commit_path, Box::new(actions), false) { Ok(()) => Ok(CommitResult::Committed(commit_version)), Err(crate::error::Error::ObjectStore(object_store::Error::AlreadyExists { - path: _, - source: _, + .. })) => Ok(CommitResult::Conflict(self, commit_version)), Err(e) => Err(e), } @@ -98,11 +97,11 @@ pub enum CommitResult { // (1) the commitInfo action to commit (and append more actions to) and // (2) the schema for the actions to commit (this is determined by the engine's commit info schema) #[allow(clippy::type_complexity)] -fn generate_commit_info<'a>( - engine: &'a dyn Engine, +fn generate_commit_info( + engine: &'_ dyn Engine, engine_commit_info: Option, ) -> DeltaResult<( - Box> + Send + 'a>, + Box> + Send + '_>, SchemaRef, )> { use crate::actions::{ From b7c351fb2f4a7685e0675fa3f3b722cc8a91f7a6 Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Thu, 10 Oct 2024 13:44:35 -0700 Subject: [PATCH 15/66] address feedback, cleanup --- kernel/src/engine/default/json.rs | 21 +--- kernel/src/transaction.rs | 163 +++++++++++++++++------------- 2 files changed, 94 insertions(+), 90 deletions(-) diff --git a/kernel/src/engine/default/json.rs b/kernel/src/engine/default/json.rs index f16b52413..59ac1b846 100644 --- a/kernel/src/engine/default/json.rs +++ b/kernel/src/engine/default/json.rs @@ -5,7 +5,6 @@ use std::ops::Range; use std::sync::Arc; use std::task::{ready, Poll}; -use arrow_array::RecordBatch; use arrow_json::ReaderBuilder; use arrow_schema::SchemaRef as ArrowSchemaRef; use bytes::{Buf, Bytes}; @@ -98,28 +97,14 @@ impl JsonHandler for DefaultJsonHandler { data: Box> + Send + '_>, _overwrite: bool, ) -> DeltaResult<()> { - let mut schema: Option = None; - let mut batches: Vec = Vec::new(); - - for chunk in data { + let mut writer = arrow_json::LineDelimitedWriter::new(Vec::new()); + for chunk in data.into_iter() { let arrow_data = ArrowEngineData::try_from_engine_data(chunk)?; let record_batch = arrow_data.record_batch(); - - if schema.is_none() { - schema = Some(record_batch.schema()); - } - - batches.push(record_batch.clone()); + writer.write(&record_batch)?; } - // collect all batches - let batches: Vec<&RecordBatch> = batches.iter().collect(); - - // write the batches to JSON - let mut writer = arrow_json::LineDelimitedWriter::new(Vec::new()); - writer.write_batches(&batches)?; writer.finish()?; - let buffer = writer.into_inner(); // Put if absent diff --git a/kernel/src/transaction.rs b/kernel/src/transaction.rs index bae54dde7..94b62d581 100644 --- a/kernel/src/transaction.rs +++ b/kernel/src/transaction.rs @@ -6,6 +6,8 @@ use crate::snapshot::Snapshot; use crate::{DataType, Expression}; use crate::{DeltaResult, Engine, EngineData}; +use tracing::warn; + const KERNEL_VERSION: &str = env!("CARGO_PKG_VERSION"); /// A transaction represents an in-progress write to a table. @@ -50,7 +52,23 @@ impl Transaction { pub fn commit(self, engine: &dyn Engine) -> DeltaResult { // step one: construct the iterator of actions we want to commit // note: only support commit_info right now. - let (actions, _actions_schema) = generate_commit_info(engine, self.commit_info.clone())?; + let (actions, _actions_schema): ( + Box> + Send>, + SchemaRef, + ) = match self.commit_info { + Some(ref engine_commit_info) => { + let (actions, schema) = generate_commit_info(engine, engine_commit_info)?; + (Box::new(std::iter::once(actions)), schema) + } + None => { + ( + // if there is no commit info, actions are empty iterator and schema is just our + // known log schema. + Box::new(std::iter::empty()), + get_log_schema().clone().into(), + ) + } + }; // step two: set new commit version (current_version + 1) and path to write let commit_version = &self.read_snapshot.version() + 1; @@ -96,85 +114,88 @@ pub enum CommitResult { // given the engine's commit info (data and schema as [EngineCommitInfo]) we want to create both: // (1) the commitInfo action to commit (and append more actions to) and // (2) the schema for the actions to commit (this is determined by the engine's commit info schema) -#[allow(clippy::type_complexity)] fn generate_commit_info( - engine: &'_ dyn Engine, - engine_commit_info: Option, -) -> DeltaResult<( - Box> + Send + '_>, - SchemaRef, -)> { + engine: &dyn Engine, + engine_commit_info: &EngineCommitInfo, +) -> DeltaResult<(Box, SchemaRef)> { use crate::actions::{ ADD_NAME, COMMIT_INFO_NAME, METADATA_NAME, PROTOCOL_NAME, REMOVE_NAME, TRANSACTION_NAME, }; // TODO: enforce single row commit info + if engine_commit_info.data.length() != 1 { + warn!( + "Engine commit info should have exactly one row, found {}", + engine_commit_info.data.length() + ); + } // TODO: for now we always require commit info - let action_schema = Arc::new(engine_commit_info.as_ref().map_or( - get_log_schema().clone(), - |commit_info| { - let mut action_fields = get_log_schema().fields().collect::>(); - let commit_info_field = action_fields - .pop() - .expect("last field is commit_info in action schema"); - let DataType::Struct(commit_info_schema) = commit_info_field.data_type() else { - unreachable!("commit_info_field is a struct"); - }; - let mut commit_info_fields = commit_info_schema.fields().collect::>(); - commit_info_fields.extend(commit_info.schema.fields()); - let commit_info_schema = - StructType::new(commit_info_fields.into_iter().cloned().collect()); - let mut action_fields = action_fields.into_iter().cloned().collect::>(); - action_fields.push(crate::schema::StructField::new( + let mut action_fields = get_log_schema().fields().collect::>(); + let commit_info_field = action_fields + .pop() + .expect("last field is commit_info in action schema"); + let DataType::Struct(commit_info_schema) = commit_info_field.data_type() else { + unreachable!("commit_info_field is a struct"); + }; + let commit_info_fields = commit_info_schema + .fields() + .chain(engine_commit_info.schema.fields()) + .cloned() + .collect(); + let commit_info_schema = StructType::new(commit_info_fields); + let action_fields = + action_fields + .into_iter() + .cloned() + .chain(std::iter::once(crate::schema::StructField::new( COMMIT_INFO_NAME, commit_info_schema, true, - )); - StructType::new(action_fields) - }, - )); - - let action_schema_ref = Arc::clone(&action_schema); - let actions = engine_commit_info.into_iter().map(move |commit_info| { - let engine_commit_info_data = commit_info.data; - let engine_commit_info_schema = commit_info.schema; - let mut commit_info_expr = vec![Expression::literal(format!("v{}", KERNEL_VERSION))]; - commit_info_expr.extend( - engine_commit_info_schema + ))); + let action_schema = StructType::new(action_fields.collect()); + + let commit_info_expr = std::iter::once(Expression::literal(format!("v{}", KERNEL_VERSION))) + .chain( + engine_commit_info + .schema .fields() - .map(|f| Expression::column(f.name())) - .collect::>(), - ); - // generate expression with null for all the fields except the commit_info field, and - // append the commit_info to the end. - let commit_info_expr_fields = [ - ADD_NAME, - REMOVE_NAME, - METADATA_NAME, - PROTOCOL_NAME, - TRANSACTION_NAME, - ] - .iter() - .map(|name| { - Expression::null_literal(action_schema_ref.field(name).unwrap().data_type().clone()) - }) - .chain(std::iter::once(Expression::Struct(commit_info_expr))) - .collect::>(); - let commit_info_expr = Expression::Struct(commit_info_expr_fields); - - // commit info has arbitrary schema ex: {engineInfo: string, operation: string} - // we want to bundle it up and put it in the commit_info field of the actions. - let commit_info_evaluator = engine.get_expression_handler().get_evaluator( - engine_commit_info_schema, - commit_info_expr, - action_schema_ref.clone().into(), - ); + .map(|f| Expression::column(f.name())), + ) + .collect(); + + // generate expression with null for all the fields except the commit_info field, and + // append the commit_info to the end. + let commit_info_expr_fields = [ + ADD_NAME, + REMOVE_NAME, + METADATA_NAME, + PROTOCOL_NAME, + TRANSACTION_NAME, + ] + .iter() + .map(|name| { + Expression::null_literal( + action_schema + .field(name) + .expect("find field in action schema") + .data_type() + .clone(), + ) + }) + .chain(std::iter::once(Expression::Struct(commit_info_expr))) + .collect::>(); + let commit_info_expr = Expression::Struct(commit_info_expr_fields); + + // commit info has arbitrary schema ex: {engineInfo: string, operation: string} + // we want to bundle it up and put it in the commit_info field of the actions. + let commit_info_evaluator = engine.get_expression_handler().get_evaluator( + engine_commit_info.schema.clone(), + commit_info_expr, + action_schema.clone().into(), + ); - commit_info_evaluator - .evaluate(engine_commit_info_data.as_ref()) - .unwrap() - }); - Ok((Box::new(actions), action_schema)) + let actions = commit_info_evaluator.evaluate(engine_commit_info.data.as_ref())?; + Ok((actions, action_schema.into())) } #[cfg(test)] @@ -233,9 +254,7 @@ mod tests { data: Arc::new(ArrowEngineData::new(data.unwrap())), schema: Arc::new(schema.try_into().unwrap()), }; - let (actions, actions_schema) = - generate_commit_info(&engine, Some(engine_commit_info)).unwrap(); - let actions: Vec<_> = actions.collect(); + let (actions, actions_schema) = generate_commit_info(&engine, &engine_commit_info).unwrap(); // FIXME actual assertions assert_eq!(actions_schema.fields().collect::>().len(), 6); @@ -244,6 +263,6 @@ mod tests { unreachable!("commitInfo is a struct"); }; assert_eq!(struct_type.fields().collect::>().len(), 4); - assert_eq!(actions.len(), 1); + assert_eq!(actions.length(), 1); } } From 9a9e9d36ecfe318b30acb9529bd280e3f04d85d8 Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Thu, 10 Oct 2024 13:44:57 -0700 Subject: [PATCH 16/66] fmt --- kernel/src/engine/default/json.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kernel/src/engine/default/json.rs b/kernel/src/engine/default/json.rs index 59ac1b846..2617e8211 100644 --- a/kernel/src/engine/default/json.rs +++ b/kernel/src/engine/default/json.rs @@ -101,7 +101,7 @@ impl JsonHandler for DefaultJsonHandler { for chunk in data.into_iter() { let arrow_data = ArrowEngineData::try_from_engine_data(chunk)?; let record_batch = arrow_data.record_batch(); - writer.write(&record_batch)?; + writer.write(record_batch)?; } writer.finish()?; From 6b0c2d4553ce88a0415a5a16ad31cfbff9f7adcf Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Thu, 10 Oct 2024 13:45:28 -0700 Subject: [PATCH 17/66] Update kernel/src/engine/sync/json.rs Co-authored-by: Ryan Johnson --- kernel/src/engine/sync/json.rs | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/kernel/src/engine/sync/json.rs b/kernel/src/engine/sync/json.rs index dc3f8f89e..3e7182e81 100644 --- a/kernel/src/engine/sync/json.rs +++ b/kernel/src/engine/sync/json.rs @@ -4,11 +4,10 @@ use arrow_schema::SchemaRef as ArrowSchemaRef; use url::Url; use super::read_files; -use crate::{ - engine::arrow_data::ArrowEngineData, engine::arrow_utils::parse_json as arrow_parse_json, - schema::SchemaRef, DeltaResult, EngineData, Expression, FileDataReadResultIterator, FileMeta, - JsonHandler, -}; +use crate::engine::arrow_data::ArrowEngineData; +use crate::engine::arrow_utils::parse_json as arrow_parse_json; +use crate::schema::SchemaRef; +use crate::{DeltaResult, EngineData, Expression, FileDataReadResultIterator, FileMeta, JsonHandler}; pub(crate) struct SyncJsonHandler; From d1af098a2b3e5a4609a4d3172256de16752bb255 Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Thu, 10 Oct 2024 14:02:52 -0700 Subject: [PATCH 18/66] more feedback --- kernel/src/engine/default/mod.rs | 16 +--------------- kernel/src/engine/sync/json.rs | 4 +++- kernel/src/lib.rs | 8 +++++--- kernel/src/table.rs | 3 +-- kernel/src/transaction.rs | 4 ++-- 5 files changed, 12 insertions(+), 23 deletions(-) diff --git a/kernel/src/engine/default/mod.rs b/kernel/src/engine/default/mod.rs index e169674e1..2c574a47f 100644 --- a/kernel/src/engine/default/mod.rs +++ b/kernel/src/engine/default/mod.rs @@ -57,21 +57,7 @@ impl DefaultEngine { { // table root is the path of the table in the ObjectStore let (store, table_root) = parse_url_opts(table_root, options)?; - let store = Arc::new(store); - Ok(Self { - file_system: Arc::new(ObjectStoreFileSystemClient::new( - store.clone(), - table_root, - task_executor.clone(), - )), - json: Arc::new(DefaultJsonHandler::new( - store.clone(), - task_executor.clone(), - )), - parquet: Arc::new(DefaultParquetHandler::new(store.clone(), task_executor)), - store, - expression: Arc::new(ArrowExpressionHandler {}), - }) + Ok(Self::new(Arc::new(store), table_root, task_executor)) } /// Create a new [`DefaultEngine`] instance diff --git a/kernel/src/engine/sync/json.rs b/kernel/src/engine/sync/json.rs index 3e7182e81..659506faa 100644 --- a/kernel/src/engine/sync/json.rs +++ b/kernel/src/engine/sync/json.rs @@ -7,7 +7,9 @@ use super::read_files; use crate::engine::arrow_data::ArrowEngineData; use crate::engine::arrow_utils::parse_json as arrow_parse_json; use crate::schema::SchemaRef; -use crate::{DeltaResult, EngineData, Expression, FileDataReadResultIterator, FileMeta, JsonHandler}; +use crate::{ + DeltaResult, EngineData, Expression, FileDataReadResultIterator, FileMeta, JsonHandler, +}; pub(crate) struct SyncJsonHandler; diff --git a/kernel/src/lib.rs b/kernel/src/lib.rs index 0999ddaae..30648fab0 100644 --- a/kernel/src/lib.rs +++ b/kernel/src/lib.rs @@ -199,15 +199,17 @@ pub trait JsonHandler: Send + Sync { /// Atomically (!) write a single JSON file. Each row of the input data represents an action /// in the delta log. this PUT must: - /// (1) serialize the data to newline-delimited json (each row is a json object) + /// (1) serialize the data to newline-delimited json (each row is a json object literal) /// (2) write the data to the object store atomically (i.e. if the file already exists, fail /// unless the overwrite flag is set) /// /// The JSON data should be written as { "column1": "value1", "column2": "value2", ... } /// with each row on a new line. /// - /// Null columns should not be written to the JSON file. For example, if a row has columns - /// ["a", "b"] and the value of "b" is null, the JSON object should be written as { "a": "..." } + /// NOTE: Null columns should not be written to the JSON file. For example, if a row has columns + /// ["a", "b"] and the value of "b" is null, the JSON object should be written as + /// { "a": "..." }. Note that including nulls is technically valid JSON, but would bloat the + /// log, therefore we recommend omitting them. /// /// # Parameters /// diff --git a/kernel/src/table.rs b/kernel/src/table.rs index 8b1cf1d20..d7aba9894 100644 --- a/kernel/src/table.rs +++ b/kernel/src/table.rs @@ -80,8 +80,7 @@ impl Table { /// Create a new write transaction for this table. pub fn new_transaction(&self, engine: &dyn Engine) -> DeltaResult { - let latest_snapshot = Snapshot::try_new(self.location.clone(), engine, None)?; - Ok(Transaction::new(latest_snapshot)) + Ok(Transaction::new(self.snapshot(engine, None)?)) } } diff --git a/kernel/src/transaction.rs b/kernel/src/transaction.rs index 94b62d581..ba0124c7d 100644 --- a/kernel/src/transaction.rs +++ b/kernel/src/transaction.rs @@ -38,10 +38,10 @@ impl Transaction { /// Create a new transaction from a snapshot. The snapshot will be used to read the current /// state of the table (e.g. to read the current version). /// - /// Instead of using this API, the more typical API is + /// Instead of using this API, the more typical (user-facing) API is /// [Table::new_transaction](crate::table::Table::new_transaction) to create a transaction from /// a table automatically backed by the latest snapshot. - pub fn new(snapshot: impl Into>) -> Self { + pub(crate) fn new(snapshot: impl Into>) -> Self { Transaction { read_snapshot: snapshot.into(), commit_info: None, From 0ba047dd7dd370999366749baf1cb5bcd9b13c2d Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Thu, 10 Oct 2024 15:08:29 -0700 Subject: [PATCH 19/66] nits --- kernel/src/engine/default/mod.rs | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/kernel/src/engine/default/mod.rs b/kernel/src/engine/default/mod.rs index 2c574a47f..56d15ca86 100644 --- a/kernel/src/engine/default/mod.rs +++ b/kernel/src/engine/default/mod.rs @@ -45,13 +45,12 @@ impl DefaultEngine { /// - `table_root`: The URL of the table within storage. /// - `options`: key/value pairs of options to pass to the object store. /// - `task_executor`: Used to spawn async IO tasks. See [executor::TaskExecutor]. - pub fn try_new( + pub fn try_new( table_root: &Url, - options: I, + options: impl IntoIterator, task_executor: Arc, ) -> DeltaResult where - I: IntoIterator, K: AsRef, V: Into, { @@ -67,11 +66,11 @@ impl DefaultEngine { /// - `store`: The object store to use. /// - `table_root_path`: The root path of the table within storage. /// - `task_executor`: Used to spawn async IO tasks. See [executor::TaskExecutor]. - pub fn new(store: Arc, table_root_path: Path, task_executor: Arc) -> Self { + pub fn new(store: Arc, table_root: Path, task_executor: Arc) -> Self { Self { file_system: Arc::new(ObjectStoreFileSystemClient::new( store.clone(), - table_root_path, + table_root, task_executor.clone(), )), json: Arc::new(DefaultJsonHandler::new( From 667a8e2f369dcaf6e9102ea17807e0dc007e8cc5 Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Thu, 10 Oct 2024 15:20:46 -0700 Subject: [PATCH 20/66] add empty commit test --- kernel/tests/write.rs | 27 +++++++++++++++++++++++++++ 1 file changed, 27 insertions(+) diff --git a/kernel/tests/write.rs b/kernel/tests/write.rs index c20a13309..f44dda5ab 100644 --- a/kernel/tests/write.rs +++ b/kernel/tests/write.rs @@ -132,3 +132,30 @@ async fn test_commit_info() -> Result<(), Box> { ); Ok(()) } + +#[tokio::test] +async fn test_empty_commit() -> Result<(), Box> { + // setup tracing + let _ = tracing_subscriber::fmt::try_init(); + // setup in-memory object store and default engine + let (store, engine, table_location) = setup("test_table"); + + // create a simple table: one int column named 'number' + let schema = Arc::new(StructType::new(vec![StructField::new( + "number", + DataType::INTEGER, + true, + )])); + let table = create_table(store.clone(), table_location, schema, &[]).await?; + + // create a transaction and commit + table.new_transaction(&engine)?.commit(&engine)?; + + let commit1 = store + .get(&Path::from( + "/test_table/_delta_log/00000000000000000001.json", + )) + .await?; + assert!(commit1.bytes().await?.to_vec().is_empty()); + Ok(()) +} From 52bd5f26b9f19e1336034e219cd8788a60c3ccca Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Thu, 10 Oct 2024 16:29:08 -0700 Subject: [PATCH 21/66] add empty commit info tests, debugging expr --- kernel/src/engine/arrow_expression.rs | 8 ++--- kernel/src/expressions/scalars.rs | 2 +- kernel/src/transaction.rs | 2 +- kernel/tests/write.rs | 48 +++++++++++++++++++++++++++ 4 files changed, 54 insertions(+), 6 deletions(-) diff --git a/kernel/src/engine/arrow_expression.rs b/kernel/src/engine/arrow_expression.rs index ffd5d42db..4d92603ca 100644 --- a/kernel/src/engine/arrow_expression.rs +++ b/kernel/src/engine/arrow_expression.rs @@ -214,12 +214,12 @@ fn evaluate_expression( let output_fields: Vec = output_cols .iter() .zip(schema.fields()) - .map(|(array, input_field)| -> DeltaResult<_> { - ensure_data_types(input_field.data_type(), array.data_type())?; + .map(|(output_col, input_field)| -> DeltaResult<_> { + ensure_data_types(input_field.data_type(), output_col.data_type())?; Ok(ArrowField::new( input_field.name(), - array.data_type().clone(), - array.is_nullable(), + output_col.data_type().clone(), + input_field.is_nullable(), )) }) .try_collect()?; diff --git a/kernel/src/expressions/scalars.rs b/kernel/src/expressions/scalars.rs index 9e9e6ce75..8c934aa3a 100644 --- a/kernel/src/expressions/scalars.rs +++ b/kernel/src/expressions/scalars.rs @@ -67,7 +67,7 @@ impl StructData { require!( f.is_nullable() || !a.is_null(), Error::invalid_struct_data(format!( - "Value for non-nullable field {:?} cannto be null, got {}", + "Value for non-nullable field {:?} cannot be null, got {}", f.name(), a )) diff --git a/kernel/src/transaction.rs b/kernel/src/transaction.rs index ba0124c7d..c7766bca7 100644 --- a/kernel/src/transaction.rs +++ b/kernel/src/transaction.rs @@ -129,7 +129,7 @@ fn generate_commit_info( engine_commit_info.data.length() ); } - // TODO: for now we always require commit info + let mut action_fields = get_log_schema().fields().collect::>(); let commit_info_field = action_fields .pop() diff --git a/kernel/tests/write.rs b/kernel/tests/write.rs index f44dda5ab..2fe0da55a 100644 --- a/kernel/tests/write.rs +++ b/kernel/tests/write.rs @@ -130,6 +130,54 @@ async fn test_commit_info() -> Result<(), Box> { String::from_utf8(commit1.bytes().await?.to_vec())?, "{\"commitInfo\":{\"kernelVersion\":\"v0.3.1\",\"engineInfo\":\"default engine\"}}\n" ); + + // empty commit info test + let mut txn = table.new_transaction(&engine)?; + let commit_info_schema = Arc::new(ArrowSchema::empty()); + let commit_info_batch = RecordBatch::new_empty(commit_info_schema.clone()); + txn.commit_info( + Box::new(ArrowEngineData::new(commit_info_batch)), + commit_info_schema.try_into()?, + ); + + // commit! + txn.commit(&engine)?; + + let commit1 = store + .get(&Path::from( + "/test_table/_delta_log/00000000000000000002.json", + )) + .await?; + assert!(commit1.bytes().await?.is_empty()); + + // one null row commit info + let mut txn = table.new_transaction(&engine)?; + let commit_info_schema = Arc::new(ArrowSchema::new(vec![Field::new( + "some_column_name", + ArrowDataType::Utf8, + true, + )])); + let commit_info_batch = RecordBatch::try_new( + commit_info_schema.clone(), + vec![Arc::new(StringArray::new_null(1))], + )?; + txn.commit_info( + Box::new(ArrowEngineData::new(commit_info_batch)), + commit_info_schema.try_into()?, + ); + + // commit! + txn.commit(&engine)?; + + let commit1 = store + .get(&Path::from( + "/test_table/_delta_log/00000000000000000003.json", + )) + .await?; + assert_eq!( + String::from_utf8(commit1.bytes().await?.to_vec())?, + "{\"commitInfo\":{\"kernelVersion\":\"v0.3.1\"}}\n" + ); Ok(()) } From 7696d7dabc33633dbd0e65795edf049423287eb6 Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Thu, 10 Oct 2024 16:33:35 -0700 Subject: [PATCH 22/66] just make my test fail --- kernel/src/engine/arrow_expression.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kernel/src/engine/arrow_expression.rs b/kernel/src/engine/arrow_expression.rs index 4d92603ca..04932a3e5 100644 --- a/kernel/src/engine/arrow_expression.rs +++ b/kernel/src/engine/arrow_expression.rs @@ -219,7 +219,7 @@ fn evaluate_expression( Ok(ArrowField::new( input_field.name(), output_col.data_type().clone(), - input_field.is_nullable(), + output_col.is_nullable(), )) }) .try_collect()?; From fa6c81d64c8e768e6df394607aa2c56781baefed Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Fri, 11 Oct 2024 11:50:50 -0700 Subject: [PATCH 23/66] try to leverage ParsedLogPath? --- kernel/src/path.rs | 24 +++++++++++++++++------- kernel/src/transaction.rs | 16 ++++++++-------- 2 files changed, 25 insertions(+), 15 deletions(-) diff --git a/kernel/src/path.rs b/kernel/src/path.rs index bdb443509..fc15392da 100644 --- a/kernel/src/path.rs +++ b/kernel/src/path.rs @@ -73,6 +73,12 @@ impl AsUrl for FileMeta { } } +impl AsUrl for Url { + fn as_url(&self) -> &Url { + self + } +} + impl ParsedLogPath { // NOTE: We can't actually impl TryFrom because Option is a foreign struct even if T is local. #[cfg_attr(feature = "developer-visibility", visibility::make(pub))] @@ -176,19 +182,23 @@ impl ParsedLogPath { } } +impl ParsedLogPath { + pub(crate) fn new_commit( + table_root: &Url, + version: Version, + ) -> DeltaResult>> { + let filename = format!("{:020}.json", version); + let location = table_root.join("_delta_log/")?.join(&filename)?; + Self::try_from(location) + } +} + #[cfg(test)] mod tests { use std::path::PathBuf; use super::*; - // Easier to test directly with Url instead of FileMeta! - impl AsUrl for Url { - fn as_url(&self) -> &Url { - self - } - } - fn table_log_dir_url() -> Url { let path = PathBuf::from("./tests/data/table-with-dv-small/_delta_log/"); let path = std::fs::canonicalize(path).unwrap(); diff --git a/kernel/src/transaction.rs b/kernel/src/transaction.rs index c7766bca7..289fd5112 100644 --- a/kernel/src/transaction.rs +++ b/kernel/src/transaction.rs @@ -1,6 +1,7 @@ use std::sync::Arc; use crate::actions::get_log_schema; +use crate::path::ParsedLogPath; use crate::schema::{Schema, SchemaRef, StructType}; use crate::snapshot::Snapshot; use crate::{DataType, Expression}; @@ -71,17 +72,16 @@ impl Transaction { }; // step two: set new commit version (current_version + 1) and path to write - let commit_version = &self.read_snapshot.version() + 1; - let commit_file_name = format!("{:020}", commit_version) + ".json"; - let commit_path = &self - .read_snapshot - .table_root - .join("_delta_log/")? - .join(&commit_file_name)?; + let commit_version = self.read_snapshot.version() + 1; + let commit_path = + ParsedLogPath::new_commit(self.read_snapshot.table_root(), commit_version)? + .expect("valid commit path"); + + assert!(commit_path.is_commit(), "commit_path should be a commit path"); // step three: commit the actions as a json file in the log let json_handler = engine.get_json_handler(); - match json_handler.write_json_file(commit_path, Box::new(actions), false) { + match json_handler.write_json_file(&commit_path.location, Box::new(actions), false) { Ok(()) => Ok(CommitResult::Committed(commit_version)), Err(crate::error::Error::ObjectStore(object_store::Error::AlreadyExists { .. From a3abbfae9485ebc233dc5673bb0fad7fb1cfd134 Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Fri, 11 Oct 2024 11:54:50 -0700 Subject: [PATCH 24/66] fmt --- kernel/src/transaction.rs | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/kernel/src/transaction.rs b/kernel/src/transaction.rs index 289fd5112..a2d2eea16 100644 --- a/kernel/src/transaction.rs +++ b/kernel/src/transaction.rs @@ -76,8 +76,10 @@ impl Transaction { let commit_path = ParsedLogPath::new_commit(self.read_snapshot.table_root(), commit_version)? .expect("valid commit path"); - - assert!(commit_path.is_commit(), "commit_path should be a commit path"); + assert!( + commit_path.is_commit(), + "commit_path should be a commit path" + ); // step three: commit the actions as a json file in the log let json_handler = engine.get_json_handler(); From d7ea4c49f74f097c68b9d61fca9705e0e656791e Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Fri, 11 Oct 2024 13:00:35 -0700 Subject: [PATCH 25/66] enforce single-row commit info --- kernel/src/error.rs | 4 ++ kernel/src/transaction.rs | 18 ++++----- kernel/tests/write.rs | 80 +++++++++++++++++++++++++++++---------- 3 files changed, 72 insertions(+), 30 deletions(-) diff --git a/kernel/src/error.rs b/kernel/src/error.rs index b35cf8319..9898ec24e 100644 --- a/kernel/src/error.rs +++ b/kernel/src/error.rs @@ -159,6 +159,10 @@ pub enum Error { /// Unable to parse the name of a log path #[error("Invalid log path: {0}")] InvalidLogPath(String), + + /// Invalid commit info passed to the transaction + #[error("Invalid commit info: {0}")] + InvalidCommitInfo(String), } // Convenience constructors for Error types that take a String argument diff --git a/kernel/src/transaction.rs b/kernel/src/transaction.rs index a2d2eea16..512a2cbed 100644 --- a/kernel/src/transaction.rs +++ b/kernel/src/transaction.rs @@ -7,8 +7,6 @@ use crate::snapshot::Snapshot; use crate::{DataType, Expression}; use crate::{DeltaResult, Engine, EngineData}; -use tracing::warn; - const KERNEL_VERSION: &str = env!("CARGO_PKG_VERSION"); /// A transaction represents an in-progress write to a table. @@ -124,12 +122,11 @@ fn generate_commit_info( ADD_NAME, COMMIT_INFO_NAME, METADATA_NAME, PROTOCOL_NAME, REMOVE_NAME, TRANSACTION_NAME, }; - // TODO: enforce single row commit info if engine_commit_info.data.length() != 1 { - warn!( + return Err(crate::error::Error::InvalidCommitInfo(format!( "Engine commit info should have exactly one row, found {}", engine_commit_info.data.length() - ); + ))); } let mut action_fields = get_log_schema().fields().collect::>(); @@ -156,14 +153,16 @@ fn generate_commit_info( ))); let action_schema = StructType::new(action_fields.collect()); + // nullable = true + // println!("action_schema: {:#?}", action_schema); + let commit_info_expr = std::iter::once(Expression::literal(format!("v{}", KERNEL_VERSION))) .chain( engine_commit_info .schema .fields() .map(|f| Expression::column(f.name())), - ) - .collect(); + ); // generate expression with null for all the fields except the commit_info field, and // append the commit_info to the end. @@ -184,9 +183,8 @@ fn generate_commit_info( .clone(), ) }) - .chain(std::iter::once(Expression::Struct(commit_info_expr))) - .collect::>(); - let commit_info_expr = Expression::Struct(commit_info_expr_fields); + .chain(std::iter::once(Expression::struct_expr(commit_info_expr))); + let commit_info_expr = Expression::struct_expr(commit_info_expr_fields); // commit info has arbitrary schema ex: {engineInfo: string, operation: string} // we want to bundle it up and put it in the commit_info field of the actions. diff --git a/kernel/tests/write.rs b/kernel/tests/write.rs index 2fe0da55a..508c8a126 100644 --- a/kernel/tests/write.rs +++ b/kernel/tests/write.rs @@ -131,25 +131,6 @@ async fn test_commit_info() -> Result<(), Box> { "{\"commitInfo\":{\"kernelVersion\":\"v0.3.1\",\"engineInfo\":\"default engine\"}}\n" ); - // empty commit info test - let mut txn = table.new_transaction(&engine)?; - let commit_info_schema = Arc::new(ArrowSchema::empty()); - let commit_info_batch = RecordBatch::new_empty(commit_info_schema.clone()); - txn.commit_info( - Box::new(ArrowEngineData::new(commit_info_batch)), - commit_info_schema.try_into()?, - ); - - // commit! - txn.commit(&engine)?; - - let commit1 = store - .get(&Path::from( - "/test_table/_delta_log/00000000000000000002.json", - )) - .await?; - assert!(commit1.bytes().await?.is_empty()); - // one null row commit info let mut txn = table.new_transaction(&engine)?; let commit_info_schema = Arc::new(ArrowSchema::new(vec![Field::new( @@ -171,7 +152,7 @@ async fn test_commit_info() -> Result<(), Box> { let commit1 = store .get(&Path::from( - "/test_table/_delta_log/00000000000000000003.json", + "/test_table/_delta_log/00000000000000000002.json", )) .await?; assert_eq!( @@ -207,3 +188,62 @@ async fn test_empty_commit() -> Result<(), Box> { assert!(commit1.bytes().await?.to_vec().is_empty()); Ok(()) } + +#[tokio::test] +async fn test_invalid_commit_info() -> Result<(), Box> { + // setup tracing + let _ = tracing_subscriber::fmt::try_init(); + // setup in-memory object store and default engine + let (store, engine, table_location) = setup("test_table"); + + // create a simple table: one int column named 'number' + let schema = Arc::new(StructType::new(vec![StructField::new( + "number", + DataType::INTEGER, + true, + )])); + let table = create_table(store.clone(), table_location, schema, &[]).await?; + + // empty commit info test + let mut txn = table.new_transaction(&engine)?; + let commit_info_schema = Arc::new(ArrowSchema::empty()); + let commit_info_batch = RecordBatch::new_empty(commit_info_schema.clone()); + assert!(commit_info_batch.num_rows() == 0); + txn.commit_info( + Box::new(ArrowEngineData::new(commit_info_batch)), + commit_info_schema.try_into()?, + ); + + // commit! + assert!(matches!( + txn.commit(&engine), + Err(delta_kernel::Error::InvalidCommitInfo(_)) + )); + + // two-row commit info test + let mut txn = table.new_transaction(&engine)?; + let commit_info_schema = Arc::new(ArrowSchema::new(vec![Field::new( + "engineInfo", + ArrowDataType::Utf8, + true, + )])); + let commit_info_batch = RecordBatch::try_new( + commit_info_schema.clone(), + vec![Arc::new(StringArray::from(vec![ + "row1: default engine", + "row2: default engine", + ]))], + )?; + + txn.commit_info( + Box::new(ArrowEngineData::new(commit_info_batch)), + commit_info_schema.try_into()?, + ); + + // commit! + assert!(matches!( + txn.commit(&engine), + Err(delta_kernel::Error::InvalidCommitInfo(_)) + )); + Ok(()) +} From bac1d091a2501e8ffae743ecbe043f8ea82e426b Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Fri, 11 Oct 2024 13:01:56 -0700 Subject: [PATCH 26/66] error FFI --- ffi/src/lib.rs | 2 ++ 1 file changed, 2 insertions(+) diff --git a/ffi/src/lib.rs b/ffi/src/lib.rs index 75a5f34a3..0dbfc1727 100644 --- a/ffi/src/lib.rs +++ b/ffi/src/lib.rs @@ -329,6 +329,7 @@ pub enum KernelError { InternalError, InvalidExpression, InvalidLogPath, + InvalidCommitInfo, } impl From for KernelError { @@ -376,6 +377,7 @@ impl From for KernelError { } => Self::from(*source), Error::InvalidExpressionEvaluation(_) => KernelError::InvalidExpression, Error::InvalidLogPath(_) => KernelError::InvalidLogPath, + Error::InvalidCommitInfo(_) => KernelError::InvalidCommitInfo, } } } From bc541dd2ff6f4a4aaf24c9b4caf768a9cae55523 Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Fri, 11 Oct 2024 13:35:07 -0700 Subject: [PATCH 27/66] better path api --- kernel/src/path.rs | 6 ++++-- kernel/src/transaction.rs | 7 +------ 2 files changed, 5 insertions(+), 8 deletions(-) diff --git a/kernel/src/path.rs b/kernel/src/path.rs index fc15392da..e73872a1e 100644 --- a/kernel/src/path.rs +++ b/kernel/src/path.rs @@ -186,10 +186,12 @@ impl ParsedLogPath { pub(crate) fn new_commit( table_root: &Url, version: Version, - ) -> DeltaResult>> { + ) -> DeltaResult> { let filename = format!("{:020}.json", version); let location = table_root.join("_delta_log/")?.join(&filename)?; - Self::try_from(location) + let path = Self::try_from(location)?.expect("valid commit path"); + assert!(path.is_commit()); + Ok(path) } } diff --git a/kernel/src/transaction.rs b/kernel/src/transaction.rs index 512a2cbed..525ab0761 100644 --- a/kernel/src/transaction.rs +++ b/kernel/src/transaction.rs @@ -72,12 +72,7 @@ impl Transaction { // step two: set new commit version (current_version + 1) and path to write let commit_version = self.read_snapshot.version() + 1; let commit_path = - ParsedLogPath::new_commit(self.read_snapshot.table_root(), commit_version)? - .expect("valid commit path"); - assert!( - commit_path.is_commit(), - "commit_path should be a commit path" - ); + ParsedLogPath::new_commit(self.read_snapshot.table_root(), commit_version)?; // step three: commit the actions as a json file in the log let json_handler = engine.get_json_handler(); From fa1caf4d6a5408bef156b6d318760356b674b6a3 Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Fri, 11 Oct 2024 13:53:23 -0700 Subject: [PATCH 28/66] comment --- kernel/src/path.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/kernel/src/path.rs b/kernel/src/path.rs index e73872a1e..00117a48c 100644 --- a/kernel/src/path.rs +++ b/kernel/src/path.rs @@ -183,6 +183,7 @@ impl ParsedLogPath { } impl ParsedLogPath { + /// Create a new ParsedCommitPath for a new json commit file at the specified version pub(crate) fn new_commit( table_root: &Url, version: Version, From 9d875cd09e6d396baf6c9000f814efc229329822 Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Fri, 11 Oct 2024 14:05:42 -0700 Subject: [PATCH 29/66] clean --- kernel/src/transaction.rs | 3 --- 1 file changed, 3 deletions(-) diff --git a/kernel/src/transaction.rs b/kernel/src/transaction.rs index 525ab0761..f1903540e 100644 --- a/kernel/src/transaction.rs +++ b/kernel/src/transaction.rs @@ -148,9 +148,6 @@ fn generate_commit_info( ))); let action_schema = StructType::new(action_fields.collect()); - // nullable = true - // println!("action_schema: {:#?}", action_schema); - let commit_info_expr = std::iter::once(Expression::literal(format!("v{}", KERNEL_VERSION))) .chain( engine_commit_info From 023b85ad53c80aed0eabdaf06065e2eb953e2580 Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Fri, 11 Oct 2024 15:46:31 -0700 Subject: [PATCH 30/66] fix all the schema mess --- kernel/src/transaction.rs | 40 +++++++++++++++++---------------------- 1 file changed, 17 insertions(+), 23 deletions(-) diff --git a/kernel/src/transaction.rs b/kernel/src/transaction.rs index f1903540e..5aac08205 100644 --- a/kernel/src/transaction.rs +++ b/kernel/src/transaction.rs @@ -2,7 +2,7 @@ use std::sync::Arc; use crate::actions::get_log_schema; use crate::path::ParsedLogPath; -use crate::schema::{Schema, SchemaRef, StructType}; +use crate::schema::{Schema, SchemaRef}; use crate::snapshot::Snapshot; use crate::{DataType, Expression}; use crate::{DeltaResult, Engine, EngineData}; @@ -124,29 +124,23 @@ fn generate_commit_info( ))); } - let mut action_fields = get_log_schema().fields().collect::>(); - let commit_info_field = action_fields - .pop() - .expect("last field is commit_info in action schema"); - let DataType::Struct(commit_info_schema) = commit_info_field.data_type() else { - unreachable!("commit_info_field is a struct"); + let mut action_schema = get_log_schema().clone(); + let commit_info_field = action_schema + .fields + .get_mut(COMMIT_INFO_NAME) + .ok_or_else(|| crate::Error::missing_column(COMMIT_INFO_NAME))?; + let DataType::Struct(mut commit_info_data_type) = commit_info_field.data_type().clone() else { + return Err(crate::error::Error::internal_error( + "commit_info_field is a struct", + )); }; - let commit_info_fields = commit_info_schema - .fields() - .chain(engine_commit_info.schema.fields()) - .cloned() - .collect(); - let commit_info_schema = StructType::new(commit_info_fields); - let action_fields = - action_fields - .into_iter() - .cloned() - .chain(std::iter::once(crate::schema::StructField::new( - COMMIT_INFO_NAME, - commit_info_schema, - true, - ))); - let action_schema = StructType::new(action_fields.collect()); + commit_info_data_type.fields.extend( + engine_commit_info + .schema + .fields() + .map(|f| (f.name.clone(), f.clone())), + ); + commit_info_field.data_type = DataType::Struct(commit_info_data_type); let commit_info_expr = std::iter::once(Expression::literal(format!("v{}", KERNEL_VERSION))) .chain( From c1c6e2a47ef96aeac40606707de62dec31489d1f Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Fri, 11 Oct 2024 16:11:09 -0700 Subject: [PATCH 31/66] remove lifetime --- kernel/src/engine/default/json.rs | 2 +- kernel/src/engine/sync/json.rs | 2 +- kernel/src/lib.rs | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kernel/src/engine/default/json.rs b/kernel/src/engine/default/json.rs index 2617e8211..a4b39b2cc 100644 --- a/kernel/src/engine/default/json.rs +++ b/kernel/src/engine/default/json.rs @@ -94,7 +94,7 @@ impl JsonHandler for DefaultJsonHandler { fn write_json_file( &self, path: &url::Url, - data: Box> + Send + '_>, + data: Box> + Send>, _overwrite: bool, ) -> DeltaResult<()> { let mut writer = arrow_json::LineDelimitedWriter::new(Vec::new()); diff --git a/kernel/src/engine/sync/json.rs b/kernel/src/engine/sync/json.rs index 659506faa..74a97f1a0 100644 --- a/kernel/src/engine/sync/json.rs +++ b/kernel/src/engine/sync/json.rs @@ -46,7 +46,7 @@ impl JsonHandler for SyncJsonHandler { fn write_json_file( &self, _path: &Url, - _data: Box> + Send + '_>, + _data: Box> + Send>, _overwrite: bool, ) -> DeltaResult<()> { unimplemented!() diff --git a/kernel/src/lib.rs b/kernel/src/lib.rs index 30648fab0..54425b52e 100644 --- a/kernel/src/lib.rs +++ b/kernel/src/lib.rs @@ -225,7 +225,7 @@ pub trait JsonHandler: Send + Sync { fn write_json_file( &self, path: &Url, - data: Box> + Send + '_>, + data: Box> + Send>, overwrite: bool, ) -> DeltaResult<()>; } From da43cf2841a3e31a0048bf6a27a13b6f24b9b049 Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Fri, 11 Oct 2024 16:25:17 -0700 Subject: [PATCH 32/66] fix executor --- kernel/src/engine/default/json.rs | 16 +++++++--------- 1 file changed, 7 insertions(+), 9 deletions(-) diff --git a/kernel/src/engine/default/json.rs b/kernel/src/engine/default/json.rs index a4b39b2cc..53b040cf5 100644 --- a/kernel/src/engine/default/json.rs +++ b/kernel/src/engine/default/json.rs @@ -11,6 +11,7 @@ use bytes::{Buf, Bytes}; use futures::{StreamExt, TryStreamExt}; use object_store::path::Path; use object_store::{DynObjectStore, GetResultPayload}; +use url::Url; use super::executor::TaskExecutor; use super::file_stream::{FileOpenFuture, FileOpener, FileStream}; @@ -93,7 +94,7 @@ impl JsonHandler for DefaultJsonHandler { // note: for now we just buffer all the data and write it out all at once fn write_json_file( &self, - path: &url::Url, + path: &Url, data: Box> + Send>, _overwrite: bool, ) -> DeltaResult<()> { @@ -103,18 +104,15 @@ impl JsonHandler for DefaultJsonHandler { let record_batch = arrow_data.record_batch(); writer.write(record_batch)?; } - writer.finish()?; let buffer = writer.into_inner(); // Put if absent - futures::executor::block_on(async { - self.store - .put_opts( - &Path::from(path.path()), - buffer.into(), - object_store::PutMode::Create.into(), - ) + let store = self.store.clone(); // cheap Arc + let path = Path::from(path.path()); + self.task_executor.block_on(async move { + store + .put_opts(&path, buffer.into(), object_store::PutMode::Create.into()) .await })?; From 26b8dbd2d36747394922cf642c936bb518a93309 Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Fri, 11 Oct 2024 16:49:19 -0700 Subject: [PATCH 33/66] docs and i forgot a test --- kernel/src/path.rs | 17 ++++++++++++++++- kernel/src/transaction.rs | 15 +++++++++++++-- 2 files changed, 29 insertions(+), 3 deletions(-) diff --git a/kernel/src/path.rs b/kernel/src/path.rs index 00117a48c..8caf82c01 100644 --- a/kernel/src/path.rs +++ b/kernel/src/path.rs @@ -191,7 +191,11 @@ impl ParsedLogPath { let filename = format!("{:020}.json", version); let location = table_root.join("_delta_log/")?.join(&filename)?; let path = Self::try_from(location)?.expect("valid commit path"); - assert!(path.is_commit()); + if !path.is_commit() { + return Err(Error::internal_error( + "ParsedLogPath::new_commit created a non-commit path", + )); + } Ok(path) } } @@ -531,4 +535,15 @@ mod tests { .unwrap(); ParsedLogPath::try_from(log_path).expect_err("non-numeric hi"); } + + #[test] + fn test_new_commit() { + let table_log_dir = table_log_dir_url(); + let log_path = ParsedLogPath::new_commit(&table_log_dir, 10).unwrap(); + assert_eq!(log_path.version, 10); + assert!(log_path.is_commit()); + assert_eq!(log_path.extension, "json"); + assert!(matches!(log_path.file_type, LogPathFileType::Commit)); + assert_eq!(log_path.filename, "00000000000000000010.json"); + } } diff --git a/kernel/src/transaction.rs b/kernel/src/transaction.rs index 5aac08205..0f934b493 100644 --- a/kernel/src/transaction.rs +++ b/kernel/src/transaction.rs @@ -86,8 +86,19 @@ impl Transaction { } /// Add commit info to the transaction. This is commit-wide metadata that is written as the - /// first action in the commit. Note it is required in order to commit. If the engine does not - /// require any commit info, pass an empty `EngineData`. + /// first action in the commit. + /// + /// Note that there are two main pieces of information included in commit info: (1) custom + /// engine commit info (specified via this API) and (2) delta's own commit info which is + /// effectively appended to the engine-specific commit info. + /// + /// If the engine elects to omit commit info, it can do so in two ways: + /// 1. skip this API entirely - this will omit the commitInfo action from the commit (that is, + /// it will prevent the kernel from writing delta-specific commitInfo). This precludes usage + /// of table features which require commitInfo like in-commit timestamps. + /// 2. pass an empty commit_info data chunk - this will allow kernel to include delta-specific + /// commit info, resulting in a commitInfo action in the log, just without any + /// engine-specific additions. pub fn commit_info(&mut self, commit_info: Box, schema: Schema) { self.commit_info = Some(EngineCommitInfo { data: commit_info.into(), From 858f3fb5fb6c4a1725ceac448f86b570d9422e1e Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Sun, 13 Oct 2024 12:58:18 -0700 Subject: [PATCH 34/66] add commit info schema test --- kernel/src/actions/mod.rs | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/kernel/src/actions/mod.rs b/kernel/src/actions/mod.rs index 093e42e09..8d4844df1 100644 --- a/kernel/src/actions/mod.rs +++ b/kernel/src/actions/mod.rs @@ -422,4 +422,22 @@ mod tests { )])); assert_eq!(schema, expected); } + + #[test] + fn test_commit_info_schema() { + let schema = get_log_schema() + .project(&["commitInfo"]) + .expect("Couldn't get commitInfo field"); + + let expected = Arc::new(StructType::new(vec![StructField::new( + "commitInfo", + StructType::new(vec![StructField::new( + "kernelVersion", + DataType::STRING, + true, + )]), + true, + )])); + assert_eq!(schema, expected); + } } From 1ef5ffca7440c288fb103af385c3a9caf2f40469 Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Sun, 13 Oct 2024 16:08:23 -0700 Subject: [PATCH 35/66] add sync json writer, add FileAlreadyExists error --- ffi/src/lib.rs | 2 + kernel/Cargo.toml | 3 + kernel/src/engine/arrow_utils.rs | 38 ++++++++++++- kernel/src/engine/default/json.rs | 31 +++++----- kernel/src/engine/sync/json.rs | 95 +++++++++++++++++++++++++++++-- kernel/src/error.rs | 4 ++ kernel/src/transaction.rs | 6 +- 7 files changed, 155 insertions(+), 24 deletions(-) diff --git a/ffi/src/lib.rs b/ffi/src/lib.rs index 0dbfc1727..f54a0d412 100644 --- a/ffi/src/lib.rs +++ b/ffi/src/lib.rs @@ -330,6 +330,7 @@ pub enum KernelError { InvalidExpression, InvalidLogPath, InvalidCommitInfo, + FileAlreadyExists, } impl From for KernelError { @@ -378,6 +379,7 @@ impl From for KernelError { Error::InvalidExpressionEvaluation(_) => KernelError::InvalidExpression, Error::InvalidLogPath(_) => KernelError::InvalidLogPath, Error::InvalidCommitInfo(_) => KernelError::InvalidCommitInfo, + Error::FileAlreadyExists(_) => KernelError::FileAlreadyExists, } } } diff --git a/kernel/Cargo.toml b/kernel/Cargo.toml index 533d90842..85663067a 100644 --- a/kernel/Cargo.toml +++ b/kernel/Cargo.toml @@ -37,6 +37,8 @@ delta_kernel_derive = { path = "../derive-macros", version = "0.3.1" } # used for developer-visibility visibility = "0.1.1" +# Used in the sync engine +tempfile = { version = "3", optional = true } # Used in default engine arrow-buffer = { workspace = true, optional = true } arrow-array = { workspace = true, optional = true, features = ["chrono-tz"] } @@ -100,6 +102,7 @@ sync-engine = [ "arrow-json", "arrow-select", "parquet", + "tempfile", ] integration-test = [ "hdfs-native-object-store/integration-test", diff --git a/kernel/src/engine/arrow_utils.rs b/kernel/src/engine/arrow_utils.rs index c0c9b0cae..b626b2ab4 100644 --- a/kernel/src/engine/arrow_utils.rs +++ b/kernel/src/engine/arrow_utils.rs @@ -15,7 +15,7 @@ use arrow_array::{ cast::AsArray, new_null_array, Array as ArrowArray, GenericListArray, OffsetSizeTrait, RecordBatch, StringArray, StructArray, }; -use arrow_json::ReaderBuilder; +use arrow_json::{LineDelimitedWriter, ReaderBuilder}; use arrow_schema::{ DataType as ArrowDataType, Field as ArrowField, FieldRef as ArrowFieldRef, Fields, SchemaRef as ArrowSchemaRef, @@ -813,6 +813,22 @@ fn parse_json_impl(json_strings: &StringArray, schema: ArrowSchemaRef) -> DeltaR Ok(concat_batches(&schema, output.iter())?) } +/// write an arrow RecordBatch to a JSON string by appending to a buffer. +/// +/// TODO: this should stream data to the JSON writer and output an iterator. +pub(crate) fn write_json( + data: Box> + Send>, +) -> DeltaResult> { + let mut writer = LineDelimitedWriter::new(Vec::new()); + for chunk in data.into_iter() { + let arrow_data = ArrowEngineData::try_from_engine_data(chunk)?; + let record_batch = arrow_data.record_batch(); + writer.write(record_batch)?; + } + writer.finish()?; + Ok(writer.into_inner()) +} + #[cfg(test)] mod tests { use std::sync::Arc; @@ -1621,4 +1637,24 @@ mod tests { assert!(!can_upcast_to_decimal(&Int64, 19u8, 0i8)); assert!(!can_upcast_to_decimal(&Int64, 20u8, 1i8)); } + + #[test] + fn test_write_json() -> DeltaResult<()> { + let schema = Arc::new(ArrowSchema::new(vec![ArrowField::new( + "string", + ArrowDataType::Utf8, + true, + )])); + let data = RecordBatch::try_new( + schema.clone(), + vec![Arc::new(StringArray::from(vec!["string1", "string2"]))], + )?; + let data: Box = Box::new(ArrowEngineData::new(data)); + let json = write_json(Box::new(std::iter::once(data)))?; + assert_eq!( + json, + "{\"string\":\"string1\"}\n{\"string\":\"string2\"}\n".as_bytes() + ); + Ok(()) + } } diff --git a/kernel/src/engine/default/json.rs b/kernel/src/engine/default/json.rs index 53b040cf5..d0b9adf56 100644 --- a/kernel/src/engine/default/json.rs +++ b/kernel/src/engine/default/json.rs @@ -15,8 +15,8 @@ use url::Url; use super::executor::TaskExecutor; use super::file_stream::{FileOpenFuture, FileOpener, FileStream}; -use crate::engine::arrow_data::ArrowEngineData; use crate::engine::arrow_utils::parse_json as arrow_parse_json; +use crate::engine::arrow_utils::write_json; use crate::schema::SchemaRef; use crate::{ DeltaResult, EngineData, Error, Expression, FileDataReadResultIterator, FileMeta, JsonHandler, @@ -98,24 +98,23 @@ impl JsonHandler for DefaultJsonHandler { data: Box> + Send>, _overwrite: bool, ) -> DeltaResult<()> { - let mut writer = arrow_json::LineDelimitedWriter::new(Vec::new()); - for chunk in data.into_iter() { - let arrow_data = ArrowEngineData::try_from_engine_data(chunk)?; - let record_batch = arrow_data.record_batch(); - writer.write(record_batch)?; - } - writer.finish()?; - let buffer = writer.into_inner(); - + let buffer = write_json(data)?; // Put if absent let store = self.store.clone(); // cheap Arc let path = Path::from(path.path()); - self.task_executor.block_on(async move { - store - .put_opts(&path, buffer.into(), object_store::PutMode::Create.into()) - .await - })?; - + let path2 = path.clone(); // FIXME gross + self.task_executor + .block_on(async move { + store + .put_opts(&path, buffer.into(), object_store::PutMode::Create.into()) + .await + }) + .map_err(|e| match e { + object_store::Error::AlreadyExists { .. } => { + crate::error::Error::FileAlreadyExists(path2.to_string()) + } + e => e.into(), + })?; Ok(()) } } diff --git a/kernel/src/engine/sync/json.rs b/kernel/src/engine/sync/json.rs index 74a97f1a0..e91ce180e 100644 --- a/kernel/src/engine/sync/json.rs +++ b/kernel/src/engine/sync/json.rs @@ -1,11 +1,13 @@ -use std::{fs::File, io::BufReader}; +use std::{fs::File, io::BufReader, io::Write}; use arrow_schema::SchemaRef as ArrowSchemaRef; +use tempfile::NamedTempFile; use url::Url; use super::read_files; use crate::engine::arrow_data::ArrowEngineData; use crate::engine::arrow_utils::parse_json as arrow_parse_json; +use crate::engine::arrow_utils::write_json; use crate::schema::SchemaRef; use crate::{ DeltaResult, EngineData, Expression, FileDataReadResultIterator, FileMeta, JsonHandler, @@ -43,12 +45,97 @@ impl JsonHandler for SyncJsonHandler { arrow_parse_json(json_strings, output_schema) } + // For sync writer we write data to a tmp file then atomically rename it to the final path. + // This is highly OS-dependent and for now relies on the atomicity of tempfile's + // `persist_noclobber`. fn write_json_file( &self, - _path: &Url, - _data: Box> + Send>, + path: &Url, + data: Box> + Send>, _overwrite: bool, ) -> DeltaResult<()> { - unimplemented!() + let path = path + .to_file_path() + .map_err(|_| crate::Error::generic("sync client can only read local files"))?; + let Some(parent) = path.parent() else { + return Err(crate::Error::generic(format!( + "no parent found for {:?}", + path + ))); + }; + + // write data to tmp file + let mut tmp_file = NamedTempFile::new_in(parent)?; + let buf = write_json(data)?; + tmp_file.write_all(&buf)?; + tmp_file.flush()?; + + // use 'persist_noclobber' to atomically rename tmp file to final path + tmp_file + .persist_noclobber(path.clone()) + .map_err(|e| match e { + tempfile::PersistError { error, .. } + if error.kind() == std::io::ErrorKind::AlreadyExists => + { + crate::Error::FileAlreadyExists(path.to_string_lossy().to_string()) + } + e => crate::Error::IOError(e.into()), + })?; + Ok(()) + } +} + +#[cfg(test)] +mod tests { + use super::*; + + use std::sync::Arc; + + use arrow_array::{RecordBatch, StringArray}; + use arrow_schema::DataType as ArrowDataType; + use arrow_schema::Field; + use arrow_schema::Schema as ArrowSchema; + use serde_json::json; + use url::Url; + + #[test] + fn test_write_json_file() -> DeltaResult<()> { + let test_dir = tempfile::tempdir().unwrap(); + let path = test_dir.path().join("00000000000000000001.json"); + let handler = SyncJsonHandler; + + let schema = Arc::new(ArrowSchema::new(vec![Field::new( + "dog", + ArrowDataType::Utf8, + true, + )])); + let data = RecordBatch::try_new( + schema.clone(), + vec![Arc::new(StringArray::from(vec!["remi", "wilson"]))], + )?; + let data: Box = Box::new(ArrowEngineData::new(data)); + let empty: Box = + Box::new(ArrowEngineData::new(RecordBatch::new_empty(schema))); + + let url = Url::from_file_path(path.clone()).unwrap(); + handler + .write_json_file(&url, Box::new(std::iter::once(data)), false) + .expect("write json file"); + assert!(matches!( + handler.write_json_file(&url, Box::new(std::iter::once(empty)), false), + Err(crate::Error::FileAlreadyExists(_)) + )); + + let file = std::fs::read_to_string(path)?; + let json: Vec<_> = serde_json::Deserializer::from_str(&file) + .into_iter::() + .flatten() + .collect(); + assert_eq!( + json, + vec![json!({"dog": "remi"}), json!({"dog": "wilson"}),] + ); + + Ok(()) } } diff --git a/kernel/src/error.rs b/kernel/src/error.rs index 9898ec24e..5458302bd 100644 --- a/kernel/src/error.rs +++ b/kernel/src/error.rs @@ -163,6 +163,10 @@ pub enum Error { /// Invalid commit info passed to the transaction #[error("Invalid commit info: {0}")] InvalidCommitInfo(String), + + /// The file already exists at the path, prohibiting a non-overwrite write + #[error("File already exists: {0}")] + FileAlreadyExists(String), } // Convenience constructors for Error types that take a String argument diff --git a/kernel/src/transaction.rs b/kernel/src/transaction.rs index 0f934b493..81014e4fd 100644 --- a/kernel/src/transaction.rs +++ b/kernel/src/transaction.rs @@ -78,9 +78,9 @@ impl Transaction { let json_handler = engine.get_json_handler(); match json_handler.write_json_file(&commit_path.location, Box::new(actions), false) { Ok(()) => Ok(CommitResult::Committed(commit_version)), - Err(crate::error::Error::ObjectStore(object_store::Error::AlreadyExists { - .. - })) => Ok(CommitResult::Conflict(self, commit_version)), + Err(crate::error::Error::FileAlreadyExists(_)) => { + Ok(CommitResult::Conflict(self, commit_version)) + } Err(e) => Err(e), } } From 6ee69e77f5582af64f33a0fdead3ecb50e61e10a Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Mon, 14 Oct 2024 11:16:41 -0700 Subject: [PATCH 36/66] fix rebase --- kernel/src/actions/mod.rs | 5 ----- kernel/src/lib.rs | 1 + kernel/src/transaction.rs | 4 ++-- 3 files changed, 3 insertions(+), 7 deletions(-) diff --git a/kernel/src/actions/mod.rs b/kernel/src/actions/mod.rs index 8d4844df1..43b0f77fc 100644 --- a/kernel/src/actions/mod.rs +++ b/kernel/src/actions/mod.rs @@ -141,11 +141,6 @@ pub struct CommitInfo { pub kernel_version: Option, } -#[derive(Debug, Clone, PartialEq, Eq, Schema)] -pub struct CommitInfo { - pub kernel_version: Option, -} - #[derive(Debug, Clone, PartialEq, Eq, Schema)] pub struct Add { /// A relative path to a data file from the root of the table or an absolute path to a file diff --git a/kernel/src/lib.rs b/kernel/src/lib.rs index 54425b52e..754ed197a 100644 --- a/kernel/src/lib.rs +++ b/kernel/src/lib.rs @@ -73,6 +73,7 @@ pub mod scan; pub mod schema; pub mod snapshot; pub mod table; +pub mod transaction; pub(crate) mod utils; pub use engine_data::{DataVisitor, EngineData}; diff --git a/kernel/src/transaction.rs b/kernel/src/transaction.rs index 81014e4fd..9067759db 100644 --- a/kernel/src/transaction.rs +++ b/kernel/src/transaction.rs @@ -125,7 +125,7 @@ fn generate_commit_info( engine_commit_info: &EngineCommitInfo, ) -> DeltaResult<(Box, SchemaRef)> { use crate::actions::{ - ADD_NAME, COMMIT_INFO_NAME, METADATA_NAME, PROTOCOL_NAME, REMOVE_NAME, TRANSACTION_NAME, + ADD_NAME, COMMIT_INFO_NAME, METADATA_NAME, PROTOCOL_NAME, REMOVE_NAME, SET_TRANSACTION_NAME, }; if engine_commit_info.data.length() != 1 { @@ -168,7 +168,7 @@ fn generate_commit_info( REMOVE_NAME, METADATA_NAME, PROTOCOL_NAME, - TRANSACTION_NAME, + SET_TRANSACTION_NAME, ] .iter() .map(|name| { From 0b2b1edac37852c9e5f56f5631596e5eec0ea1c6 Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Mon, 14 Oct 2024 11:43:11 -0700 Subject: [PATCH 37/66] remove old file --- kernel/src/actions/transaction.rs | 120 ------------------------------ 1 file changed, 120 deletions(-) delete mode 100644 kernel/src/actions/transaction.rs diff --git a/kernel/src/actions/transaction.rs b/kernel/src/actions/transaction.rs deleted file mode 100644 index 1242f754e..000000000 --- a/kernel/src/actions/transaction.rs +++ /dev/null @@ -1,120 +0,0 @@ -use std::sync::Arc; - -use crate::actions::visitors::TransactionVisitor; -use crate::actions::{get_log_schema, Transaction, TRANSACTION_NAME}; -use crate::snapshot::Snapshot; -use crate::{DeltaResult, Engine}; - -pub use crate::actions::visitors::TransactionMap; - -pub struct TransactionScanner { - snapshot: Arc, -} - -impl TransactionScanner { - pub fn new(snapshot: Arc) -> Self { - TransactionScanner { snapshot } - } - - /// Scan the entire log for all application ids but terminate early if a specific application id is provided - fn scan_application_transactions( - &self, - engine: &dyn Engine, - application_id: Option<&str>, - ) -> DeltaResult { - let schema = get_log_schema().project(&[TRANSACTION_NAME])?; - - let mut visitor = TransactionVisitor::new(application_id.map(|s| s.to_owned())); - - // when all ids are requested then a full scan of the log to the latest checkpoint is required - let iter = - self.snapshot - .log_segment - .replay(engine, schema.clone(), schema.clone(), None)?; - - for maybe_data in iter { - let (txns, _) = maybe_data?; - txns.extract(schema.clone(), &mut visitor)?; - // if a specific id is requested and a transaction was found, then return - if application_id.is_some() && !visitor.transactions.is_empty() { - break; - } - } - - Ok(visitor.transactions) - } - - /// Scan the Delta Log for the latest transaction entry of an application - pub fn application_transaction( - &self, - engine: &dyn Engine, - application_id: &str, - ) -> DeltaResult> { - let mut transactions = self.scan_application_transactions(engine, Some(application_id))?; - Ok(transactions.remove(application_id)) - } - - /// Scan the Delta Log to obtain the latest transaction for all applications - pub fn application_transactions(&self, engine: &dyn Engine) -> DeltaResult { - self.scan_application_transactions(engine, None) - } -} - -#[cfg(all(test, feature = "default-engine"))] -mod tests { - use std::path::PathBuf; - - use super::*; - use crate::engine::sync::SyncEngine; - use crate::Table; - - fn get_latest_transactions(path: &str, app_id: &str) -> (TransactionMap, Option) { - let path = std::fs::canonicalize(PathBuf::from(path)).unwrap(); - let url = url::Url::from_directory_path(path).unwrap(); - let engine = SyncEngine::new(); - - let table = Table::new(url); - let snapshot = table.snapshot(&engine, None).unwrap(); - let txn_scan = TransactionScanner::new(snapshot.into()); - - ( - txn_scan.application_transactions(&engine).unwrap(), - txn_scan.application_transaction(&engine, app_id).unwrap(), - ) - } - - #[test] - fn test_txn() { - let (txns, txn) = get_latest_transactions("./tests/data/basic_partitioned/", "test"); - assert!(txn.is_none()); - assert_eq!(txns.len(), 0); - - let (txns, txn) = get_latest_transactions("./tests/data/app-txn-no-checkpoint/", "my-app"); - assert!(txn.is_some()); - assert_eq!(txns.len(), 2); - assert_eq!(txns.get("my-app"), txn.as_ref()); - assert_eq!( - txns.get("my-app2"), - Some(Transaction { - app_id: "my-app2".to_owned(), - version: 2, - last_updated: None - }) - .as_ref() - ); - - let (txns, txn) = get_latest_transactions("./tests/data/app-txn-checkpoint/", "my-app"); - assert!(txn.is_some()); - assert_eq!(txns.len(), 2); - assert_eq!(txns.get("my-app"), txn.as_ref()); - assert_eq!( - txns.get("my-app2"), - Some(Transaction { - app_id: "my-app2".to_owned(), - version: 2, - last_updated: None - }) - .as_ref() - ); - } -} From 2258549b763b19b7c1f08695626a1ac9ba611c36 Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Mon, 14 Oct 2024 11:45:50 -0700 Subject: [PATCH 38/66] revert arrow_expression and default/mod.rs --- kernel/src/engine/arrow_expression.rs | 8 ++--- kernel/src/engine/default/mod.rs | 43 ++++++++++++++------------- 2 files changed, 26 insertions(+), 25 deletions(-) diff --git a/kernel/src/engine/arrow_expression.rs b/kernel/src/engine/arrow_expression.rs index 04932a3e5..ffd5d42db 100644 --- a/kernel/src/engine/arrow_expression.rs +++ b/kernel/src/engine/arrow_expression.rs @@ -214,12 +214,12 @@ fn evaluate_expression( let output_fields: Vec = output_cols .iter() .zip(schema.fields()) - .map(|(output_col, input_field)| -> DeltaResult<_> { - ensure_data_types(input_field.data_type(), output_col.data_type())?; + .map(|(array, input_field)| -> DeltaResult<_> { + ensure_data_types(input_field.data_type(), array.data_type())?; Ok(ArrowField::new( input_field.name(), - output_col.data_type().clone(), - output_col.is_nullable(), + array.data_type().clone(), + array.is_nullable(), )) }) .try_collect()?; diff --git a/kernel/src/engine/default/mod.rs b/kernel/src/engine/default/mod.rs index 56d15ca86..5a8ef7af6 100644 --- a/kernel/src/engine/default/mod.rs +++ b/kernel/src/engine/default/mod.rs @@ -40,37 +40,38 @@ pub struct DefaultEngine { impl DefaultEngine { /// Create a new [`DefaultEngine`] instance /// - /// # Parameters + /// The `path` parameter is used to determine the type of storage used. /// - /// - `table_root`: The URL of the table within storage. - /// - `options`: key/value pairs of options to pass to the object store. - /// - `task_executor`: Used to spawn async IO tasks. See [executor::TaskExecutor]. - pub fn try_new( - table_root: &Url, - options: impl IntoIterator, - task_executor: Arc, - ) -> DeltaResult + /// The `task_executor` is used to spawn async IO tasks. See [executor::TaskExecutor]. + pub fn try_new(path: &Url, options: I, task_executor: Arc) -> DeltaResult where + I: IntoIterator, K: AsRef, V: Into, { - // table root is the path of the table in the ObjectStore - let (store, table_root) = parse_url_opts(table_root, options)?; - Ok(Self::new(Arc::new(store), table_root, task_executor)) + let (store, prefix) = parse_url_opts(path, options)?; + let store = Arc::new(store); + Ok(Self { + file_system: Arc::new(ObjectStoreFileSystemClient::new( + store.clone(), + prefix, + task_executor.clone(), + )), + json: Arc::new(DefaultJsonHandler::new( + store.clone(), + task_executor.clone(), + )), + parquet: Arc::new(DefaultParquetHandler::new(store.clone(), task_executor)), + store, + expression: Arc::new(ArrowExpressionHandler {}), + }) } - /// Create a new [`DefaultEngine`] instance - /// - /// # Parameters - /// - /// - `store`: The object store to use. - /// - `table_root_path`: The root path of the table within storage. - /// - `task_executor`: Used to spawn async IO tasks. See [executor::TaskExecutor]. - pub fn new(store: Arc, table_root: Path, task_executor: Arc) -> Self { + pub fn new(store: Arc, prefix: Path, task_executor: Arc) -> Self { Self { file_system: Arc::new(ObjectStoreFileSystemClient::new( store.clone(), - table_root, + prefix, task_executor.clone(), )), json: Arc::new(DefaultJsonHandler::new( From f463e22db4495366953a3ace0a64c1da469fd88c Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Mon, 14 Oct 2024 11:51:38 -0700 Subject: [PATCH 39/66] revert little spelling fix (in separate pr) --- kernel/src/expressions/scalars.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kernel/src/expressions/scalars.rs b/kernel/src/expressions/scalars.rs index 8c934aa3a..9e9e6ce75 100644 --- a/kernel/src/expressions/scalars.rs +++ b/kernel/src/expressions/scalars.rs @@ -67,7 +67,7 @@ impl StructData { require!( f.is_nullable() || !a.is_null(), Error::invalid_struct_data(format!( - "Value for non-nullable field {:?} cannot be null, got {}", + "Value for non-nullable field {:?} cannto be null, got {}", f.name(), a )) From 1149a17feaa6bdbb5ec01a4e02bf0187f8a5aa75 Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Mon, 14 Oct 2024 13:58:20 -0700 Subject: [PATCH 40/66] clean up some crate:: with use --- kernel/src/transaction.rs | 45 ++++++++++++++++++--------------------- 1 file changed, 21 insertions(+), 24 deletions(-) diff --git a/kernel/src/transaction.rs b/kernel/src/transaction.rs index 9067759db..8b98cf57e 100644 --- a/kernel/src/transaction.rs +++ b/kernel/src/transaction.rs @@ -1,11 +1,14 @@ use std::sync::Arc; use crate::actions::get_log_schema; +use crate::actions::{ + ADD_NAME, COMMIT_INFO_NAME, METADATA_NAME, PROTOCOL_NAME, REMOVE_NAME, SET_TRANSACTION_NAME, +}; +use crate::error::Error; use crate::path::ParsedLogPath; use crate::schema::{Schema, SchemaRef}; use crate::snapshot::Snapshot; -use crate::{DataType, Expression}; -use crate::{DeltaResult, Engine, EngineData}; +use crate::{DataType, DeltaResult, Engine, EngineData, Expression, Version}; const KERNEL_VERSION: &str = env!("CARGO_PKG_VERSION"); @@ -78,9 +81,7 @@ impl Transaction { let json_handler = engine.get_json_handler(); match json_handler.write_json_file(&commit_path.location, Box::new(actions), false) { Ok(()) => Ok(CommitResult::Committed(commit_version)), - Err(crate::error::Error::FileAlreadyExists(_)) => { - Ok(CommitResult::Conflict(self, commit_version)) - } + Err(Error::FileAlreadyExists(_)) => Ok(CommitResult::Conflict(self, commit_version)), Err(e) => Err(e), } } @@ -112,9 +113,9 @@ impl Transaction { /// (along with the version which conflicted). pub enum CommitResult { /// The transaction was successfully committed at the version. - Committed(crate::Version), + Committed(Version), /// The transaction conflicted with an existing version (at the version given). - Conflict(Transaction, crate::Version), + Conflict(Transaction, Version), } // given the engine's commit info (data and schema as [EngineCommitInfo]) we want to create both: @@ -124,12 +125,8 @@ fn generate_commit_info( engine: &dyn Engine, engine_commit_info: &EngineCommitInfo, ) -> DeltaResult<(Box, SchemaRef)> { - use crate::actions::{ - ADD_NAME, COMMIT_INFO_NAME, METADATA_NAME, PROTOCOL_NAME, REMOVE_NAME, SET_TRANSACTION_NAME, - }; - if engine_commit_info.data.length() != 1 { - return Err(crate::error::Error::InvalidCommitInfo(format!( + return Err(Error::InvalidCommitInfo(format!( "Engine commit info should have exactly one row, found {}", engine_commit_info.data.length() ))); @@ -139,11 +136,9 @@ fn generate_commit_info( let commit_info_field = action_schema .fields .get_mut(COMMIT_INFO_NAME) - .ok_or_else(|| crate::Error::missing_column(COMMIT_INFO_NAME))?; + .ok_or_else(|| Error::missing_column(COMMIT_INFO_NAME))?; let DataType::Struct(mut commit_info_data_type) = commit_info_field.data_type().clone() else { - return Err(crate::error::Error::internal_error( - "commit_info_field is a struct", - )); + return Err(Error::internal_error("commit_info_field is a struct")); }; commit_info_data_type.fields.extend( engine_commit_info @@ -198,34 +193,36 @@ fn generate_commit_info( #[cfg(test)] mod tests { use super::*; + use crate::engine::arrow_data::ArrowEngineData; + use crate::engine::arrow_expression::ArrowExpressionHandler; + use crate::{ExpressionHandler, FileSystemClient, JsonHandler, ParquetHandler}; + use arrow::array::{Int32Array, StringArray}; use arrow::record_batch::RecordBatch; - struct ExprEngine(Arc); + struct ExprEngine(Arc); impl ExprEngine { fn new() -> Self { - ExprEngine(Arc::new( - crate::engine::arrow_expression::ArrowExpressionHandler, - )) + ExprEngine(Arc::new(ArrowExpressionHandler)) } } impl Engine for ExprEngine { - fn get_expression_handler(&self) -> Arc { + fn get_expression_handler(&self) -> Arc { self.0.clone() } - fn get_json_handler(&self) -> Arc { + fn get_json_handler(&self) -> Arc { unimplemented!() } - fn get_parquet_handler(&self) -> Arc { + fn get_parquet_handler(&self) -> Arc { unimplemented!() } - fn get_file_system_client(&self) -> Arc { + fn get_file_system_client(&self) -> Arc { unimplemented!() } } From 3877cccf1e04429cee0f80d23c8e511dd4cabe9e Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Mon, 14 Oct 2024 14:03:59 -0700 Subject: [PATCH 41/66] cleanup --- kernel/src/engine/default/json.rs | 2 +- kernel/src/engine/sync/json.rs | 8 ++++---- kernel/tests/write.rs | 6 +++--- 3 files changed, 8 insertions(+), 8 deletions(-) diff --git a/kernel/src/engine/default/json.rs b/kernel/src/engine/default/json.rs index d0b9adf56..7b4a331be 100644 --- a/kernel/src/engine/default/json.rs +++ b/kernel/src/engine/default/json.rs @@ -111,7 +111,7 @@ impl JsonHandler for DefaultJsonHandler { }) .map_err(|e| match e { object_store::Error::AlreadyExists { .. } => { - crate::error::Error::FileAlreadyExists(path2.to_string()) + Error::FileAlreadyExists(path2.to_string()) } e => e.into(), })?; diff --git a/kernel/src/engine/sync/json.rs b/kernel/src/engine/sync/json.rs index e91ce180e..7d5a00a51 100644 --- a/kernel/src/engine/sync/json.rs +++ b/kernel/src/engine/sync/json.rs @@ -10,7 +10,7 @@ use crate::engine::arrow_utils::parse_json as arrow_parse_json; use crate::engine::arrow_utils::write_json; use crate::schema::SchemaRef; use crate::{ - DeltaResult, EngineData, Expression, FileDataReadResultIterator, FileMeta, JsonHandler, + DeltaResult, EngineData, Error, Expression, FileDataReadResultIterator, FileMeta, JsonHandler, }; pub(crate) struct SyncJsonHandler; @@ -77,9 +77,9 @@ impl JsonHandler for SyncJsonHandler { tempfile::PersistError { error, .. } if error.kind() == std::io::ErrorKind::AlreadyExists => { - crate::Error::FileAlreadyExists(path.to_string_lossy().to_string()) + Error::FileAlreadyExists(path.to_string_lossy().to_string()) } - e => crate::Error::IOError(e.into()), + e => Error::IOError(e.into()), })?; Ok(()) } @@ -123,7 +123,7 @@ mod tests { .expect("write json file"); assert!(matches!( handler.write_json_file(&url, Box::new(std::iter::once(empty)), false), - Err(crate::Error::FileAlreadyExists(_)) + Err(Error::FileAlreadyExists(_)) )); let file = std::fs::read_to_string(path)?; diff --git a/kernel/tests/write.rs b/kernel/tests/write.rs index 508c8a126..197d1d7be 100644 --- a/kernel/tests/write.rs +++ b/kernel/tests/write.rs @@ -14,7 +14,7 @@ use delta_kernel::engine::arrow_data::ArrowEngineData; use delta_kernel::engine::default::executor::tokio::TokioBackgroundExecutor; use delta_kernel::engine::default::DefaultEngine; use delta_kernel::schema::{DataType, SchemaRef, StructField, StructType}; -use delta_kernel::Table; +use delta_kernel::{Error as KernelError, Table}; // setup default engine with in-memory object store. fn setup( @@ -217,7 +217,7 @@ async fn test_invalid_commit_info() -> Result<(), Box> { // commit! assert!(matches!( txn.commit(&engine), - Err(delta_kernel::Error::InvalidCommitInfo(_)) + Err(KernelError::InvalidCommitInfo(_)) )); // two-row commit info test @@ -243,7 +243,7 @@ async fn test_invalid_commit_info() -> Result<(), Box> { // commit! assert!(matches!( txn.commit(&engine), - Err(delta_kernel::Error::InvalidCommitInfo(_)) + Err(KernelError::InvalidCommitInfo(_)) )); Ok(()) } From 3daed9bf4c9ede7a135a3563e471be2ee76712c8 Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Thu, 17 Oct 2024 21:38:35 -0700 Subject: [PATCH 42/66] it's getting close --- kernel/src/actions/mod.rs | 39 ++++- kernel/src/engine/arrow_expression.rs | 30 +++- kernel/src/transaction.rs | 235 +++++++++++++------------- kernel/tests/write.rs | 134 +++++++++------ 4 files changed, 258 insertions(+), 180 deletions(-) diff --git a/kernel/src/actions/mod.rs b/kernel/src/actions/mod.rs index 43b0f77fc..bf2f3d05d 100644 --- a/kernel/src/actions/mod.rs +++ b/kernel/src/actions/mod.rs @@ -137,8 +137,23 @@ impl Protocol { } #[derive(Debug, Clone, PartialEq, Eq, Schema)] -pub struct CommitInfo { - pub kernel_version: Option, +#[cfg_attr(feature = "developer-visibility", visibility::make(pub))] +#[cfg_attr(not(feature = "developer-visibility"), visibility::make(pub(crate)))] +struct CommitInfo { + /// The time this logical file was created, as milliseconds since the epoch. + /// TODO should this be a Timestamp? + pub(crate) timestamp: i64, + /// An arbitrary string that identifies the operation associated with this commit. This is + /// specified by the engine. + pub(crate) operation: String, + /// Map of arbitrary string key-value pairs that provide additional information about the + /// operation. This is specified by the engine. For now this is always empty. + pub(crate) operation_parameters: HashMap, + /// The version of the delta_kernel crate used to write this commit. + pub(crate) kernel_version: Option, + /// A place for the engine to store additional metadata associated with this commit encoded as + /// a map of strings. + pub(crate) engine_commit_info: HashMap, } #[derive(Debug, Clone, PartialEq, Eq, Schema)] @@ -426,11 +441,21 @@ mod tests { let expected = Arc::new(StructType::new(vec![StructField::new( "commitInfo", - StructType::new(vec![StructField::new( - "kernelVersion", - DataType::STRING, - true, - )]), + StructType::new(vec![ + StructField::new("timestamp", DataType::LONG, false), + StructField::new("operation", DataType::STRING, false), + StructField::new( + "operationParameters", + MapType::new(DataType::STRING, DataType::STRING, true), + false, + ), + StructField::new("kernelVersion", DataType::STRING, true), + StructField::new( + "engineCommitInfo", + MapType::new(DataType::STRING, DataType::STRING, true), + false, + ), + ]), true, )])); assert_eq!(schema, expected); diff --git a/kernel/src/engine/arrow_expression.rs b/kernel/src/engine/arrow_expression.rs index 93cc247ac..8b11420a7 100644 --- a/kernel/src/engine/arrow_expression.rs +++ b/kernel/src/engine/arrow_expression.rs @@ -26,7 +26,7 @@ use crate::engine::arrow_utils::ensure_data_types; use crate::engine::arrow_utils::prim_array_cmp; use crate::error::{DeltaResult, Error}; use crate::expressions::{BinaryOperator, Expression, Scalar, UnaryOperator, VariadicOperator}; -use crate::schema::{DataType, PrimitiveType, SchemaRef}; +use crate::schema::{DataType, MapType, PrimitiveType, SchemaRef}; use crate::{EngineData, ExpressionEvaluator, ExpressionHandler}; // TODO leverage scalars / Datum @@ -120,7 +120,33 @@ impl Scalar { ArrowField::new(LIST_ARRAY_ROOT, t.element_type().try_into()?, true); Arc::new(ListArray::new_null(Arc::new(field), num_rows)) } - DataType::Map { .. } => unimplemented!(), + DataType::Map(t) => { + let MapType { + key_type, + value_type, + .. // FIXME should use value_contains_null + } = t.as_ref(); + if key_type != &DataType::STRING && value_type != &DataType::STRING { + return Err(Error::generic("Only string key/values are supported")); + } + use arrow_array::builder::StringBuilder; + let key_builder = StringBuilder::new(); + let val_builder = StringBuilder::new(); + let names = arrow_array::builder::MapFieldNames { + entry: "entries".to_string(), + key: "key".to_string(), + value: "value".to_string(), + }; + let mut builder = arrow_array::builder::MapBuilder::new( + Some(names), + key_builder, + val_builder, + ); + (0..num_rows).for_each(|_| { + builder.append(true).unwrap(); + }); + Arc::new(builder.finish()) + } }, }; Ok(arr) diff --git a/kernel/src/transaction.rs b/kernel/src/transaction.rs index 8b98cf57e..210a984e2 100644 --- a/kernel/src/transaction.rs +++ b/kernel/src/transaction.rs @@ -1,29 +1,21 @@ use std::sync::Arc; +use std::time::{SystemTime, UNIX_EPOCH}; use crate::actions::get_log_schema; -use crate::actions::{ - ADD_NAME, COMMIT_INFO_NAME, METADATA_NAME, PROTOCOL_NAME, REMOVE_NAME, SET_TRANSACTION_NAME, -}; use crate::error::Error; use crate::path::ParsedLogPath; -use crate::schema::{Schema, SchemaRef}; +use crate::schema::{MapType, StructField, StructType}; use crate::snapshot::Snapshot; use crate::{DataType, DeltaResult, Engine, EngineData, Expression, Version}; const KERNEL_VERSION: &str = env!("CARGO_PKG_VERSION"); +const UNKNOWN_OPERATION: &str = "UNKNOWN"; /// A transaction represents an in-progress write to a table. pub struct Transaction { read_snapshot: Arc, - commit_info: Option, -} - -// Since the engine can include any commit info it likes, we unify the data/schema pair as a single -// struct with Arc semantics. -#[derive(Clone)] -struct EngineCommitInfo { - data: Arc, - schema: SchemaRef, + operation: Option, + commit_info: Option>, } impl std::fmt::Debug for Transaction { @@ -46,6 +38,7 @@ impl Transaction { pub(crate) fn new(snapshot: impl Into>) -> Self { Transaction { read_snapshot: snapshot.into(), + operation: None, commit_info: None, } } @@ -54,21 +47,15 @@ impl Transaction { pub fn commit(self, engine: &dyn Engine) -> DeltaResult { // step one: construct the iterator of actions we want to commit // note: only support commit_info right now. - let (actions, _actions_schema): ( - Box> + Send>, - SchemaRef, - ) = match self.commit_info { - Some(ref engine_commit_info) => { - let (actions, schema) = generate_commit_info(engine, engine_commit_info)?; - (Box::new(std::iter::once(actions)), schema) + let actions: Box> + Send> = match self.commit_info { + Some(engine_commit_info) => { + let actions = + generate_commit_info(engine, self.operation.as_deref(), engine_commit_info)?; + Box::new(std::iter::once(actions)) } None => { - ( - // if there is no commit info, actions are empty iterator and schema is just our - // known log schema. - Box::new(std::iter::empty()), - get_log_schema().clone().into(), - ) + // if there is no commit info, actions start empty + Box::new(std::iter::empty()) } }; @@ -81,13 +68,23 @@ impl Transaction { let json_handler = engine.get_json_handler(); match json_handler.write_json_file(&commit_path.location, Box::new(actions), false) { Ok(()) => Ok(CommitResult::Committed(commit_version)), - Err(Error::FileAlreadyExists(_)) => Ok(CommitResult::Conflict(self, commit_version)), + Err(Error::FileAlreadyExists(_)) => Err(Error::generic("fixme")), // Ok(CommitResult::Conflict(self, commit_version)), Err(e) => Err(e), } } + /// Set the operation that this transaction is performing. This string will be persisted in the + /// commitInfo action in the log. <- TODO include this bit? + pub fn operation(&mut self, operation: String) { + self.operation = Some(operation); + } + + /// WARNING: This is an unstable API and will likely change in the future. + /// /// Add commit info to the transaction. This is commit-wide metadata that is written as the - /// first action in the commit. + /// first action in the commit. The engine data passed here must have exactly one row, and we + /// only read one column: `engineCommitInfo` which must be a map encoding the + /// metadata. /// /// Note that there are two main pieces of information included in commit info: (1) custom /// engine commit info (specified via this API) and (2) delta's own commit info which is @@ -97,14 +94,11 @@ impl Transaction { /// 1. skip this API entirely - this will omit the commitInfo action from the commit (that is, /// it will prevent the kernel from writing delta-specific commitInfo). This precludes usage /// of table features which require commitInfo like in-commit timestamps. - /// 2. pass an empty commit_info data chunk - this will allow kernel to include delta-specific - /// commit info, resulting in a commitInfo action in the log, just without any - /// engine-specific additions. - pub fn commit_info(&mut self, commit_info: Box, schema: Schema) { - self.commit_info = Some(EngineCommitInfo { - data: commit_info.into(), - schema: schema.into(), - }); + /// 2. pass a commit_info data chunk with one row of `engineCommitInfo` with an empty map. This + /// allows kernel to include delta-specific commit info, resulting in a commitInfo action in + /// the log, just without any engine-specific additions. + pub fn commit_info(&mut self, commit_info: Box) { + self.commit_info = Some(commit_info); } } @@ -118,76 +112,54 @@ pub enum CommitResult { Conflict(Transaction, Version), } -// given the engine's commit info (data and schema as [EngineCommitInfo]) we want to create both: -// (1) the commitInfo action to commit (and append more actions to) and -// (2) the schema for the actions to commit (this is determined by the engine's commit info schema) +// given the engine's commit info we want to create commitInfo action to commit (and append more actions to) fn generate_commit_info( engine: &dyn Engine, - engine_commit_info: &EngineCommitInfo, -) -> DeltaResult<(Box, SchemaRef)> { - if engine_commit_info.data.length() != 1 { + operation: Option<&str>, + engine_commit_info: Box, +) -> DeltaResult> { + if engine_commit_info.length() != 1 { return Err(Error::InvalidCommitInfo(format!( "Engine commit info should have exactly one row, found {}", - engine_commit_info.data.length() + engine_commit_info.length() ))); } - let mut action_schema = get_log_schema().clone(); - let commit_info_field = action_schema - .fields - .get_mut(COMMIT_INFO_NAME) - .ok_or_else(|| Error::missing_column(COMMIT_INFO_NAME))?; - let DataType::Struct(mut commit_info_data_type) = commit_info_field.data_type().clone() else { - return Err(Error::internal_error("commit_info_field is a struct")); - }; - commit_info_data_type.fields.extend( - engine_commit_info - .schema - .fields() - .map(|f| (f.name.clone(), f.clone())), - ); - commit_info_field.data_type = DataType::Struct(commit_info_data_type); - - let commit_info_expr = std::iter::once(Expression::literal(format!("v{}", KERNEL_VERSION))) - .chain( - engine_commit_info - .schema - .fields() - .map(|f| Expression::column(f.name())), - ); - - // generate expression with null for all the fields except the commit_info field, and - // append the commit_info to the end. - let commit_info_expr_fields = [ - ADD_NAME, - REMOVE_NAME, - METADATA_NAME, - PROTOCOL_NAME, - SET_TRANSACTION_NAME, - ] - .iter() - .map(|name| { - Expression::null_literal( - action_schema - .field(name) - .expect("find field in action schema") - .data_type() - .clone(), - ) - }) - .chain(std::iter::once(Expression::struct_expr(commit_info_expr))); - let commit_info_expr = Expression::struct_expr(commit_info_expr_fields); + let commit_info_exprs = [ + // FIXME we should take a timestamp closer to commit time? + Expression::literal( + SystemTime::now() + .duration_since(UNIX_EPOCH) + .expect("time went backwards..?") + .as_millis() as i64, // FIXME safe cast + ), + Expression::literal(operation.unwrap_or(UNKNOWN_OPERATION)), + // Expression::struct_expr([Expression::null_literal(DataType::LONG)]), + Expression::null_literal(DataType::Map(Box::new(MapType::new( + DataType::STRING, + DataType::STRING, + true, + )))), + Expression::literal(format!("v{}", KERNEL_VERSION)), + Expression::column("engineCommitInfo"), + ]; + let commit_info_expr = Expression::struct_expr([Expression::struct_expr(commit_info_exprs)]); + // TODO probably just create a static + let commit_info_schema = get_log_schema().project_as_struct(&["commitInfo"])?; + + let engine_commit_info_schema = StructType::new(vec![StructField::new( + "engineCommitInfo", + MapType::new(DataType::STRING, DataType::STRING, true), + false, + )]); - // commit info has arbitrary schema ex: {engineInfo: string, operation: string} - // we want to bundle it up and put it in the commit_info field of the actions. let commit_info_evaluator = engine.get_expression_handler().get_evaluator( - engine_commit_info.schema.clone(), + engine_commit_info_schema.into(), commit_info_expr, - action_schema.clone().into(), + commit_info_schema.into(), ); - let actions = commit_info_evaluator.evaluate(engine_commit_info.data.as_ref())?; - Ok((actions, action_schema.into())) + commit_info_evaluator.evaluate(engine_commit_info.as_ref()) } #[cfg(test)] @@ -198,8 +170,9 @@ mod tests { use crate::engine::arrow_expression::ArrowExpressionHandler; use crate::{ExpressionHandler, FileSystemClient, JsonHandler, ParquetHandler}; - use arrow::array::{Int32Array, StringArray}; use arrow::record_batch::RecordBatch; + use arrow_schema::Schema as ArrowSchema; + use arrow_schema::{DataType as ArrowDataType, Field}; struct ExprEngine(Arc); @@ -229,34 +202,52 @@ mod tests { // simple test for generating commit info #[test] - fn test_generate_commit_info() { + fn test_generate_commit_info() -> DeltaResult<()> { let engine = ExprEngine::new(); - let schema = Arc::new(arrow_schema::Schema::new(vec![ - arrow_schema::Field::new("engine_info", arrow_schema::DataType::Utf8, true), - arrow_schema::Field::new("operation", arrow_schema::DataType::Utf8, true), - arrow_schema::Field::new("int", arrow_schema::DataType::Int32, true), - ])); - let data = RecordBatch::try_new( - schema.clone(), - vec![ - Arc::new(StringArray::from(vec!["test engine info"])), - Arc::new(StringArray::from(vec!["append"])), - Arc::new(Int32Array::from(vec![42])), - ], - ); - let engine_commit_info = EngineCommitInfo { - data: Arc::new(ArrowEngineData::new(data.unwrap())), - schema: Arc::new(schema.try_into().unwrap()), - }; - let (actions, actions_schema) = generate_commit_info(&engine, &engine_commit_info).unwrap(); - - // FIXME actual assertions - assert_eq!(actions_schema.fields().collect::>().len(), 6); - let DataType::Struct(struct_type) = actions_schema.field("commitInfo").unwrap().data_type() - else { - unreachable!("commitInfo is a struct"); + let engine_commit_info_schema = Arc::new(ArrowSchema::new(vec![Field::new( + "engineCommitInfo", + ArrowDataType::Map( + Arc::new(Field::new( + "entries", + ArrowDataType::Struct( + vec![ + Field::new("key", ArrowDataType::Utf8, false), + Field::new("value", ArrowDataType::Utf8, true), + ] + .into(), + ), + false, + )), + false, + ), + false, + )])); + + use arrow_array::builder::StringBuilder; + let key_builder = StringBuilder::new(); + let val_builder = StringBuilder::new(); + let names = arrow_array::builder::MapFieldNames { + entry: "entries".to_string(), + key: "key".to_string(), + value: "value".to_string(), }; - assert_eq!(struct_type.fields().collect::>().len(), 4); - assert_eq!(actions.length(), 1); + let mut builder = + arrow_array::builder::MapBuilder::new(Some(names), key_builder, val_builder); + builder.keys().append_value("engineInfo"); + builder.values().append_value("default engine"); + builder.append(true).unwrap(); + let array = builder.finish(); + + let commit_info_batch = + RecordBatch::try_new(engine_commit_info_schema, vec![Arc::new(array)])?; + + let actions = generate_commit_info( + &engine, + Some("test operation"), + Box::new(ArrowEngineData::new(commit_info_batch)), + )?; + + // TODO test it lol: more test cases, assert + Ok(()) } } diff --git a/kernel/tests/write.rs b/kernel/tests/write.rs index 197d1d7be..1535e7c0b 100644 --- a/kernel/tests/write.rs +++ b/kernel/tests/write.rs @@ -105,18 +105,41 @@ async fn test_commit_info() -> Result<(), Box> { // add commit info of the form {engineInfo: "default engine"} let commit_info_schema = Arc::new(ArrowSchema::new(vec![Field::new( - "engineInfo", - ArrowDataType::Utf8, - true, + "engineCommitInfo", + ArrowDataType::Map( + Arc::new(Field::new( + "entries", + ArrowDataType::Struct( + vec![ + Field::new("key", ArrowDataType::Utf8, false), + Field::new("value", ArrowDataType::Utf8, true), + ] + .into(), + ), + false, + )), + false, + ), + false, )])); - let commit_info_batch = RecordBatch::try_new( - commit_info_schema.clone(), - vec![Arc::new(StringArray::from(vec!["default engine"]))], - )?; - txn.commit_info( - Box::new(ArrowEngineData::new(commit_info_batch)), - commit_info_schema.try_into()?, - ); + + use arrow_array::builder::StringBuilder; + let key_builder = StringBuilder::new(); + let val_builder = StringBuilder::new(); + let names = arrow_array::builder::MapFieldNames { + entry: "entries".to_string(), + key: "key".to_string(), + value: "value".to_string(), + }; + let mut builder = arrow_array::builder::MapBuilder::new(Some(names), key_builder, val_builder); + builder.keys().append_value("engineInfo"); + builder.values().append_value("default engine"); + builder.append(true).unwrap(); + let array = builder.finish(); + + let commit_info_batch = + RecordBatch::try_new(commit_info_schema.clone(), vec![Arc::new(array)])?; + txn.commit_info(Box::new(ArrowEngineData::new(commit_info_batch))); // commit! txn.commit(&engine)?; @@ -126,42 +149,61 @@ async fn test_commit_info() -> Result<(), Box> { "/test_table/_delta_log/00000000000000000001.json", )) .await?; - assert_eq!( - String::from_utf8(commit1.bytes().await?.to_vec())?, - "{\"commitInfo\":{\"kernelVersion\":\"v0.3.1\",\"engineInfo\":\"default engine\"}}\n" - ); - // one null row commit info - let mut txn = table.new_transaction(&engine)?; - let commit_info_schema = Arc::new(ArrowSchema::new(vec![Field::new( - "some_column_name", - ArrowDataType::Utf8, - true, - )])); - let commit_info_batch = RecordBatch::try_new( - commit_info_schema.clone(), - vec![Arc::new(StringArray::new_null(1))], - )?; - txn.commit_info( - Box::new(ArrowEngineData::new(commit_info_batch)), - commit_info_schema.try_into()?, - ); + let mut parsed_commit: serde_json::Value = serde_json::from_slice(&commit1.bytes().await?)?; + *parsed_commit + .get_mut("commitInfo") + .unwrap() + .get_mut("timestamp") + .unwrap() = serde_json::Value::Number(0.into()); - // commit! - txn.commit(&engine)?; + let expected_commit = json!({ + "commitInfo": { + "timestamp": 0, + "operation": "UNKNOWN", + "kernelVersion": format!("v{}", env!("CARGO_PKG_VERSION")), + "operationParameters": {}, + "engineCommitInfo": { + "engineInfo": "default engine" + } + } + }); - let commit1 = store - .get(&Path::from( - "/test_table/_delta_log/00000000000000000002.json", - )) - .await?; - assert_eq!( - String::from_utf8(commit1.bytes().await?.to_vec())?, - "{\"commitInfo\":{\"kernelVersion\":\"v0.3.1\"}}\n" - ); + assert_eq!(parsed_commit, expected_commit); + + //// one null row commit info + //let mut txn = table.new_transaction(&engine)?; + //let commit_info_schema = Arc::new(ArrowSchema::new(vec![Field::new( + // "some_column_name", + // ArrowDataType::Utf8, + // true, + //)])); + //let commit_info_batch = RecordBatch::try_new( + // commit_info_schema.clone(), + // vec![Arc::new(StringArray::new_null(1))], + //)?; + //txn.commit_info(Box::new(ArrowEngineData::new(commit_info_batch))); + // + //// commit! + //txn.commit(&engine)?; + // + //let commit1 = store + // .get(&Path::from( + // "/test_table/_delta_log/00000000000000000002.json", + // )) + // .await?; + //assert_eq!( + // String::from_utf8(commit1.bytes().await?.to_vec())?, + // "{\"commitInfo\":{\"kernelVersion\":\"v0.3.1\"}}\n" + //); Ok(()) } +// need to test various invalid commit infos +// 1. missing engineCommitInfo +// 2. invalid engineCommitInfo type +// 3. other columns that could override (like operation) + #[tokio::test] async fn test_empty_commit() -> Result<(), Box> { // setup tracing @@ -209,10 +251,7 @@ async fn test_invalid_commit_info() -> Result<(), Box> { let commit_info_schema = Arc::new(ArrowSchema::empty()); let commit_info_batch = RecordBatch::new_empty(commit_info_schema.clone()); assert!(commit_info_batch.num_rows() == 0); - txn.commit_info( - Box::new(ArrowEngineData::new(commit_info_batch)), - commit_info_schema.try_into()?, - ); + txn.commit_info(Box::new(ArrowEngineData::new(commit_info_batch))); // commit! assert!(matches!( @@ -235,10 +274,7 @@ async fn test_invalid_commit_info() -> Result<(), Box> { ]))], )?; - txn.commit_info( - Box::new(ArrowEngineData::new(commit_info_batch)), - commit_info_schema.try_into()?, - ); + txn.commit_info(Box::new(ArrowEngineData::new(commit_info_batch))); // commit! assert!(matches!( From 327bbdea87d8d9cac43dd50695e8c7030946e1e2 Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Thu, 17 Oct 2024 21:43:59 -0700 Subject: [PATCH 43/66] have i done it? --- kernel/src/actions/mod.rs | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/kernel/src/actions/mod.rs b/kernel/src/actions/mod.rs index bf2f3d05d..c9360bc39 100644 --- a/kernel/src/actions/mod.rs +++ b/kernel/src/actions/mod.rs @@ -139,6 +139,7 @@ impl Protocol { #[derive(Debug, Clone, PartialEq, Eq, Schema)] #[cfg_attr(feature = "developer-visibility", visibility::make(pub))] #[cfg_attr(not(feature = "developer-visibility"), visibility::make(pub(crate)))] +// TODO need to have a way to always write some fields but not always read them struct CommitInfo { /// The time this logical file was created, as milliseconds since the epoch. /// TODO should this be a Timestamp? @@ -153,7 +154,8 @@ struct CommitInfo { pub(crate) kernel_version: Option, /// A place for the engine to store additional metadata associated with this commit encoded as /// a map of strings. - pub(crate) engine_commit_info: HashMap, + /// TODO need to have a way to always write this but not always read it + pub(crate) engine_commit_info: Option>, } #[derive(Debug, Clone, PartialEq, Eq, Schema)] @@ -446,14 +448,14 @@ mod tests { StructField::new("operation", DataType::STRING, false), StructField::new( "operationParameters", - MapType::new(DataType::STRING, DataType::STRING, true), + MapType::new(DataType::STRING, DataType::STRING, false), false, ), StructField::new("kernelVersion", DataType::STRING, true), StructField::new( "engineCommitInfo", - MapType::new(DataType::STRING, DataType::STRING, true), - false, + MapType::new(DataType::STRING, DataType::STRING, false), + true, ), ]), true, From 6d2b41a16516e909f039006b011d62fe2b68bcdd Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Mon, 21 Oct 2024 16:31:31 -0700 Subject: [PATCH 44/66] wip --- kernel/src/engine/arrow_expression.rs | 61 +++++++++++++++++++++----- kernel/src/error.rs | 4 ++ kernel/src/transaction.rs | 63 ++++++++++++++------------- kernel/tests/write.rs | 12 ++--- 4 files changed, 91 insertions(+), 49 deletions(-) diff --git a/kernel/src/engine/arrow_expression.rs b/kernel/src/engine/arrow_expression.rs index 8b11420a7..917ed4f0b 100644 --- a/kernel/src/engine/arrow_expression.rs +++ b/kernel/src/engine/arrow_expression.rs @@ -20,7 +20,9 @@ use arrow_schema::{ use arrow_select::concat::concat; use itertools::Itertools; -use super::arrow_conversion::LIST_ARRAY_ROOT; +use super::arrow_conversion::{ + LIST_ARRAY_ROOT, MAP_KEY_DEFAULT, MAP_ROOT_DEFAULT, MAP_VALUE_DEFAULT, +}; use crate::engine::arrow_data::ArrowEngineData; use crate::engine::arrow_utils::ensure_data_types; use crate::engine::arrow_utils::prim_array_cmp; @@ -126,25 +128,25 @@ impl Scalar { value_type, .. // FIXME should use value_contains_null } = t.as_ref(); - if key_type != &DataType::STRING && value_type != &DataType::STRING { + if key_type != &DataType::STRING || value_type != &DataType::STRING { return Err(Error::generic("Only string key/values are supported")); } use arrow_array::builder::StringBuilder; let key_builder = StringBuilder::new(); let val_builder = StringBuilder::new(); let names = arrow_array::builder::MapFieldNames { - entry: "entries".to_string(), - key: "key".to_string(), - value: "value".to_string(), + entry: MAP_ROOT_DEFAULT.to_string(), + key: MAP_KEY_DEFAULT.to_string(), + value: MAP_VALUE_DEFAULT.to_string(), }; let mut builder = arrow_array::builder::MapBuilder::new( Some(names), key_builder, val_builder, ); - (0..num_rows).for_each(|_| { - builder.append(true).unwrap(); - }); + std::iter::repeat_with(|| builder.append(true)) + .take(num_rows) + .collect::>()?; Arc::new(builder.finish()) } }, @@ -459,7 +461,7 @@ mod tests { use super::*; use crate::expressions::*; use crate::schema::ArrayType; - use crate::DataType as DeltaDataTypes; + use crate::DataType as DeltaDataType; #[test] fn test_array_column() { @@ -526,7 +528,7 @@ mod tests { BinaryOperator::NotIn, Expression::literal(5), Expression::literal(Scalar::Array(ArrayData::new( - ArrayType::new(DeltaDataTypes::INTEGER, false), + ArrayType::new(DeltaDataType::INTEGER, false), vec![Scalar::Integer(1), Scalar::Integer(2)], ))), ); @@ -776,4 +778,43 @@ mod tests { let expected = Arc::new(BooleanArray::from(vec![true, false])); assert_eq!(results.as_ref(), expected.as_ref()); } + + #[test] + fn test_null_map() { + let schema = Schema::new(vec![Field::new("to_become_map", DataType::Int64, false)]); + let batch = RecordBatch::try_new( + Arc::new(schema.clone()), + vec![Arc::new(Int64Array::from(vec![0, 1]))], + ) + .unwrap(); + let expr = Expression::null_literal(DeltaDataType::Map(Box::new(MapType::new( + DeltaDataType::STRING, + DeltaDataType::STRING, + true, + )))); + let output_schema = crate::schema::DataType::Map(Box::new(MapType::new( + DeltaDataType::STRING, + DeltaDataType::STRING, + true, + ))); + let results = evaluate_expression(&expr, &batch, Some(&output_schema)).unwrap(); + let keys = results.as_map().keys(); + let values = results.as_map().values(); + assert_eq!(keys.as_ref(), &StringArray::new_null(0)); + assert_eq!(values.as_ref(), &StringArray::new_null(0)); + + // also check we only support string keys and values + let expr = Expression::null_literal(DeltaDataType::Map(Box::new(MapType::new( + DeltaDataType::STRING, + DeltaDataType::INTEGER, + true, + )))); + evaluate_expression(&expr, &batch, Some(&output_schema)).unwrap_err(); + let expr = Expression::null_literal(DeltaDataType::Map(Box::new(MapType::new( + DeltaDataType::INTEGER, + DeltaDataType::STRING, + true, + )))); + evaluate_expression(&expr, &batch, Some(&output_schema)).unwrap_err(); + } } diff --git a/kernel/src/error.rs b/kernel/src/error.rs index 5458302bd..b247d0850 100644 --- a/kernel/src/error.rs +++ b/kernel/src/error.rs @@ -164,6 +164,10 @@ pub enum Error { #[error("Invalid commit info: {0}")] InvalidCommitInfo(String), + /// Commit info was not passed to the transaction + #[error("Missing commit info")] + MissingCommitInfo, + /// The file already exists at the path, prohibiting a non-overwrite write #[error("File already exists: {0}")] FileAlreadyExists(String), diff --git a/kernel/src/transaction.rs b/kernel/src/transaction.rs index 210a984e2..34976178a 100644 --- a/kernel/src/transaction.rs +++ b/kernel/src/transaction.rs @@ -1,3 +1,4 @@ +use std::iter; use std::sync::Arc; use std::time::{SystemTime, UNIX_EPOCH}; @@ -15,7 +16,7 @@ const UNKNOWN_OPERATION: &str = "UNKNOWN"; pub struct Transaction { read_snapshot: Arc, operation: Option, - commit_info: Option>, + commit_info: Option>, } impl std::fmt::Debug for Transaction { @@ -46,18 +47,16 @@ impl Transaction { /// Consume the transaction and commit the in-progress write to the table. pub fn commit(self, engine: &dyn Engine) -> DeltaResult { // step one: construct the iterator of actions we want to commit - // note: only support commit_info right now. - let actions: Box> + Send> = match self.commit_info { - Some(engine_commit_info) => { - let actions = - generate_commit_info(engine, self.operation.as_deref(), engine_commit_info)?; - Box::new(std::iter::once(actions)) - } - None => { - // if there is no commit info, actions start empty - Box::new(std::iter::empty()) - } - }; + // note: only support commit_info right now (and it's required) + let engine_commit_info = self + .commit_info + .clone() + .ok_or_else(|| Error::MissingCommitInfo)?; + let actions = Box::new(iter::once(generate_commit_info( + engine, + self.operation.as_deref(), + engine_commit_info, + )?)); // step two: set new commit version (current_version + 1) and path to write let commit_version = self.read_snapshot.version() + 1; @@ -68,13 +67,13 @@ impl Transaction { let json_handler = engine.get_json_handler(); match json_handler.write_json_file(&commit_path.location, Box::new(actions), false) { Ok(()) => Ok(CommitResult::Committed(commit_version)), - Err(Error::FileAlreadyExists(_)) => Err(Error::generic("fixme")), // Ok(CommitResult::Conflict(self, commit_version)), + Err(Error::FileAlreadyExists(_)) => Ok(CommitResult::Conflict(self, commit_version)), Err(e) => Err(e), } } /// Set the operation that this transaction is performing. This string will be persisted in the - /// commitInfo action in the log. <- TODO include this bit? + /// commit and visible to anyone who describes the table history. pub fn operation(&mut self, operation: String) { self.operation = Some(operation); } @@ -86,25 +85,23 @@ impl Transaction { /// only read one column: `engineCommitInfo` which must be a map encoding the /// metadata. /// - /// Note that there are two main pieces of information included in commit info: (1) custom - /// engine commit info (specified via this API) and (2) delta's own commit info which is - /// effectively appended to the engine-specific commit info. - /// - /// If the engine elects to omit commit info, it can do so in two ways: - /// 1. skip this API entirely - this will omit the commitInfo action from the commit (that is, - /// it will prevent the kernel from writing delta-specific commitInfo). This precludes usage - /// of table features which require commitInfo like in-commit timestamps. - /// 2. pass a commit_info data chunk with one row of `engineCommitInfo` with an empty map. This - /// allows kernel to include delta-specific commit info, resulting in a commitInfo action in - /// the log, just without any engine-specific additions. + /// The engine is required to provide commit info before committing the transaction. If the + /// engine would like to omit engine-specific commit info, it can do so by passing pass a + /// commit_info engine data chunk with one row and one column of: + /// 1. `engineCommitInfo` column with an empty Map + /// 2. `engineCommitInfo` null column of type Map + /// 3. a column that has a name other than `engineCommitInfo`; Delta can detect that the column + /// is missing and substitute a null literal in its place. The type of that column doesn't + /// matter, Delta will ignore it. pub fn commit_info(&mut self, commit_info: Box) { - self.commit_info = Some(commit_info); + self.commit_info = Some(commit_info.into()); } } /// Result after committing a transaction. If 'committed', the version is the new version written /// to the log. If 'conflict', the transaction is returned so the caller can resolve the conflict /// (along with the version which conflicted). +#[derive(Debug)] pub enum CommitResult { /// The transaction was successfully committed at the version. Committed(Version), @@ -116,7 +113,7 @@ pub enum CommitResult { fn generate_commit_info( engine: &dyn Engine, operation: Option<&str>, - engine_commit_info: Box, + engine_commit_info: Arc, ) -> DeltaResult> { if engine_commit_info.length() != 1 { return Err(Error::InvalidCommitInfo(format!( @@ -134,7 +131,6 @@ fn generate_commit_info( .as_millis() as i64, // FIXME safe cast ), Expression::literal(operation.unwrap_or(UNKNOWN_OPERATION)), - // Expression::struct_expr([Expression::null_literal(DataType::LONG)]), Expression::null_literal(DataType::Map(Box::new(MapType::new( DataType::STRING, DataType::STRING, @@ -244,10 +240,15 @@ mod tests { let actions = generate_commit_info( &engine, Some("test operation"), - Box::new(ArrowEngineData::new(commit_info_batch)), + Arc::new(ArrowEngineData::new(commit_info_batch)), )?; - // TODO test it lol: more test cases, assert + assert_eq!(actions.length(), 1); + let record_batch: RecordBatch = actions + .into_any() + .downcast::() + .unwrap() + .into(); Ok(()) } } diff --git a/kernel/tests/write.rs b/kernel/tests/write.rs index 1535e7c0b..d51d4e18b 100644 --- a/kernel/tests/write.rs +++ b/kernel/tests/write.rs @@ -219,15 +219,11 @@ async fn test_empty_commit() -> Result<(), Box> { )])); let table = create_table(store.clone(), table_location, schema, &[]).await?; - // create a transaction and commit - table.new_transaction(&engine)?.commit(&engine)?; + assert!(matches!( + table.new_transaction(&engine)?.commit(&engine).unwrap_err(), + KernelError::MissingCommitInfo + )); - let commit1 = store - .get(&Path::from( - "/test_table/_delta_log/00000000000000000001.json", - )) - .await?; - assert!(commit1.bytes().await?.to_vec().is_empty()); Ok(()) } From 0abd29114f50a3f897c98c2ad2d04c9a4f6fe7eb Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Mon, 21 Oct 2024 16:35:40 -0700 Subject: [PATCH 45/66] remove my wrong null_literal for map lol rip --- kernel/src/engine/arrow_expression.rs | 93 ++++----------------------- 1 file changed, 13 insertions(+), 80 deletions(-) diff --git a/kernel/src/engine/arrow_expression.rs b/kernel/src/engine/arrow_expression.rs index 917ed4f0b..ffd5d42db 100644 --- a/kernel/src/engine/arrow_expression.rs +++ b/kernel/src/engine/arrow_expression.rs @@ -20,15 +20,13 @@ use arrow_schema::{ use arrow_select::concat::concat; use itertools::Itertools; -use super::arrow_conversion::{ - LIST_ARRAY_ROOT, MAP_KEY_DEFAULT, MAP_ROOT_DEFAULT, MAP_VALUE_DEFAULT, -}; +use super::arrow_conversion::LIST_ARRAY_ROOT; use crate::engine::arrow_data::ArrowEngineData; use crate::engine::arrow_utils::ensure_data_types; use crate::engine::arrow_utils::prim_array_cmp; use crate::error::{DeltaResult, Error}; use crate::expressions::{BinaryOperator, Expression, Scalar, UnaryOperator, VariadicOperator}; -use crate::schema::{DataType, MapType, PrimitiveType, SchemaRef}; +use crate::schema::{DataType, PrimitiveType, SchemaRef}; use crate::{EngineData, ExpressionEvaluator, ExpressionHandler}; // TODO leverage scalars / Datum @@ -122,33 +120,7 @@ impl Scalar { ArrowField::new(LIST_ARRAY_ROOT, t.element_type().try_into()?, true); Arc::new(ListArray::new_null(Arc::new(field), num_rows)) } - DataType::Map(t) => { - let MapType { - key_type, - value_type, - .. // FIXME should use value_contains_null - } = t.as_ref(); - if key_type != &DataType::STRING || value_type != &DataType::STRING { - return Err(Error::generic("Only string key/values are supported")); - } - use arrow_array::builder::StringBuilder; - let key_builder = StringBuilder::new(); - let val_builder = StringBuilder::new(); - let names = arrow_array::builder::MapFieldNames { - entry: MAP_ROOT_DEFAULT.to_string(), - key: MAP_KEY_DEFAULT.to_string(), - value: MAP_VALUE_DEFAULT.to_string(), - }; - let mut builder = arrow_array::builder::MapBuilder::new( - Some(names), - key_builder, - val_builder, - ); - std::iter::repeat_with(|| builder.append(true)) - .take(num_rows) - .collect::>()?; - Arc::new(builder.finish()) - } + DataType::Map { .. } => unimplemented!(), }, }; Ok(arr) @@ -233,21 +205,21 @@ fn evaluate_expression( .cloned() } } - (Struct(fields), Some(DataType::Struct(output_schema))) => { + (Struct(fields), Some(DataType::Struct(schema))) => { let columns = fields .iter() - .zip(output_schema.fields()) + .zip(schema.fields()) .map(|(expr, field)| evaluate_expression(expr, batch, Some(field.data_type()))); let output_cols: Vec> = columns.try_collect()?; let output_fields: Vec = output_cols .iter() - .zip(output_schema.fields()) - .map(|(output_col, output_field)| -> DeltaResult<_> { - ensure_data_types(output_field.data_type(), output_col.data_type())?; + .zip(schema.fields()) + .map(|(array, input_field)| -> DeltaResult<_> { + ensure_data_types(input_field.data_type(), array.data_type())?; Ok(ArrowField::new( - output_field.name(), - output_col.data_type().clone(), - output_col.is_nullable(), + input_field.name(), + array.data_type().clone(), + array.is_nullable(), )) }) .try_collect()?; @@ -461,7 +433,7 @@ mod tests { use super::*; use crate::expressions::*; use crate::schema::ArrayType; - use crate::DataType as DeltaDataType; + use crate::DataType as DeltaDataTypes; #[test] fn test_array_column() { @@ -528,7 +500,7 @@ mod tests { BinaryOperator::NotIn, Expression::literal(5), Expression::literal(Scalar::Array(ArrayData::new( - ArrayType::new(DeltaDataType::INTEGER, false), + ArrayType::new(DeltaDataTypes::INTEGER, false), vec![Scalar::Integer(1), Scalar::Integer(2)], ))), ); @@ -778,43 +750,4 @@ mod tests { let expected = Arc::new(BooleanArray::from(vec![true, false])); assert_eq!(results.as_ref(), expected.as_ref()); } - - #[test] - fn test_null_map() { - let schema = Schema::new(vec![Field::new("to_become_map", DataType::Int64, false)]); - let batch = RecordBatch::try_new( - Arc::new(schema.clone()), - vec![Arc::new(Int64Array::from(vec![0, 1]))], - ) - .unwrap(); - let expr = Expression::null_literal(DeltaDataType::Map(Box::new(MapType::new( - DeltaDataType::STRING, - DeltaDataType::STRING, - true, - )))); - let output_schema = crate::schema::DataType::Map(Box::new(MapType::new( - DeltaDataType::STRING, - DeltaDataType::STRING, - true, - ))); - let results = evaluate_expression(&expr, &batch, Some(&output_schema)).unwrap(); - let keys = results.as_map().keys(); - let values = results.as_map().values(); - assert_eq!(keys.as_ref(), &StringArray::new_null(0)); - assert_eq!(values.as_ref(), &StringArray::new_null(0)); - - // also check we only support string keys and values - let expr = Expression::null_literal(DeltaDataType::Map(Box::new(MapType::new( - DeltaDataType::STRING, - DeltaDataType::INTEGER, - true, - )))); - evaluate_expression(&expr, &batch, Some(&output_schema)).unwrap_err(); - let expr = Expression::null_literal(DeltaDataType::Map(Box::new(MapType::new( - DeltaDataType::INTEGER, - DeltaDataType::STRING, - true, - )))); - evaluate_expression(&expr, &batch, Some(&output_schema)).unwrap_err(); - } } From b793523bf2f61ba9b94a0db29392a3b7c3daf8ce Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Mon, 21 Oct 2024 19:32:35 -0700 Subject: [PATCH 46/66] back to using empty struct for operationParameters --- kernel/src/actions/mod.rs | 17 ++++++++++------- kernel/src/transaction.rs | 23 ++++++++++++----------- 2 files changed, 22 insertions(+), 18 deletions(-) diff --git a/kernel/src/actions/mod.rs b/kernel/src/actions/mod.rs index c9360bc39..ae0a42e66 100644 --- a/kernel/src/actions/mod.rs +++ b/kernel/src/actions/mod.rs @@ -136,6 +136,9 @@ impl Protocol { } } +#[derive(Debug, Clone, PartialEq, Eq, Schema)] +struct OperationParameters {} + #[derive(Debug, Clone, PartialEq, Eq, Schema)] #[cfg_attr(feature = "developer-visibility", visibility::make(pub))] #[cfg_attr(not(feature = "developer-visibility"), visibility::make(pub(crate)))] @@ -148,8 +151,12 @@ struct CommitInfo { /// specified by the engine. pub(crate) operation: String, /// Map of arbitrary string key-value pairs that provide additional information about the - /// operation. This is specified by the engine. For now this is always empty. - pub(crate) operation_parameters: HashMap, + /// operation. This is specified by the engine. + /// + /// For now this is always empty; and since we don't have the ability to construct an empty + /// string-string map, we spoof the operation_parameters with an empty struct so it serializes + /// the same as an empty map (as `{}`). + operation_parameters: OperationParameters, /// The version of the delta_kernel crate used to write this commit. pub(crate) kernel_version: Option, /// A place for the engine to store additional metadata associated with this commit encoded as @@ -446,11 +453,7 @@ mod tests { StructType::new(vec![ StructField::new("timestamp", DataType::LONG, false), StructField::new("operation", DataType::STRING, false), - StructField::new( - "operationParameters", - MapType::new(DataType::STRING, DataType::STRING, false), - false, - ), + StructField::new("operationParameters", StructType::new([]), false), StructField::new("kernelVersion", DataType::STRING, true), StructField::new( "engineCommitInfo", diff --git a/kernel/src/transaction.rs b/kernel/src/transaction.rs index 34976178a..84365f368 100644 --- a/kernel/src/transaction.rs +++ b/kernel/src/transaction.rs @@ -4,6 +4,7 @@ use std::time::{SystemTime, UNIX_EPOCH}; use crate::actions::get_log_schema; use crate::error::Error; +use crate::expressions::{Scalar, StructData}; use crate::path::ParsedLogPath; use crate::schema::{MapType, StructField, StructType}; use crate::snapshot::Snapshot; @@ -122,20 +123,20 @@ fn generate_commit_info( ))); } + let timestamp: i64 = SystemTime::now() + .duration_since(UNIX_EPOCH) + .expect("time went backwards..?") + .as_millis() + .try_into() + .map_err(|_| Error::generic("milliseconds since unix_epoch exceeded i64 size"))?; let commit_info_exprs = [ // FIXME we should take a timestamp closer to commit time? - Expression::literal( - SystemTime::now() - .duration_since(UNIX_EPOCH) - .expect("time went backwards..?") - .as_millis() as i64, // FIXME safe cast - ), + Expression::literal(timestamp), Expression::literal(operation.unwrap_or(UNKNOWN_OPERATION)), - Expression::null_literal(DataType::Map(Box::new(MapType::new( - DataType::STRING, - DataType::STRING, - true, - )))), + Expression::literal(Scalar::Struct(StructData::try_new( + vec![StructField::new("idk", DataType::INTEGER, true)], + vec![Scalar::Null(DataType::INTEGER)], + )?)), Expression::literal(format!("v{}", KERNEL_VERSION)), Expression::column("engineCommitInfo"), ]; From 2f4e4d02b8af7c2f850a7e8420e05f2b14ff30e5 Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Tue, 22 Oct 2024 13:54:11 -0700 Subject: [PATCH 47/66] comment --- kernel/src/actions/mod.rs | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/kernel/src/actions/mod.rs b/kernel/src/actions/mod.rs index ae0a42e66..7165de784 100644 --- a/kernel/src/actions/mod.rs +++ b/kernel/src/actions/mod.rs @@ -142,10 +142,8 @@ struct OperationParameters {} #[derive(Debug, Clone, PartialEq, Eq, Schema)] #[cfg_attr(feature = "developer-visibility", visibility::make(pub))] #[cfg_attr(not(feature = "developer-visibility"), visibility::make(pub(crate)))] -// TODO need to have a way to always write some fields but not always read them struct CommitInfo { /// The time this logical file was created, as milliseconds since the epoch. - /// TODO should this be a Timestamp? pub(crate) timestamp: i64, /// An arbitrary string that identifies the operation associated with this commit. This is /// specified by the engine. @@ -157,11 +155,12 @@ struct CommitInfo { /// string-string map, we spoof the operation_parameters with an empty struct so it serializes /// the same as an empty map (as `{}`). operation_parameters: OperationParameters, - /// The version of the delta_kernel crate used to write this commit. + /// The version of the delta_kernel crate used to write this commit. The kernel will always + /// write this field, but it is optional since many tables will not have this field (i.e. any + /// tables not written by kernel). pub(crate) kernel_version: Option, /// A place for the engine to store additional metadata associated with this commit encoded as /// a map of strings. - /// TODO need to have a way to always write this but not always read it pub(crate) engine_commit_info: Option>, } From 673af96b83cfe1fc27a38d320259d36c5351b19e Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Tue, 22 Oct 2024 14:02:18 -0700 Subject: [PATCH 48/66] wip need to fix commit info operationParameters --- kernel/src/engine/arrow_expression.rs | 8 ++++---- kernel/src/transaction.rs | 7 ++----- 2 files changed, 6 insertions(+), 9 deletions(-) diff --git a/kernel/src/engine/arrow_expression.rs b/kernel/src/engine/arrow_expression.rs index a8830f1ea..fa87cb8bb 100644 --- a/kernel/src/engine/arrow_expression.rs +++ b/kernel/src/engine/arrow_expression.rs @@ -202,7 +202,7 @@ fn evaluate_expression( (Struct(fields), Some(DataType::Struct(output_schema))) => { let columns = fields .iter() - .zip(schema.fields()) + .zip(output_schema.fields()) .map(|(expr, field)| evaluate_expression(expr, batch, Some(field.data_type()))); let output_cols: Vec = columns.try_collect()?; let output_fields: Vec = output_cols @@ -210,9 +210,9 @@ fn evaluate_expression( .zip(output_schema.fields()) .map(|(output_col, output_field)| -> DeltaResult<_> { Ok(ArrowField::new( - input_field.name(), - array.data_type().clone(), - array.is_nullable(), + output_field.name(), + output_col.data_type().clone(), + output_col.is_nullable(), )) }) .try_collect()?; diff --git a/kernel/src/transaction.rs b/kernel/src/transaction.rs index 84365f368..5b219d889 100644 --- a/kernel/src/transaction.rs +++ b/kernel/src/transaction.rs @@ -133,14 +133,11 @@ fn generate_commit_info( // FIXME we should take a timestamp closer to commit time? Expression::literal(timestamp), Expression::literal(operation.unwrap_or(UNKNOWN_OPERATION)), - Expression::literal(Scalar::Struct(StructData::try_new( - vec![StructField::new("idk", DataType::INTEGER, true)], - vec![Scalar::Null(DataType::INTEGER)], - )?)), + Expression::literal(Scalar::Struct(StructData::try_new(vec![], vec![])?)), Expression::literal(format!("v{}", KERNEL_VERSION)), Expression::column("engineCommitInfo"), ]; - let commit_info_expr = Expression::struct_expr([Expression::struct_expr(commit_info_exprs)]); + let commit_info_expr = Expression::struct_from([Expression::struct_from(commit_info_exprs)]); // TODO probably just create a static let commit_info_schema = get_log_schema().project_as_struct(&["commitInfo"])?; From 559bbea0f29b322778dbe3cce6320dd9e78379c1 Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Tue, 22 Oct 2024 14:30:20 -0700 Subject: [PATCH 49/66] fix commit info --- kernel/src/actions/mod.rs | 24 +++++++++++++++++------- kernel/src/engine/arrow_expression.rs | 3 +++ kernel/src/transaction.rs | 5 ++++- 3 files changed, 24 insertions(+), 8 deletions(-) diff --git a/kernel/src/actions/mod.rs b/kernel/src/actions/mod.rs index 7165de784..9d5c5eb6f 100644 --- a/kernel/src/actions/mod.rs +++ b/kernel/src/actions/mod.rs @@ -137,24 +137,26 @@ impl Protocol { } #[derive(Debug, Clone, PartialEq, Eq, Schema)] -struct OperationParameters {} +struct OperationParameters { + operation_parameters: Option, +} #[derive(Debug, Clone, PartialEq, Eq, Schema)] #[cfg_attr(feature = "developer-visibility", visibility::make(pub))] #[cfg_attr(not(feature = "developer-visibility"), visibility::make(pub(crate)))] struct CommitInfo { /// The time this logical file was created, as milliseconds since the epoch. - pub(crate) timestamp: i64, + pub(crate) timestamp: Option, /// An arbitrary string that identifies the operation associated with this commit. This is /// specified by the engine. - pub(crate) operation: String, + pub(crate) operation: Option, /// Map of arbitrary string key-value pairs that provide additional information about the /// operation. This is specified by the engine. /// /// For now this is always empty; and since we don't have the ability to construct an empty /// string-string map, we spoof the operation_parameters with an empty struct so it serializes /// the same as an empty map (as `{}`). - operation_parameters: OperationParameters, + operation_parameters: Option, /// The version of the delta_kernel crate used to write this commit. The kernel will always /// write this field, but it is optional since many tables will not have this field (i.e. any /// tables not written by kernel). @@ -450,9 +452,17 @@ mod tests { let expected = Arc::new(StructType::new(vec![StructField::new( "commitInfo", StructType::new(vec![ - StructField::new("timestamp", DataType::LONG, false), - StructField::new("operation", DataType::STRING, false), - StructField::new("operationParameters", StructType::new([]), false), + StructField::new("timestamp", DataType::LONG, true), + StructField::new("operation", DataType::STRING, true), + StructField::new( + "operationParameters", + StructType::new([StructField::new( + "operationParameters", + DataType::LONG, + true, + )]), + true, + ), StructField::new("kernelVersion", DataType::STRING, true), StructField::new( "engineCommitInfo", diff --git a/kernel/src/engine/arrow_expression.rs b/kernel/src/engine/arrow_expression.rs index fa87cb8bb..0209234e1 100644 --- a/kernel/src/engine/arrow_expression.rs +++ b/kernel/src/engine/arrow_expression.rs @@ -74,6 +74,8 @@ impl Scalar { .iter() .map(ArrowField::try_from) .try_collect()?; + println!("doing try new fields: {:#?}", fields); + println!("doing try new arrays: {:#?}", arrays); Arc::new(StructArray::try_new(fields, arrays, None)?) } Array(data) => { @@ -205,6 +207,7 @@ fn evaluate_expression( .zip(output_schema.fields()) .map(|(expr, field)| evaluate_expression(expr, batch, Some(field.data_type()))); let output_cols: Vec = columns.try_collect()?; + println!("output columns: {:#?}", output_cols); let output_fields: Vec = output_cols .iter() .zip(output_schema.fields()) diff --git a/kernel/src/transaction.rs b/kernel/src/transaction.rs index 5b219d889..1b7e6aefa 100644 --- a/kernel/src/transaction.rs +++ b/kernel/src/transaction.rs @@ -133,7 +133,10 @@ fn generate_commit_info( // FIXME we should take a timestamp closer to commit time? Expression::literal(timestamp), Expression::literal(operation.unwrap_or(UNKNOWN_OPERATION)), - Expression::literal(Scalar::Struct(StructData::try_new(vec![], vec![])?)), + Expression::literal(Scalar::Struct(StructData::try_new( + vec![StructField::new("operation_parameters", DataType::INTEGER, true)], + vec![Scalar::Null(DataType::INTEGER)], + )?)), Expression::literal(format!("v{}", KERNEL_VERSION)), Expression::column("engineCommitInfo"), ]; From 5afe8dbb0582c877a0a36a8a96250b25bce1dec1 Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Tue, 22 Oct 2024 14:31:34 -0700 Subject: [PATCH 50/66] fix error ffi --- ffi/src/lib.rs | 2 ++ 1 file changed, 2 insertions(+) diff --git a/ffi/src/lib.rs b/ffi/src/lib.rs index f54a0d412..ff3d60f2e 100644 --- a/ffi/src/lib.rs +++ b/ffi/src/lib.rs @@ -331,6 +331,7 @@ pub enum KernelError { InvalidLogPath, InvalidCommitInfo, FileAlreadyExists, + MissingCommitInfo, } impl From for KernelError { @@ -380,6 +381,7 @@ impl From for KernelError { Error::InvalidLogPath(_) => KernelError::InvalidLogPath, Error::InvalidCommitInfo(_) => KernelError::InvalidCommitInfo, Error::FileAlreadyExists(_) => KernelError::FileAlreadyExists, + Error::MissingCommitInfo => KernelError::MissingCommitInfo, } } } From 7f87591d29b21910d0eea71bffb6be2cfe1377a3 Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Tue, 22 Oct 2024 14:31:52 -0700 Subject: [PATCH 51/66] fmt --- kernel/src/engine/sync/json.rs | 3 ++- kernel/src/transaction.rs | 6 +++++- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/kernel/src/engine/sync/json.rs b/kernel/src/engine/sync/json.rs index 26c1c2e83..d452b443d 100644 --- a/kernel/src/engine/sync/json.rs +++ b/kernel/src/engine/sync/json.rs @@ -10,7 +10,8 @@ use crate::engine::arrow_utils::parse_json as arrow_parse_json; use crate::engine::arrow_utils::write_json; use crate::schema::SchemaRef; use crate::{ - DeltaResult, EngineData, Error, ExpressionRef, FileDataReadResultIterator, FileMeta, JsonHandler, + DeltaResult, EngineData, Error, ExpressionRef, FileDataReadResultIterator, FileMeta, + JsonHandler, }; pub(crate) struct SyncJsonHandler; diff --git a/kernel/src/transaction.rs b/kernel/src/transaction.rs index 1b7e6aefa..6398f7f3b 100644 --- a/kernel/src/transaction.rs +++ b/kernel/src/transaction.rs @@ -134,7 +134,11 @@ fn generate_commit_info( Expression::literal(timestamp), Expression::literal(operation.unwrap_or(UNKNOWN_OPERATION)), Expression::literal(Scalar::Struct(StructData::try_new( - vec![StructField::new("operation_parameters", DataType::INTEGER, true)], + vec![StructField::new( + "operation_parameters", + DataType::INTEGER, + true, + )], vec![Scalar::Null(DataType::INTEGER)], )?)), Expression::literal(format!("v{}", KERNEL_VERSION)), From 76cdfaa8303604033baf9c2184dc950817127931 Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Tue, 22 Oct 2024 14:54:02 -0700 Subject: [PATCH 52/66] remove my debugging --- kernel/src/engine/arrow_expression.rs | 3 --- 1 file changed, 3 deletions(-) diff --git a/kernel/src/engine/arrow_expression.rs b/kernel/src/engine/arrow_expression.rs index 0209234e1..fa87cb8bb 100644 --- a/kernel/src/engine/arrow_expression.rs +++ b/kernel/src/engine/arrow_expression.rs @@ -74,8 +74,6 @@ impl Scalar { .iter() .map(ArrowField::try_from) .try_collect()?; - println!("doing try new fields: {:#?}", fields); - println!("doing try new arrays: {:#?}", arrays); Arc::new(StructArray::try_new(fields, arrays, None)?) } Array(data) => { @@ -207,7 +205,6 @@ fn evaluate_expression( .zip(output_schema.fields()) .map(|(expr, field)| evaluate_expression(expr, batch, Some(field.data_type()))); let output_cols: Vec = columns.try_collect()?; - println!("output columns: {:#?}", output_cols); let output_fields: Vec = output_cols .iter() .zip(output_schema.fields()) From cc7598ccd1a87fa195259b2a1c9cdcdce18c8f34 Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Tue, 22 Oct 2024 17:13:53 -0700 Subject: [PATCH 53/66] docs, cleanup, better tests --- kernel/src/actions/mod.rs | 9 + kernel/src/engine/arrow_utils.rs | 4 +- kernel/src/engine/default/json.rs | 6 +- kernel/src/lib.rs | 17 +- kernel/src/transaction.rs | 312 +++++++++++++++++++++++++++--- kernel/tests/write.rs | 33 +--- 6 files changed, 301 insertions(+), 80 deletions(-) diff --git a/kernel/src/actions/mod.rs b/kernel/src/actions/mod.rs index 9d5c5eb6f..145d582db 100644 --- a/kernel/src/actions/mod.rs +++ b/kernel/src/actions/mod.rs @@ -10,6 +10,7 @@ use visitors::{AddVisitor, MetadataVisitor, ProtocolVisitor}; use self::deletion_vector::DeletionVectorDescriptor; use crate::actions::schemas::GetStructField; use crate::features::{ReaderFeatures, WriterFeatures}; +use crate::SchemaRef; use crate::{schema::StructType, DeltaResult, EngineData}; pub mod deletion_vector; @@ -42,12 +43,20 @@ static LOG_SCHEMA: LazyLock = LazyLock::new(|| { ]) }); +static LOG_COMMIT_INFO_SCHEMA: LazyLock = LazyLock::new(|| { + StructType::new([Option::::get_struct_field(COMMIT_INFO_NAME)]).into() +}); + #[cfg_attr(feature = "developer-visibility", visibility::make(pub))] #[cfg_attr(not(feature = "developer-visibility"), visibility::make(pub(crate)))] fn get_log_schema() -> &'static StructType { &LOG_SCHEMA } +pub(crate) fn get_log_commit_info_schema() -> &'static SchemaRef { + &LOG_COMMIT_INFO_SCHEMA +} + #[derive(Debug, Clone, PartialEq, Eq, Schema)] pub struct Format { /// Name of the encoding for files in this table diff --git a/kernel/src/engine/arrow_utils.rs b/kernel/src/engine/arrow_utils.rs index 84d351ea9..b9c14cd96 100644 --- a/kernel/src/engine/arrow_utils.rs +++ b/kernel/src/engine/arrow_utils.rs @@ -664,9 +664,9 @@ fn parse_json_impl(json_strings: &StringArray, schema: ArrowSchemaRef) -> DeltaR /// write an arrow RecordBatch to a JSON string by appending to a buffer. /// -/// TODO: this should stream data to the JSON writer and output an iterator. +/// TODO (zach): this should stream data to the JSON writer and output an iterator. pub(crate) fn write_json( - data: Box> + Send>, + data: impl Iterator> + Send, ) -> DeltaResult> { let mut writer = LineDelimitedWriter::new(Vec::new()); for chunk in data.into_iter() { diff --git a/kernel/src/engine/default/json.rs b/kernel/src/engine/default/json.rs index b8aaae154..4c1948f4b 100644 --- a/kernel/src/engine/default/json.rs +++ b/kernel/src/engine/default/json.rs @@ -103,7 +103,7 @@ impl JsonHandler for DefaultJsonHandler { // Put if absent let store = self.store.clone(); // cheap Arc let path = Path::from(path.path()); - let path2 = path.clone(); // FIXME gross + let path_str = path.to_string(); self.task_executor .block_on(async move { store @@ -111,9 +111,7 @@ impl JsonHandler for DefaultJsonHandler { .await }) .map_err(|e| match e { - object_store::Error::AlreadyExists { .. } => { - Error::FileAlreadyExists(path2.to_string()) - } + object_store::Error::AlreadyExists { .. } => Error::FileAlreadyExists(path_str), e => e.into(), })?; Ok(()) diff --git a/kernel/src/lib.rs b/kernel/src/lib.rs index c9776c04b..6d47d9ae2 100644 --- a/kernel/src/lib.rs +++ b/kernel/src/lib.rs @@ -208,13 +208,13 @@ pub trait JsonHandler: Send + Sync { predicate: Option, ) -> DeltaResult; - /// Atomically (!) write a single JSON file. Each row of the input data represents an action - /// in the delta log. this PUT must: + /// Atomically (!) write a single JSON file. Each row of the input data should be written as a + /// new JSON object appended to the file. this write must: /// (1) serialize the data to newline-delimited json (each row is a json object literal) - /// (2) write the data to the object store atomically (i.e. if the file already exists, fail - /// unless the overwrite flag is set) + /// (2) write the data to storage atomically (i.e. if the file already exists, fail unless the + /// overwrite flag is set) /// - /// The JSON data should be written as { "column1": "value1", "column2": "value2", ... } + /// For example, the JSON data should be written as { "column1": "val1", "column2": "val2", .. } /// with each row on a new line. /// /// NOTE: Null columns should not be written to the JSON file. For example, if a row has columns @@ -224,15 +224,12 @@ pub trait JsonHandler: Send + Sync { /// /// # Parameters /// - /// - `path` - URL to write the JSON file to - /// - `data` - Iterator of EngineData to write to the JSON file. each row should be written as + /// - `path` - URL specifying the location to write the JSON file + /// - `data` - Iterator of EngineData to write to the JSON file. Each row should be written as /// a new JSON object appended to the file. (that is, the file is newline-delimeted JSON, and /// each row is a JSON object on a single line) /// - `overwrite` - If true, overwrite the file if it exists. If false, the call must fail if /// the file exists. - /// - /// NOTE: the `overwrite` flag isn't used for the existing commit flow, but will be used in the - /// future to write `_last_checkpoint` files. fn write_json_file( &self, path: &Url, diff --git a/kernel/src/transaction.rs b/kernel/src/transaction.rs index 6398f7f3b..1a23023c7 100644 --- a/kernel/src/transaction.rs +++ b/kernel/src/transaction.rs @@ -2,7 +2,7 @@ use std::iter; use std::sync::Arc; use std::time::{SystemTime, UNIX_EPOCH}; -use crate::actions::get_log_schema; +use crate::actions::get_log_commit_info_schema; use crate::error::Error; use crate::expressions::{Scalar, StructData}; use crate::path::ParsedLogPath; @@ -13,7 +13,20 @@ use crate::{DataType, DeltaResult, Engine, EngineData, Expression, Version}; const KERNEL_VERSION: &str = env!("CARGO_PKG_VERSION"); const UNKNOWN_OPERATION: &str = "UNKNOWN"; -/// A transaction represents an in-progress write to a table. +/// A transaction represents an in-progress write to a table. After creating a transaction, changes +/// to the table may be staged via the transaction methods before calling `commit` to commit the +/// changes to the table. +/// +/// # Examples +/// +/// ```rust,no_run +/// // create a transaction +/// let mut txn = table.new_transaction(&engine)?; +/// // stage table changes (right now only commit info) +/// txn.commit_info(Box::new(ArrowEngineData::new(engine_commit_info))); +/// // commit! (consume the transaction) +/// txn.commit(&engine)?; +/// ``` pub struct Transaction { read_snapshot: Arc, operation: Option, @@ -45,7 +58,8 @@ impl Transaction { } } - /// Consume the transaction and commit the in-progress write to the table. + /// Consume the transaction and commit it to the table. The result is a [CommitResult] which + /// will include the failed transaction in case of a conflict so the user can retry. pub fn commit(self, engine: &dyn Engine) -> DeltaResult { // step one: construct the iterator of actions we want to commit // note: only support commit_info right now (and it's required) @@ -91,9 +105,7 @@ impl Transaction { /// commit_info engine data chunk with one row and one column of: /// 1. `engineCommitInfo` column with an empty Map /// 2. `engineCommitInfo` null column of type Map - /// 3. a column that has a name other than `engineCommitInfo`; Delta can detect that the column - /// is missing and substitute a null literal in its place. The type of that column doesn't - /// matter, Delta will ignore it. + /// any other columns in the data chunk are ignored. pub fn commit_info(&mut self, commit_info: Box) { self.commit_info = Some(commit_info.into()); } @@ -102,6 +114,8 @@ impl Transaction { /// Result after committing a transaction. If 'committed', the version is the new version written /// to the log. If 'conflict', the transaction is returned so the caller can resolve the conflict /// (along with the version which conflicted). +// TODO(zach): in order to make the returning of a transcation useful, we need to add APIs to +// update the transaction to a new version etc. #[derive(Debug)] pub enum CommitResult { /// The transaction was successfully committed at the version. @@ -130,7 +144,7 @@ fn generate_commit_info( .try_into() .map_err(|_| Error::generic("milliseconds since unix_epoch exceeded i64 size"))?; let commit_info_exprs = [ - // FIXME we should take a timestamp closer to commit time? + // TODO(zach): we should probably take a timestamp closer to actual commit time? Expression::literal(timestamp), Expression::literal(operation.unwrap_or(UNKNOWN_OPERATION)), Expression::literal(Scalar::Struct(StructData::try_new( @@ -145,8 +159,6 @@ fn generate_commit_info( Expression::column("engineCommitInfo"), ]; let commit_info_expr = Expression::struct_from([Expression::struct_from(commit_info_exprs)]); - // TODO probably just create a static - let commit_info_schema = get_log_schema().project_as_struct(&["commitInfo"])?; let engine_commit_info_schema = StructType::new(vec![StructField::new( "engineCommitInfo", @@ -157,7 +169,7 @@ fn generate_commit_info( let commit_info_evaluator = engine.get_expression_handler().get_evaluator( engine_commit_info_schema.into(), commit_info_expr, - commit_info_schema.into(), + get_log_commit_info_schema().clone().into(), ); commit_info_evaluator.evaluate(engine_commit_info.as_ref()) @@ -171,7 +183,9 @@ mod tests { use crate::engine::arrow_expression::ArrowExpressionHandler; use crate::{ExpressionHandler, FileSystemClient, JsonHandler, ParquetHandler}; + use arrow::json::writer::LineDelimitedWriter; use arrow::record_batch::RecordBatch; + use arrow_array::builder::StringBuilder; use arrow_schema::Schema as ArrowSchema; use arrow_schema::{DataType as ArrowDataType, Field}; @@ -201,7 +215,47 @@ mod tests { } } - // simple test for generating commit info + fn build_map(entries: Vec<(&str, &str)>) -> arrow_array::MapArray { + let key_builder = StringBuilder::new(); + let val_builder = StringBuilder::new(); + let names = arrow_array::builder::MapFieldNames { + entry: "entries".to_string(), + key: "key".to_string(), + value: "value".to_string(), + }; + let mut builder = + arrow_array::builder::MapBuilder::new(Some(names), key_builder, val_builder); + for (key, val) in entries { + builder.keys().append_value(key); + builder.values().append_value(val); + builder.append(true).unwrap(); + } + builder.finish() + } + + // convert it to JSON just for ease of comparison (and since we ultimately persist as JSON) + fn as_json_and_scrub_timestamp(data: Box) -> serde_json::Value { + let record_batch: RecordBatch = data + .into_any() + .downcast::() + .unwrap() + .into(); + + let buf = Vec::new(); + let mut writer = LineDelimitedWriter::new(buf); + writer.write_batches(&vec![&record_batch]).unwrap(); + writer.finish().unwrap(); + let buf = writer.into_inner(); + + let mut result: serde_json::Value = serde_json::from_slice(&buf).unwrap(); + *result + .get_mut("commitInfo") + .unwrap() + .get_mut("timestamp") + .unwrap() = serde_json::Value::Number(0.into()); + result + } + #[test] fn test_generate_commit_info() -> DeltaResult<()> { let engine = ExprEngine::new(); @@ -224,23 +278,9 @@ mod tests { false, )])); - use arrow_array::builder::StringBuilder; - let key_builder = StringBuilder::new(); - let val_builder = StringBuilder::new(); - let names = arrow_array::builder::MapFieldNames { - entry: "entries".to_string(), - key: "key".to_string(), - value: "value".to_string(), - }; - let mut builder = - arrow_array::builder::MapBuilder::new(Some(names), key_builder, val_builder); - builder.keys().append_value("engineInfo"); - builder.values().append_value("default engine"); - builder.append(true).unwrap(); - let array = builder.finish(); - + let map_array = build_map(vec![("engineInfo", "default engine")]); let commit_info_batch = - RecordBatch::try_new(engine_commit_info_schema, vec![Arc::new(array)])?; + RecordBatch::try_new(engine_commit_info_schema, vec![Arc::new(map_array)])?; let actions = generate_commit_info( &engine, @@ -248,12 +288,220 @@ mod tests { Arc::new(ArrowEngineData::new(commit_info_batch)), )?; + let expected = serde_json::json!({ + "commitInfo": { + "timestamp": 0, + "operation": "test operation", + "kernelVersion": format!("v{}", env!("CARGO_PKG_VERSION")), + "operationParameters": {}, + "engineCommitInfo": { + "engineInfo": "default engine" + } + } + }); + assert_eq!(actions.length(), 1); - let record_batch: RecordBatch = actions - .into_any() - .downcast::() - .unwrap() - .into(); + let result = as_json_and_scrub_timestamp(actions); + assert_eq!(result, expected); + + Ok(()) + } + + #[test] + fn test_commit_info_with_multiple_columns() -> DeltaResult<()> { + let engine = ExprEngine::new(); + let engine_commit_info_schema = Arc::new(ArrowSchema::new(vec![ + Field::new( + "engineCommitInfo", + ArrowDataType::Map( + Arc::new(Field::new( + "entries", + ArrowDataType::Struct( + vec![ + Field::new("key", ArrowDataType::Utf8, false), + Field::new("value", ArrowDataType::Utf8, true), + ] + .into(), + ), + false, + )), + false, + ), + false, + ), + Field::new("operation", ArrowDataType::Utf8, true), + ])); + + let map_array = build_map(vec![("engineInfo", "default engine")]); + + let commit_info_batch = RecordBatch::try_new( + engine_commit_info_schema, + vec![ + Arc::new(map_array), + Arc::new(arrow_array::StringArray::from(vec!["some_string"])), + ], + )?; + + let actions = generate_commit_info( + &engine, + Some("test operation"), + Arc::new(ArrowEngineData::new(commit_info_batch)), + )?; + + let expected = serde_json::json!({ + "commitInfo": { + "timestamp": 0, + "operation": "test operation", + "kernelVersion": format!("v{}", env!("CARGO_PKG_VERSION")), + "operationParameters": {}, + "engineCommitInfo": { + "engineInfo": "default engine" + } + } + }); + + assert_eq!(actions.length(), 1); + let result = as_json_and_scrub_timestamp(actions); + assert_eq!(result, expected); + + Ok(()) + } + + #[test] + fn test_invalid_commit_info_missing_column() -> DeltaResult<()> { + let engine = ExprEngine::new(); + let engine_commit_info_schema = Arc::new(ArrowSchema::new(vec![Field::new( + "some_column_name", + ArrowDataType::Utf8, + true, + )])); + let commit_info_batch = RecordBatch::try_new( + engine_commit_info_schema, + vec![Arc::new(arrow_array::StringArray::new_null(1))], + )?; + + let _ = generate_commit_info( + &engine, + Some("test operation"), + Arc::new(ArrowEngineData::new(commit_info_batch)), + ) + .map_err(|e| match e { + Error::Arrow(arrow_schema::ArrowError::SchemaError(_)) => (), + _ => panic!("expected arrow schema error error, got {:?}", e), + }); + + Ok(()) + } + + #[test] + fn test_invalid_commit_info_invalid_column_type() -> DeltaResult<()> { + let engine = ExprEngine::new(); + let engine_commit_info_schema = Arc::new(ArrowSchema::new(vec![Field::new( + "engineCommitInfo", + ArrowDataType::Utf8, + true, + )])); + let commit_info_batch = RecordBatch::try_new( + engine_commit_info_schema, + vec![Arc::new(arrow_array::StringArray::new_null(1))], + )?; + + let _ = generate_commit_info( + &engine, + Some("test operation"), + Arc::new(ArrowEngineData::new(commit_info_batch)), + ) + .map_err(|e| match e { + Error::Arrow(arrow_schema::ArrowError::InvalidArgumentError(_)) => (), + _ => panic!("expected arrow invalid arg error, got {:?}", e), + }); + + Ok(()) + } + + fn assert_empty_commit_info( + data: Box, + write_engine_commit_info: bool, + ) -> DeltaResult<()> { + assert_eq!(data.length(), 1); + let expected = if write_engine_commit_info { + serde_json::json!({ + "commitInfo": { + "timestamp": 0, + "operation": "test operation", + "kernelVersion": format!("v{}", env!("CARGO_PKG_VERSION")), + "operationParameters": {}, + "engineCommitInfo": {} + } + }) + } else { + serde_json::json!({ + "commitInfo": { + "timestamp": 0, + "operation": "test operation", + "kernelVersion": format!("v{}", env!("CARGO_PKG_VERSION")), + "operationParameters": {}, + } + }) + }; + let result = as_json_and_scrub_timestamp(data); + assert_eq!(result, expected); + Ok(()) + } + + // Three cases for empty commit info: + // 1. `engineCommitInfo` column with an empty Map + // 2. `engineCommitInfo` null column of type Map + // 3. a column that has a name other than `engineCommitInfo`; Delta can detect that the column + // is missing and substitute a null literal in its place. The type of that column doesn't + // matter, Delta will ignore it. + #[test] + fn test_empty_commit_info() -> DeltaResult<()> { + // test with null map and empty map + for is_null in [true, false] { + let engine = ExprEngine::new(); + let engine_commit_info_schema = Arc::new(ArrowSchema::new(vec![Field::new( + "engineCommitInfo", + ArrowDataType::Map( + Arc::new(Field::new( + "entries", + ArrowDataType::Struct( + vec![ + Field::new("key", ArrowDataType::Utf8, false), + Field::new("value", ArrowDataType::Utf8, true), + ] + .into(), + ), + false, + )), + false, + ), + true, + )])); + use arrow_array::builder::StringBuilder; + let key_builder = StringBuilder::new(); + let val_builder = StringBuilder::new(); + let names = arrow_array::builder::MapFieldNames { + entry: "entries".to_string(), + key: "key".to_string(), + value: "value".to_string(), + }; + let mut builder = + arrow_array::builder::MapBuilder::new(Some(names), key_builder, val_builder); + builder.append(is_null).unwrap(); + let array = builder.finish(); + + let commit_info_batch = + RecordBatch::try_new(engine_commit_info_schema, vec![Arc::new(array)])?; + + let actions = generate_commit_info( + &engine, + Some("test operation"), + Arc::new(ArrowEngineData::new(commit_info_batch)), + )?; + + assert_empty_commit_info(actions, is_null)?; + } Ok(()) } } diff --git a/kernel/tests/write.rs b/kernel/tests/write.rs index d51d4e18b..00101f0b1 100644 --- a/kernel/tests/write.rs +++ b/kernel/tests/write.rs @@ -103,7 +103,7 @@ async fn test_commit_info() -> Result<(), Box> { // create a transaction let mut txn = table.new_transaction(&engine)?; - // add commit info of the form {engineInfo: "default engine"} + // add commit info of the form {engineCommitInfo: Map { "engineInfo": "default engine" } } let commit_info_schema = Arc::new(ArrowSchema::new(vec![Field::new( "engineCommitInfo", ArrowDataType::Map( @@ -170,40 +170,9 @@ async fn test_commit_info() -> Result<(), Box> { }); assert_eq!(parsed_commit, expected_commit); - - //// one null row commit info - //let mut txn = table.new_transaction(&engine)?; - //let commit_info_schema = Arc::new(ArrowSchema::new(vec![Field::new( - // "some_column_name", - // ArrowDataType::Utf8, - // true, - //)])); - //let commit_info_batch = RecordBatch::try_new( - // commit_info_schema.clone(), - // vec![Arc::new(StringArray::new_null(1))], - //)?; - //txn.commit_info(Box::new(ArrowEngineData::new(commit_info_batch))); - // - //// commit! - //txn.commit(&engine)?; - // - //let commit1 = store - // .get(&Path::from( - // "/test_table/_delta_log/00000000000000000002.json", - // )) - // .await?; - //assert_eq!( - // String::from_utf8(commit1.bytes().await?.to_vec())?, - // "{\"commitInfo\":{\"kernelVersion\":\"v0.3.1\"}}\n" - //); Ok(()) } -// need to test various invalid commit infos -// 1. missing engineCommitInfo -// 2. invalid engineCommitInfo type -// 3. other columns that could override (like operation) - #[tokio::test] async fn test_empty_commit() -> Result<(), Box> { // setup tracing From a1ba00891e0caedc2b8ef44b7370e972aefec3f0 Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Tue, 22 Oct 2024 17:14:57 -0700 Subject: [PATCH 54/66] clippy --- kernel/src/transaction.rs | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/kernel/src/transaction.rs b/kernel/src/transaction.rs index 1a23023c7..95ebd79ab 100644 --- a/kernel/src/transaction.rs +++ b/kernel/src/transaction.rs @@ -105,7 +105,8 @@ impl Transaction { /// commit_info engine data chunk with one row and one column of: /// 1. `engineCommitInfo` column with an empty Map /// 2. `engineCommitInfo` null column of type Map - /// any other columns in the data chunk are ignored. + /// + /// Any other columns in the data chunk are ignored. pub fn commit_info(&mut self, commit_info: Box) { self.commit_info = Some(commit_info.into()); } @@ -243,7 +244,7 @@ mod tests { let buf = Vec::new(); let mut writer = LineDelimitedWriter::new(buf); - writer.write_batches(&vec![&record_batch]).unwrap(); + writer.write_batches(&[&record_batch]).unwrap(); writer.finish().unwrap(); let buf = writer.into_inner(); From 525b8ffcf0a9eea4ff01589ef812cff960a9759a Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Tue, 22 Oct 2024 22:24:28 -0700 Subject: [PATCH 55/66] rename + docs --- kernel/src/engine/arrow_utils.rs | 4 ++-- kernel/src/engine/default/json.rs | 4 ++-- kernel/src/engine/sync/json.rs | 4 ++-- kernel/src/transaction.rs | 5 ++--- 4 files changed, 8 insertions(+), 9 deletions(-) diff --git a/kernel/src/engine/arrow_utils.rs b/kernel/src/engine/arrow_utils.rs index b9c14cd96..ba98b06c4 100644 --- a/kernel/src/engine/arrow_utils.rs +++ b/kernel/src/engine/arrow_utils.rs @@ -665,7 +665,7 @@ fn parse_json_impl(json_strings: &StringArray, schema: ArrowSchemaRef) -> DeltaR /// write an arrow RecordBatch to a JSON string by appending to a buffer. /// /// TODO (zach): this should stream data to the JSON writer and output an iterator. -pub(crate) fn write_json( +pub(crate) fn to_json_bytes( data: impl Iterator> + Send, ) -> DeltaResult> { let mut writer = LineDelimitedWriter::new(Vec::new()); @@ -1437,7 +1437,7 @@ mod tests { vec![Arc::new(StringArray::from(vec!["string1", "string2"]))], )?; let data: Box = Box::new(ArrowEngineData::new(data)); - let json = write_json(Box::new(std::iter::once(data)))?; + let json = to_json_bytes(Box::new(std::iter::once(data)))?; assert_eq!( json, "{\"string\":\"string1\"}\n{\"string\":\"string2\"}\n".as_bytes() diff --git a/kernel/src/engine/default/json.rs b/kernel/src/engine/default/json.rs index 4c1948f4b..169c5b647 100644 --- a/kernel/src/engine/default/json.rs +++ b/kernel/src/engine/default/json.rs @@ -16,7 +16,7 @@ use url::Url; use super::executor::TaskExecutor; use super::file_stream::{FileOpenFuture, FileOpener, FileStream}; use crate::engine::arrow_utils::parse_json as arrow_parse_json; -use crate::engine::arrow_utils::write_json; +use crate::engine::arrow_utils::to_json_bytes; use crate::schema::SchemaRef; use crate::{ DeltaResult, EngineData, Error, ExpressionRef, FileDataReadResultIterator, FileMeta, @@ -99,7 +99,7 @@ impl JsonHandler for DefaultJsonHandler { data: Box> + Send>, _overwrite: bool, ) -> DeltaResult<()> { - let buffer = write_json(data)?; + let buffer = to_json_bytes(data)?; // Put if absent let store = self.store.clone(); // cheap Arc let path = Path::from(path.path()); diff --git a/kernel/src/engine/sync/json.rs b/kernel/src/engine/sync/json.rs index d452b443d..016fb2658 100644 --- a/kernel/src/engine/sync/json.rs +++ b/kernel/src/engine/sync/json.rs @@ -7,7 +7,7 @@ use url::Url; use super::read_files; use crate::engine::arrow_data::ArrowEngineData; use crate::engine::arrow_utils::parse_json as arrow_parse_json; -use crate::engine::arrow_utils::write_json; +use crate::engine::arrow_utils::to_json_bytes; use crate::schema::SchemaRef; use crate::{ DeltaResult, EngineData, Error, ExpressionRef, FileDataReadResultIterator, FileMeta, @@ -67,7 +67,7 @@ impl JsonHandler for SyncJsonHandler { // write data to tmp file let mut tmp_file = NamedTempFile::new_in(parent)?; - let buf = write_json(data)?; + let buf = to_json_bytes(data)?; tmp_file.write_all(&buf)?; tmp_file.flush()?; diff --git a/kernel/src/transaction.rs b/kernel/src/transaction.rs index 95ebd79ab..b5f0ef2f7 100644 --- a/kernel/src/transaction.rs +++ b/kernel/src/transaction.rs @@ -102,9 +102,8 @@ impl Transaction { /// /// The engine is required to provide commit info before committing the transaction. If the /// engine would like to omit engine-specific commit info, it can do so by passing pass a - /// commit_info engine data chunk with one row and one column of: - /// 1. `engineCommitInfo` column with an empty Map - /// 2. `engineCommitInfo` null column of type Map + /// commit_info engine data chunk with one row and one column of type `Map` + /// that can either be `null` or contain an empty map. /// /// Any other columns in the data chunk are ignored. pub fn commit_info(&mut self, commit_info: Box) { From a86495a181f6b331332fb09f6a6b1cc4923c79ad Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Wed, 23 Oct 2024 11:52:45 -0700 Subject: [PATCH 56/66] make CommitInfo have correct schema and isolate the hack inside generate_commit_info --- kernel/src/actions/mod.rs | 19 +++---------------- kernel/src/transaction.rs | 32 +++++++++++++++++++++++++++++--- 2 files changed, 32 insertions(+), 19 deletions(-) diff --git a/kernel/src/actions/mod.rs b/kernel/src/actions/mod.rs index 145d582db..333cd0aeb 100644 --- a/kernel/src/actions/mod.rs +++ b/kernel/src/actions/mod.rs @@ -145,11 +145,6 @@ impl Protocol { } } -#[derive(Debug, Clone, PartialEq, Eq, Schema)] -struct OperationParameters { - operation_parameters: Option, -} - #[derive(Debug, Clone, PartialEq, Eq, Schema)] #[cfg_attr(feature = "developer-visibility", visibility::make(pub))] #[cfg_attr(not(feature = "developer-visibility"), visibility::make(pub(crate)))] @@ -160,12 +155,8 @@ struct CommitInfo { /// specified by the engine. pub(crate) operation: Option, /// Map of arbitrary string key-value pairs that provide additional information about the - /// operation. This is specified by the engine. - /// - /// For now this is always empty; and since we don't have the ability to construct an empty - /// string-string map, we spoof the operation_parameters with an empty struct so it serializes - /// the same as an empty map (as `{}`). - operation_parameters: Option, + /// operation. This is specified by the engine. For now this is always empty on write. + pub(crate) operation_parameters: Option>, /// The version of the delta_kernel crate used to write this commit. The kernel will always /// write this field, but it is optional since many tables will not have this field (i.e. any /// tables not written by kernel). @@ -465,11 +456,7 @@ mod tests { StructField::new("operation", DataType::STRING, true), StructField::new( "operationParameters", - StructType::new([StructField::new( - "operationParameters", - DataType::LONG, - true, - )]), + MapType::new(DataType::STRING, DataType::STRING, false), true, ), StructField::new("kernelVersion", DataType::STRING, true), diff --git a/kernel/src/transaction.rs b/kernel/src/transaction.rs index b5f0ef2f7..2c8ec2488 100644 --- a/kernel/src/transaction.rs +++ b/kernel/src/transaction.rs @@ -3,6 +3,7 @@ use std::sync::Arc; use std::time::{SystemTime, UNIX_EPOCH}; use crate::actions::get_log_commit_info_schema; +use crate::actions::COMMIT_INFO_NAME; use crate::error::Error; use crate::expressions::{Scalar, StructData}; use crate::path::ParsedLogPath; @@ -19,7 +20,7 @@ const UNKNOWN_OPERATION: &str = "UNKNOWN"; /// /// # Examples /// -/// ```rust,no_run +/// ```rust,ignore /// // create a transaction /// let mut txn = table.new_transaction(&engine)?; /// // stage table changes (right now only commit info) @@ -147,9 +148,11 @@ fn generate_commit_info( // TODO(zach): we should probably take a timestamp closer to actual commit time? Expression::literal(timestamp), Expression::literal(operation.unwrap_or(UNKNOWN_OPERATION)), + // HACK (part 1/2): since we don't have proper map support, we create a literal struct with + // one null field to create data that serializes as "operationParameters": {} Expression::literal(Scalar::Struct(StructData::try_new( vec![StructField::new( - "operation_parameters", + "operation_parameter_int", DataType::INTEGER, true, )], @@ -166,10 +169,33 @@ fn generate_commit_info( false, )]); + // HACK (part 2/2): we need to modify the commit info schema to match the expression above (a + // struct with a single null int field). + let mut commit_info_schema = get_log_commit_info_schema().as_ref().clone(); + let commit_info_field = commit_info_schema + .fields + .get_mut(COMMIT_INFO_NAME) + .ok_or_else(|| Error::missing_column(COMMIT_INFO_NAME))?; + let DataType::Struct(mut commit_info_data_type) = commit_info_field.data_type().clone() else { + return Err(Error::internal_error( + "commit_info_field should be a struct", + )); + }; + commit_info_data_type + .fields + .get_mut("operationParameters") + .unwrap() + .data_type = DataType::Struct(Box::new(StructType::new(vec![StructField::new( + "hack_operation_parameter_int", + DataType::INTEGER, + true, + )]))); + commit_info_field.data_type = DataType::Struct(commit_info_data_type); + let commit_info_evaluator = engine.get_expression_handler().get_evaluator( engine_commit_info_schema.into(), commit_info_expr, - get_log_commit_info_schema().clone().into(), + commit_info_schema.into(), ); commit_info_evaluator.evaluate(engine_commit_info.as_ref()) From c22f625efdd4b76c560f45462439b69ac6b009de Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Wed, 23 Oct 2024 13:01:49 -0700 Subject: [PATCH 57/66] fix tests to match on Backtraced { .. } --- kernel/src/transaction.rs | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/kernel/src/transaction.rs b/kernel/src/transaction.rs index 2c8ec2488..ca4d311c5 100644 --- a/kernel/src/transaction.rs +++ b/kernel/src/transaction.rs @@ -413,6 +413,14 @@ mod tests { ) .map_err(|e| match e { Error::Arrow(arrow_schema::ArrowError::SchemaError(_)) => (), + Error::Backtraced { source, .. } + if matches!( + &*source, + Error::Arrow(arrow_schema::ArrowError::SchemaError(_)) + ) => + { + () + } _ => panic!("expected arrow schema error error, got {:?}", e), }); @@ -439,6 +447,14 @@ mod tests { ) .map_err(|e| match e { Error::Arrow(arrow_schema::ArrowError::InvalidArgumentError(_)) => (), + Error::Backtraced { source, .. } + if matches!( + &*source, + Error::Arrow(arrow_schema::ArrowError::InvalidArgumentError(_)) + ) => + { + () + } _ => panic!("expected arrow invalid arg error, got {:?}", e), }); From 630c6944bbf0d68fe0c80b286aeaa28a0ac873bc Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Wed, 23 Oct 2024 13:09:05 -0700 Subject: [PATCH 58/66] appease clippy --- kernel/src/transaction.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/kernel/src/transaction.rs b/kernel/src/transaction.rs index ca4d311c5..96c73211f 100644 --- a/kernel/src/transaction.rs +++ b/kernel/src/transaction.rs @@ -419,7 +419,7 @@ mod tests { Error::Arrow(arrow_schema::ArrowError::SchemaError(_)) ) => { - () + } _ => panic!("expected arrow schema error error, got {:?}", e), }); @@ -453,7 +453,7 @@ mod tests { Error::Arrow(arrow_schema::ArrowError::InvalidArgumentError(_)) ) => { - () + } _ => panic!("expected arrow invalid arg error, got {:?}", e), }); From f5530f98130e4518011416a4a1b8b86a4f250eae Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Wed, 23 Oct 2024 13:11:36 -0700 Subject: [PATCH 59/66] fmt --- kernel/src/transaction.rs | 10 ++-------- 1 file changed, 2 insertions(+), 8 deletions(-) diff --git a/kernel/src/transaction.rs b/kernel/src/transaction.rs index 96c73211f..90795959f 100644 --- a/kernel/src/transaction.rs +++ b/kernel/src/transaction.rs @@ -417,10 +417,7 @@ mod tests { if matches!( &*source, Error::Arrow(arrow_schema::ArrowError::SchemaError(_)) - ) => - { - - } + ) => {} _ => panic!("expected arrow schema error error, got {:?}", e), }); @@ -451,10 +448,7 @@ mod tests { if matches!( &*source, Error::Arrow(arrow_schema::ArrowError::InvalidArgumentError(_)) - ) => - { - - } + ) => {} _ => panic!("expected arrow invalid arg error, got {:?}", e), }); From d7ad2e66ce13ffb8a3f4e72a34b7481c1afb4f37 Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Wed, 23 Oct 2024 13:30:06 -0700 Subject: [PATCH 60/66] use column_* macros --- kernel/src/transaction.rs | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/kernel/src/transaction.rs b/kernel/src/transaction.rs index 90795959f..d29e46acc 100644 --- a/kernel/src/transaction.rs +++ b/kernel/src/transaction.rs @@ -5,7 +5,7 @@ use std::time::{SystemTime, UNIX_EPOCH}; use crate::actions::get_log_commit_info_schema; use crate::actions::COMMIT_INFO_NAME; use crate::error::Error; -use crate::expressions::{Scalar, StructData}; +use crate::expressions::{column_expr, column_name, Scalar, StructData}; use crate::path::ParsedLogPath; use crate::schema::{MapType, StructField, StructType}; use crate::snapshot::Snapshot; @@ -152,14 +152,14 @@ fn generate_commit_info( // one null field to create data that serializes as "operationParameters": {} Expression::literal(Scalar::Struct(StructData::try_new( vec![StructField::new( - "operation_parameter_int", + column_name!("operation_parameter_int").into_inner(), DataType::INTEGER, true, )], vec![Scalar::Null(DataType::INTEGER)], )?)), Expression::literal(format!("v{}", KERNEL_VERSION)), - Expression::column("engineCommitInfo"), + column_expr!("engineCommitInfo"), ]; let commit_info_expr = Expression::struct_from([Expression::struct_from(commit_info_exprs)]); @@ -186,7 +186,7 @@ fn generate_commit_info( .get_mut("operationParameters") .unwrap() .data_type = DataType::Struct(Box::new(StructType::new(vec![StructField::new( - "hack_operation_parameter_int", + column_name!("hack_operation_parameter_int").into_inner(), DataType::INTEGER, true, )]))); From 2141ecfcfb911bbd87838f21d27191f721dab71b Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Wed, 23 Oct 2024 14:56:39 -0700 Subject: [PATCH 61/66] Update kernel/src/engine/arrow_utils.rs Co-authored-by: Nick Lanham --- kernel/src/engine/arrow_utils.rs | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/kernel/src/engine/arrow_utils.rs b/kernel/src/engine/arrow_utils.rs index ba98b06c4..d8daba774 100644 --- a/kernel/src/engine/arrow_utils.rs +++ b/kernel/src/engine/arrow_utils.rs @@ -662,9 +662,8 @@ fn parse_json_impl(json_strings: &StringArray, schema: ArrowSchemaRef) -> DeltaR Ok(concat_batches(&schema, output.iter())?) } -/// write an arrow RecordBatch to a JSON string by appending to a buffer. -/// -/// TODO (zach): this should stream data to the JSON writer and output an iterator. +/// serialize an arrow RecordBatch to a JSON string by appending to a buffer. +// TODO (zach): this should stream data to the JSON writer and output an iterator. pub(crate) fn to_json_bytes( data: impl Iterator> + Send, ) -> DeltaResult> { From 75c976c631bf7ea975cdf9201436184ca184f2dd Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Wed, 23 Oct 2024 14:58:38 -0700 Subject: [PATCH 62/66] rename --- kernel/src/transaction.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/kernel/src/transaction.rs b/kernel/src/transaction.rs index d29e46acc..55d73d489 100644 --- a/kernel/src/transaction.rs +++ b/kernel/src/transaction.rs @@ -171,8 +171,8 @@ fn generate_commit_info( // HACK (part 2/2): we need to modify the commit info schema to match the expression above (a // struct with a single null int field). - let mut commit_info_schema = get_log_commit_info_schema().as_ref().clone(); - let commit_info_field = commit_info_schema + let mut commit_info_empty_struct_schema = get_log_commit_info_schema().as_ref().clone(); + let commit_info_field = commit_info_empty_struct_schema .fields .get_mut(COMMIT_INFO_NAME) .ok_or_else(|| Error::missing_column(COMMIT_INFO_NAME))?; @@ -195,7 +195,7 @@ fn generate_commit_info( let commit_info_evaluator = engine.get_expression_handler().get_evaluator( engine_commit_info_schema.into(), commit_info_expr, - commit_info_schema.into(), + commit_info_empty_struct_schema.into(), ); commit_info_evaluator.evaluate(engine_commit_info.as_ref()) From 4908174e2dd89069a3c56b1c7e3e261c8f2eb345 Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Wed, 23 Oct 2024 18:40:53 -0700 Subject: [PATCH 63/66] make generate_commit_info take & not Arc --- kernel/src/transaction.rs | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/kernel/src/transaction.rs b/kernel/src/transaction.rs index 55d73d489..4babc4178 100644 --- a/kernel/src/transaction.rs +++ b/kernel/src/transaction.rs @@ -66,12 +66,12 @@ impl Transaction { // note: only support commit_info right now (and it's required) let engine_commit_info = self .commit_info - .clone() + .as_ref() .ok_or_else(|| Error::MissingCommitInfo)?; let actions = Box::new(iter::once(generate_commit_info( engine, self.operation.as_deref(), - engine_commit_info, + engine_commit_info.as_ref(), )?)); // step two: set new commit version (current_version + 1) and path to write @@ -129,7 +129,7 @@ pub enum CommitResult { fn generate_commit_info( engine: &dyn Engine, operation: Option<&str>, - engine_commit_info: Arc, + engine_commit_info: &dyn EngineData, ) -> DeltaResult> { if engine_commit_info.length() != 1 { return Err(Error::InvalidCommitInfo(format!( @@ -198,7 +198,7 @@ fn generate_commit_info( commit_info_empty_struct_schema.into(), ); - commit_info_evaluator.evaluate(engine_commit_info.as_ref()) + commit_info_evaluator.evaluate(engine_commit_info) } #[cfg(test)] @@ -311,7 +311,7 @@ mod tests { let actions = generate_commit_info( &engine, Some("test operation"), - Arc::new(ArrowEngineData::new(commit_info_batch)), + &ArrowEngineData::new(commit_info_batch), )?; let expected = serde_json::json!({ @@ -371,7 +371,7 @@ mod tests { let actions = generate_commit_info( &engine, Some("test operation"), - Arc::new(ArrowEngineData::new(commit_info_batch)), + &ArrowEngineData::new(commit_info_batch), )?; let expected = serde_json::json!({ @@ -409,7 +409,7 @@ mod tests { let _ = generate_commit_info( &engine, Some("test operation"), - Arc::new(ArrowEngineData::new(commit_info_batch)), + &ArrowEngineData::new(commit_info_batch), ) .map_err(|e| match e { Error::Arrow(arrow_schema::ArrowError::SchemaError(_)) => (), @@ -440,7 +440,7 @@ mod tests { let _ = generate_commit_info( &engine, Some("test operation"), - Arc::new(ArrowEngineData::new(commit_info_batch)), + &ArrowEngineData::new(commit_info_batch), ) .map_err(|e| match e { Error::Arrow(arrow_schema::ArrowError::InvalidArgumentError(_)) => (), @@ -533,7 +533,7 @@ mod tests { let actions = generate_commit_info( &engine, Some("test operation"), - Arc::new(ArrowEngineData::new(commit_info_batch)), + &ArrowEngineData::new(commit_info_batch), )?; assert_empty_commit_info(actions, is_null)?; From 20ffd337301cc265fec398906ccfeb70428006ec Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Thu, 24 Oct 2024 12:39:38 -0700 Subject: [PATCH 64/66] fix unwrap --- kernel/src/transaction.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kernel/src/transaction.rs b/kernel/src/transaction.rs index 4babc4178..ac0fc54d9 100644 --- a/kernel/src/transaction.rs +++ b/kernel/src/transaction.rs @@ -184,7 +184,7 @@ fn generate_commit_info( commit_info_data_type .fields .get_mut("operationParameters") - .unwrap() + .ok_or_else(|| Error::missing_column("operationParameters"))? .data_type = DataType::Struct(Box::new(StructType::new(vec![StructField::new( column_name!("hack_operation_parameter_int").into_inner(), DataType::INTEGER, From 4aba873df556cf2409599c3c2b735efd3a0d919f Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Thu, 24 Oct 2024 16:18:20 -0700 Subject: [PATCH 65/66] address comments --- kernel/src/actions/mod.rs | 4 +++- kernel/src/path.rs | 3 ++- kernel/src/transaction.rs | 31 +++++++++++++++---------------- 3 files changed, 20 insertions(+), 18 deletions(-) diff --git a/kernel/src/actions/mod.rs b/kernel/src/actions/mod.rs index b205e2238..b23dd6511 100644 --- a/kernel/src/actions/mod.rs +++ b/kernel/src/actions/mod.rs @@ -159,9 +159,11 @@ impl Protocol { #[cfg_attr(not(feature = "developer-visibility"), visibility::make(pub(crate)))] struct CommitInfo { /// The time this logical file was created, as milliseconds since the epoch. + /// Read: optional, write: required (that is, kernel always writes). + /// If in-commit timestamps are enabled, this is always required. pub(crate) timestamp: Option, /// An arbitrary string that identifies the operation associated with this commit. This is - /// specified by the engine. + /// specified by the engine. Read: optional, write: required (that is, kernel alwarys writes). pub(crate) operation: Option, /// Map of arbitrary string key-value pairs that provide additional information about the /// operation. This is specified by the engine. For now this is always empty on write. diff --git a/kernel/src/path.rs b/kernel/src/path.rs index 8caf82c01..cc91a8ea9 100644 --- a/kernel/src/path.rs +++ b/kernel/src/path.rs @@ -190,7 +190,8 @@ impl ParsedLogPath { ) -> DeltaResult> { let filename = format!("{:020}.json", version); let location = table_root.join("_delta_log/")?.join(&filename)?; - let path = Self::try_from(location)?.expect("valid commit path"); + let path = Self::try_from(location)? + .ok_or_else(|| Error::internal_error("attempted to create invalid commit path"))?; if !path.is_commit() { return Err(Error::internal_error( "ParsedLogPath::new_commit created a non-commit path", diff --git a/kernel/src/transaction.rs b/kernel/src/transaction.rs index ac0fc54d9..423ad592b 100644 --- a/kernel/src/transaction.rs +++ b/kernel/src/transaction.rs @@ -5,9 +5,9 @@ use std::time::{SystemTime, UNIX_EPOCH}; use crate::actions::get_log_commit_info_schema; use crate::actions::COMMIT_INFO_NAME; use crate::error::Error; -use crate::expressions::{column_expr, column_name, Scalar, StructData}; +use crate::expressions::{column_expr, Scalar, StructData}; use crate::path::ParsedLogPath; -use crate::schema::{MapType, StructField, StructType}; +use crate::schema::{StructField, StructType}; use crate::snapshot::Snapshot; use crate::{DataType, DeltaResult, Engine, EngineData, Expression, Version}; @@ -140,7 +140,7 @@ fn generate_commit_info( let timestamp: i64 = SystemTime::now() .duration_since(UNIX_EPOCH) - .expect("time went backwards..?") + .map_err(|_| Error::generic("time went backwards"))? .as_millis() .try_into() .map_err(|_| Error::generic("milliseconds since unix_epoch exceeded i64 size"))?; @@ -152,7 +152,7 @@ fn generate_commit_info( // one null field to create data that serializes as "operationParameters": {} Expression::literal(Scalar::Struct(StructData::try_new( vec![StructField::new( - column_name!("operation_parameter_int").into_inner(), + "operation_parameter_int", DataType::INTEGER, true, )], @@ -162,16 +162,11 @@ fn generate_commit_info( column_expr!("engineCommitInfo"), ]; let commit_info_expr = Expression::struct_from([Expression::struct_from(commit_info_exprs)]); - - let engine_commit_info_schema = StructType::new(vec![StructField::new( - "engineCommitInfo", - MapType::new(DataType::STRING, DataType::STRING, true), - false, - )]); + let commit_info_schema = get_log_commit_info_schema().as_ref(); // HACK (part 2/2): we need to modify the commit info schema to match the expression above (a // struct with a single null int field). - let mut commit_info_empty_struct_schema = get_log_commit_info_schema().as_ref().clone(); + let mut commit_info_empty_struct_schema = commit_info_schema.clone(); let commit_info_field = commit_info_empty_struct_schema .fields .get_mut(COMMIT_INFO_NAME) @@ -181,15 +176,19 @@ fn generate_commit_info( "commit_info_field should be a struct", )); }; + let engine_commit_info_schema = + commit_info_data_type.project_as_struct(&["engineCommitInfo"])?; + let hack_data_type = DataType::Struct(Box::new(StructType::new(vec![StructField::new( + "hack_operation_parameter_int", + DataType::INTEGER, + true, + )]))); + commit_info_data_type .fields .get_mut("operationParameters") .ok_or_else(|| Error::missing_column("operationParameters"))? - .data_type = DataType::Struct(Box::new(StructType::new(vec![StructField::new( - column_name!("hack_operation_parameter_int").into_inner(), - DataType::INTEGER, - true, - )]))); + .data_type = hack_data_type; commit_info_field.data_type = DataType::Struct(commit_info_data_type); let commit_info_evaluator = engine.get_expression_handler().get_evaluator( From 1fc535e2e28625a2eb51ae60c14dae95b3ba8ea5 Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Fri, 25 Oct 2024 12:26:32 -0700 Subject: [PATCH 66/66] make it with_operation and with_commit_info --- kernel/src/transaction.rs | 6 ++++-- kernel/tests/write.rs | 21 ++++++++++++--------- 2 files changed, 16 insertions(+), 11 deletions(-) diff --git a/kernel/src/transaction.rs b/kernel/src/transaction.rs index 423ad592b..81b0f31f8 100644 --- a/kernel/src/transaction.rs +++ b/kernel/src/transaction.rs @@ -90,8 +90,9 @@ impl Transaction { /// Set the operation that this transaction is performing. This string will be persisted in the /// commit and visible to anyone who describes the table history. - pub fn operation(&mut self, operation: String) { + pub fn with_operation(mut self, operation: String) -> Self { self.operation = Some(operation); + self } /// WARNING: This is an unstable API and will likely change in the future. @@ -107,8 +108,9 @@ impl Transaction { /// that can either be `null` or contain an empty map. /// /// Any other columns in the data chunk are ignored. - pub fn commit_info(&mut self, commit_info: Box) { + pub fn with_commit_info(mut self, commit_info: Box) -> Self { self.commit_info = Some(commit_info.into()); + self } } diff --git a/kernel/tests/write.rs b/kernel/tests/write.rs index 00101f0b1..212b06cae 100644 --- a/kernel/tests/write.rs +++ b/kernel/tests/write.rs @@ -100,10 +100,7 @@ async fn test_commit_info() -> Result<(), Box> { )])); let table = create_table(store.clone(), table_location, schema, &[]).await?; - // create a transaction - let mut txn = table.new_transaction(&engine)?; - - // add commit info of the form {engineCommitInfo: Map { "engineInfo": "default engine" } } + // create commit info of the form {engineCommitInfo: Map { "engineInfo": "default engine" } } let commit_info_schema = Arc::new(ArrowSchema::new(vec![Field::new( "engineCommitInfo", ArrowDataType::Map( @@ -139,7 +136,11 @@ async fn test_commit_info() -> Result<(), Box> { let commit_info_batch = RecordBatch::try_new(commit_info_schema.clone(), vec![Arc::new(array)])?; - txn.commit_info(Box::new(ArrowEngineData::new(commit_info_batch))); + + // create a transaction + let txn = table + .new_transaction(&engine)? + .with_commit_info(Box::new(ArrowEngineData::new(commit_info_batch))); // commit! txn.commit(&engine)?; @@ -212,11 +213,12 @@ async fn test_invalid_commit_info() -> Result<(), Box> { let table = create_table(store.clone(), table_location, schema, &[]).await?; // empty commit info test - let mut txn = table.new_transaction(&engine)?; let commit_info_schema = Arc::new(ArrowSchema::empty()); let commit_info_batch = RecordBatch::new_empty(commit_info_schema.clone()); assert!(commit_info_batch.num_rows() == 0); - txn.commit_info(Box::new(ArrowEngineData::new(commit_info_batch))); + let txn = table + .new_transaction(&engine)? + .with_commit_info(Box::new(ArrowEngineData::new(commit_info_batch))); // commit! assert!(matches!( @@ -225,7 +227,6 @@ async fn test_invalid_commit_info() -> Result<(), Box> { )); // two-row commit info test - let mut txn = table.new_transaction(&engine)?; let commit_info_schema = Arc::new(ArrowSchema::new(vec![Field::new( "engineInfo", ArrowDataType::Utf8, @@ -239,7 +240,9 @@ async fn test_invalid_commit_info() -> Result<(), Box> { ]))], )?; - txn.commit_info(Box::new(ArrowEngineData::new(commit_info_batch))); + let txn = table + .new_transaction(&engine)? + .with_commit_info(Box::new(ArrowEngineData::new(commit_info_batch))); // commit! assert!(matches!(