diff --git a/crates/aws/helpers.rs b/crates/aws/helpers.rs index 1e6d88a340..60242d2117 100644 --- a/crates/aws/helpers.rs +++ b/crates/aws/helpers.rs @@ -39,7 +39,10 @@ pub async fn setup_s3_context() -> TestContext { config.insert("AWS_ACCESS_KEY_ID".to_owned(), "deltalake".to_owned()); config.insert("AWS_SECRET_ACCESS_KEY".to_owned(), "weloverust".to_owned()); config.insert("AWS_S3_LOCKING_PROVIDER".to_owned(), "dynamodb".to_owned()); - config.insert(constants::LOCK_TABLE_KEY_NAME.to_owned(), lock_table.clone()); + config.insert( + constants::LOCK_TABLE_KEY_NAME.to_owned(), + lock_table.clone(), + ); config.insert("AWS_ALLOW_HTTP".to_owned(), "TRUE".to_string()); TestContext { @@ -72,7 +75,7 @@ impl S3Cli { child.wait().unwrap(); } - pub fn rm_recurive(&self, prefix: &str, endpoint: &str) { + pub fn rm_recursive(&self, prefix: &str, endpoint: &str) { let mut child = Command::new("aws") .args([ "s3", @@ -140,7 +143,7 @@ struct S3 { impl Drop for S3 { fn drop(&mut self) { let cli = S3Cli::default(); - cli.rm_recurive(&self.uri, &self.endpoint); + cli.rm_recursive(&self.uri, &self.endpoint); cli.delete_table(&self.lock_table, &self.endpoint); } } diff --git a/crates/aws/src/credentials.rs b/crates/aws/src/credentials.rs index a362610564..9835549bb7 100644 --- a/crates/aws/src/credentials.rs +++ b/crates/aws/src/credentials.rs @@ -211,7 +211,7 @@ mod options_tests { } /// Generate a random session name for assuming IAM roles -fn assume_role_sessio_name() -> String { +fn assume_role_session_name() -> String { let now = chrono::Utc::now(); format!("delta-rs_{}", now.timestamp_millis()) @@ -256,7 +256,7 @@ fn assume_session_name(options: &StorageOptions) -> String { ) .cloned(); - assume_session.unwrap_or_else(assume_role_sessio_name) + assume_session.unwrap_or_else(assume_role_session_name) } /// Take a set of [StorageOptions] and produce an appropriate AWS SDK [SdkConfig] diff --git a/crates/aws/src/errors.rs b/crates/aws/src/errors.rs index 55f2a2d013..6053ef8343 100644 --- a/crates/aws/src/errors.rs +++ b/crates/aws/src/errors.rs @@ -128,7 +128,7 @@ impl From for LockClientError { fn from(err: PutItemError) -> Self { match err { PutItemError::ConditionalCheckFailedException(_) => { - unreachable!("error must be handled explicitely") + unreachable!("error must be handled explicitly") } PutItemError::ProvisionedThroughputExceededException(_) => { LockClientError::ProvisionedThroughputExceeded diff --git a/crates/aws/src/lib.rs b/crates/aws/src/lib.rs index bd076700ad..00b2460f16 100644 --- a/crates/aws/src/lib.rs +++ b/crates/aws/src/lib.rs @@ -249,7 +249,7 @@ impl DynamoDbLockClient { /// Transparently handles the case where that table already exists, so it's safe to call. /// After `create_table` operation is executed, the table state in DynamoDb is `creating`, and /// it's not immediately useable. This method does not wait for the table state to become - /// `active`, so transient failures might occurr when immediately using the lock client. + /// `active`, so transient failures might occur when immediately using the lock client. pub async fn try_create_lock_table(&self) -> Result { let attribute_definitions = vec![ AttributeDefinition::builder() diff --git a/crates/aws/src/logstore/dynamodb_logstore.rs b/crates/aws/src/logstore/dynamodb_logstore.rs index 2d1630e795..95da0f4365 100644 --- a/crates/aws/src/logstore/dynamodb_logstore.rs +++ b/crates/aws/src/logstore/dynamodb_logstore.rs @@ -329,7 +329,7 @@ pub enum RepairLogEntryResult { MovedFileAndFixedEntry, /// The database entry has been rewritten, but the file was already moved. FixedEntry, - /// Moved file, but the database entry was alrady updated. + /// Moved file, but the database entry was already updated. MovedFile, /// Both parts of the repair process where already carried. AlreadyCompleted, diff --git a/crates/azure/src/config.rs b/crates/azure/src/config.rs index d30272768e..360391cc08 100644 --- a/crates/azure/src/config.rs +++ b/crates/azure/src/config.rs @@ -124,7 +124,7 @@ impl AzureConfigHelper { .all(|key| self.config.contains_key(key) || self.env_config.contains_key(key)) } - /// Generate a cofiguration augmented with options from the environment + /// Generate a configuration augmented with options from the environment pub fn build(mut self) -> Result> { let mut has_credential = false; @@ -142,7 +142,7 @@ impl AzureConfigHelper { } } - // try partially avaialbe credentials augmented by environment + // try partially available credentials augmented by environment if !has_credential { for cred in &self.priority { if self.has_any_config(cred) && self.has_full_config_with_env(cred) { diff --git a/crates/core/src/delta_datafusion/mod.rs b/crates/core/src/delta_datafusion/mod.rs index a89aa710d9..c4c74886bd 100644 --- a/crates/core/src/delta_datafusion/mod.rs +++ b/crates/core/src/delta_datafusion/mod.rs @@ -118,7 +118,7 @@ impl From for DeltaTableError { } } -/// Convience trait for calling common methods on snapshot heirarchies +/// Convenience trait for calling common methods on snapshot hierarchies pub trait DataFusionMixins { /// The physical datafusion schema of a table fn arrow_schema(&self) -> DeltaResult; @@ -2659,7 +2659,7 @@ mod tests { #[tokio::test] async fn passes_sanity_checker_when_all_files_filtered() { // Run a query that filters out all files and sorts. - // Verify that it returns an empty set of rows without panicing. + // Verify that it returns an empty set of rows without panicking. // // Historically, we had a bug that caused us to emit a query plan with 0 partitions, which // datafusion rejected. diff --git a/crates/core/src/kernel/arrow/extract.rs b/crates/core/src/kernel/arrow/extract.rs index 1a0d2ad301..372c5168d4 100644 --- a/crates/core/src/kernel/arrow/extract.rs +++ b/crates/core/src/kernel/arrow/extract.rs @@ -1,4 +1,4 @@ -//! Utilties to extract columns from a record batch or nested / complex arrays. +//! Utilities to extract columns from a record batch or nested / complex arrays. use std::sync::Arc; @@ -70,7 +70,7 @@ pub(crate) fn extract_column<'a>( if let Some(next_path_step) = remaining_path_steps.next() { match child.data_type() { DataType::Map(_, _) => { - // NOTE a map has exatly one child, but we wnat to be agnostic of its name. + // NOTE a map has exactly one child, but we want to be agnostic of its name. // so we case the current array as map, and use the entries accessor. let maparr = cast_column_as::(path_step, &Some(child))?; if let Some(next_path) = remaining_path_steps.next() { diff --git a/crates/core/src/kernel/models/actions.rs b/crates/core/src/kernel/models/actions.rs index d825d5bec4..16a9f94991 100644 --- a/crates/core/src/kernel/models/actions.rs +++ b/crates/core/src/kernel/models/actions.rs @@ -188,19 +188,19 @@ impl Protocol { mut self, writer_features: impl IntoIterator>, ) -> Self { - let all_writer_feautures = writer_features + let all_writer_features = writer_features .into_iter() .map(|c| c.into()) .collect::>(); - if !all_writer_feautures.is_empty() { + if !all_writer_features.is_empty() { self.min_writer_version = 7; match self.writer_features { Some(mut features) => { - features.extend(all_writer_feautures); + features.extend(all_writer_features); self.writer_features = Some(features); } - None => self.writer_features = Some(all_writer_feautures), + None => self.writer_features = Some(all_writer_features), }; } self diff --git a/crates/core/src/kernel/models/schema.rs b/crates/core/src/kernel/models/schema.rs index bd76f0b3e9..c3343747c9 100644 --- a/crates/core/src/kernel/models/schema.rs +++ b/crates/core/src/kernel/models/schema.rs @@ -46,7 +46,7 @@ impl DataCheck for Invariant { } } -/// Trait to add convenince functions to struct type +/// Trait to add convenience functions to struct type pub trait StructTypeExt { /// Get all invariants in the schemas fn get_invariants(&self) -> Result, Error>; diff --git a/crates/core/src/kernel/snapshot/log_data.rs b/crates/core/src/kernel/snapshot/log_data.rs index 05d1790dc9..a5e40903b4 100644 --- a/crates/core/src/kernel/snapshot/log_data.rs +++ b/crates/core/src/kernel/snapshot/log_data.rs @@ -158,7 +158,7 @@ impl LogicalFile<'_> { /// An object store [`Path`] to the file. /// /// this tries to parse the file string and if that fails, it will return the string as is. - // TODO assert consisent handling of the paths encoding when reading log data so this logic can be removed. + // TODO assert consistent handling of the paths encoding when reading log data so this logic can be removed. pub fn object_store_path(&self) -> Path { let path = self.path(); // Try to preserve percent encoding if possible diff --git a/crates/core/src/kernel/snapshot/log_segment.rs b/crates/core/src/kernel/snapshot/log_segment.rs index 7906dd6bef..49d2b036b0 100644 --- a/crates/core/src/kernel/snapshot/log_segment.rs +++ b/crates/core/src/kernel/snapshot/log_segment.rs @@ -136,7 +136,7 @@ impl LogSegment { /// Try to create a new [`LogSegment`] from a slice of the log. /// - /// Ths will create a new [`LogSegment`] from the log with all relevant log files + /// This will create a new [`LogSegment`] from the log with all relevant log files /// starting at `start_version` and ending at `end_version`. pub async fn try_new_slice( table_root: &Path, @@ -190,7 +190,7 @@ impl LogSegment { Ok(()) } - /// Returns the highes commit version number in the log segment + /// Returns the highest commit version number in the log segment pub fn file_version(&self) -> Option { self.commit_files .iter() @@ -358,7 +358,7 @@ impl LogSegment { /// Advance the log segment with new commits /// /// Returns an iterator over record batches, as if the commits were read from the log. - /// The input commits should be in order in which they would be commited to the table. + /// The input commits should be in order in which they would be committed to the table. pub(super) fn advance<'a>( &mut self, commits: impl IntoIterator, diff --git a/crates/core/src/kernel/snapshot/mod.rs b/crates/core/src/kernel/snapshot/mod.rs index 55e413cea0..2938b3d3db 100644 --- a/crates/core/src/kernel/snapshot/mod.rs +++ b/crates/core/src/kernel/snapshot/mod.rs @@ -447,7 +447,7 @@ impl EagerSnapshot { schema_actions.insert(ActionType::Add); let checkpoint_stream = if new_slice.checkpoint_files.is_empty() { - // NOTE: we don't need to add the visitor relevant data here, as it is repüresented in teh state already + // NOTE: we don't need to add the visitor relevant data here, as it is repüresented in the state already futures::stream::iter(files.into_iter().map(Ok)).boxed() } else { let read_schema = diff --git a/crates/core/src/kernel/snapshot/replay.rs b/crates/core/src/kernel/snapshot/replay.rs index 22a9d733b8..171808f704 100644 --- a/crates/core/src/kernel/snapshot/replay.rs +++ b/crates/core/src/kernel/snapshot/replay.rs @@ -513,7 +513,7 @@ impl LogReplayScanner { self.seen.insert(seen_key(&r)); keep.push(false); } - // NOTE: there sould always be only one action per row. + // NOTE: there should always be only one action per row. (None, None) => debug!("WARNING: no action found for row"), (Some(a), Some(r)) => { debug!( diff --git a/crates/core/src/kernel/snapshot/visitors.rs b/crates/core/src/kernel/snapshot/visitors.rs index fc2234a4e2..7716fd96ef 100644 --- a/crates/core/src/kernel/snapshot/visitors.rs +++ b/crates/core/src/kernel/snapshot/visitors.rs @@ -144,7 +144,7 @@ mod tests { Some(Some(123)) ); - // test that only the first encountered txn ist tacked for every app id. + // test that only the first encountered txn is tacked for every app id. data_app.extend([None, Some("my-app")]); data_version.extend([None, Some(2)]); data_last_updated.extend([None, Some(124)]); diff --git a/crates/core/src/operations/cast/merge_schema.rs b/crates/core/src/operations/cast/merge_schema.rs index b57c29b2e8..c1ac1f00d8 100644 --- a/crates/core/src/operations/cast/merge_schema.rs +++ b/crates/core/src/operations/cast/merge_schema.rs @@ -336,9 +336,9 @@ fn merge_arrow_vec_fields( // field.try_merge f.set_metadata(right_field.metadata().clone()); - let mut field_matadata = f.metadata().clone(); - try_merge_metadata(&mut field_matadata, right_field.metadata())?; - f.set_metadata(field_matadata); + let mut field_metadata = f.metadata().clone(); + try_merge_metadata(&mut field_metadata, right_field.metadata())?; + f.set_metadata(field_metadata); Ok(f) } } diff --git a/crates/core/src/operations/constraints.rs b/crates/core/src/operations/constraints.rs index 2f9c172646..16a85c1e50 100644 --- a/crates/core/src/operations/constraints.rs +++ b/crates/core/src/operations/constraints.rs @@ -121,7 +121,7 @@ impl std::future::IntoFuture for ConstraintBuilder { let expr = this .expr - .ok_or_else(|| DeltaTableError::Generic("No Expresion provided".to_string()))?; + .ok_or_else(|| DeltaTableError::Generic("No Expression provided".to_string()))?; let mut metadata = this.snapshot.metadata().clone(); let configuration_key = format!("delta.constraints.{}", name); diff --git a/crates/core/src/operations/convert_to_delta.rs b/crates/core/src/operations/convert_to_delta.rs index 915ced6402..8dd8188200 100644 --- a/crates/core/src/operations/convert_to_delta.rs +++ b/crates/core/src/operations/convert_to_delta.rs @@ -75,7 +75,7 @@ impl From for DeltaTableError { } /// The partition strategy used by the Parquet table -/// Currently only hive-partitioning is supproted for Parquet paths +/// Currently only hive-partitioning is supported for Parquet paths #[non_exhaustive] #[derive(Default)] pub enum PartitionStrategy { @@ -187,7 +187,7 @@ impl ConvertToDeltaBuilder { } /// Specify the partition strategy of the Parquet table - /// Currently only hive-partitioning is supproted for Parquet paths + /// Currently only hive-partitioning is supported for Parquet paths pub fn with_partition_strategy(mut self, strategy: PartitionStrategy) -> Self { self.partition_strategy = strategy; self @@ -402,7 +402,7 @@ impl ConvertToDeltaBuilder { let mut arrow_schema = batch_builder.schema().as_ref().clone(); - // Arrow schema of Parquet files may have conflicting metatdata + // Arrow schema of Parquet files may have conflicting metadata // Since Arrow schema metadata is not used to generate Delta table schema, we set the metadata field to an empty HashMap arrow_schema.metadata = HashMap::new(); arrow_schemas.push(arrow_schema); diff --git a/crates/core/src/operations/create.rs b/crates/core/src/operations/create.rs index ea051c58ef..ae007a35ea 100644 --- a/crates/core/src/operations/create.rs +++ b/crates/core/src/operations/create.rs @@ -224,7 +224,7 @@ impl CreateBuilder { self } - /// Specify whether to raise an error if the table properties in the configuration are not TablePropertys + /// Specify whether to raise an error if the table properties in the configuration are not TableProperties pub fn with_raise_if_key_not_exists(mut self, raise_if_key_not_exists: bool) -> Self { self.raise_if_key_not_exists = raise_if_key_not_exists; self diff --git a/crates/core/src/operations/delete.rs b/crates/core/src/operations/delete.rs index c00d20a064..fd6ad70820 100644 --- a/crates/core/src/operations/delete.rs +++ b/crates/core/src/operations/delete.rs @@ -1,4 +1,4 @@ -//! Delete records from a Delta Table that statisfy a predicate +//! Delete records from a Delta Table that satisfy a predicate //! //! When a predicate is not provided then all records are deleted from the Delta //! Table. Otherwise a scan of the Delta table is performed to mark any files @@ -134,7 +134,7 @@ impl DeleteBuilder { self } - /// Additonal information to write to the commit + /// Additional information to write to the commit pub fn with_commit_properties(mut self, commit_properties: CommitProperties) -> Self { self.commit_properties = commit_properties; self @@ -184,7 +184,7 @@ impl ExtensionPlanner for DeleteMetricExtensionPlanner { } #[allow(clippy::too_many_arguments)] -async fn excute_non_empty_expr( +async fn execute_non_empty_expr( snapshot: &DeltaTableState, log_store: LogStoreRef, state: &SessionState, @@ -330,7 +330,7 @@ async fn execute( let mut actions = { let write_start = Instant::now(); - let add = excute_non_empty_expr( + let add = execute_non_empty_expr( &snapshot, log_store.clone(), &state, diff --git a/crates/core/src/operations/filesystem_check.rs b/crates/core/src/operations/filesystem_check.rs index 1e5f9d736e..708d430873 100644 --- a/crates/core/src/operations/filesystem_check.rs +++ b/crates/core/src/operations/filesystem_check.rs @@ -104,7 +104,7 @@ impl FileSystemCheckBuilder { self } - /// Additonal information to write to the commit + /// Additional information to write to the commit pub fn with_commit_properties(mut self, commit_properties: CommitProperties) -> Self { self.commit_properties = commit_properties; self diff --git a/crates/core/src/operations/merge/barrier.rs b/crates/core/src/operations/merge/barrier.rs index 09f58a6979..0df507826b 100644 --- a/crates/core/src/operations/merge/barrier.rs +++ b/crates/core/src/operations/merge/barrier.rs @@ -547,8 +547,8 @@ mod tests { } #[tokio::test] - async fn test_barrier_changing_indicies() { - // Validate implementation can handle different dictionary indicies between batches + async fn test_barrier_changing_indices() { + // Validate implementation can handle different dictionary indices between batches let schema = get_schema(); let mut batches = vec![]; diff --git a/crates/core/src/operations/merge/mod.rs b/crates/core/src/operations/merge/mod.rs index 6afa90fdd9..972d6b4291 100644 --- a/crates/core/src/operations/merge/mod.rs +++ b/crates/core/src/operations/merge/mod.rs @@ -1283,12 +1283,12 @@ async fn execute( LogicalPlanBuilder::from(plan).project(fields)?.build()? }; - let distrbute_expr = col(file_column.as_str()); + let distribute_expr = col(file_column.as_str()); let merge_barrier = LogicalPlan::Extension(Extension { node: Arc::new(MergeBarrier { input: new_columns.clone(), - expr: distrbute_expr, + expr: distribute_expr, file_column, }), }); diff --git a/crates/core/src/operations/optimize.rs b/crates/core/src/operations/optimize.rs index 5947aea906..f44f2cb5e6 100644 --- a/crates/core/src/operations/optimize.rs +++ b/crates/core/src/operations/optimize.rs @@ -267,7 +267,7 @@ impl<'a> OptimizeBuilder<'a> { self } - /// Additonal information to write to the commit + /// Additional information to write to the commit pub fn with_commit_properties(mut self, commit_properties: CommitProperties) -> Self { self.commit_properties = commit_properties; self diff --git a/crates/core/src/operations/transaction/conflict_checker.rs b/crates/core/src/operations/transaction/conflict_checker.rs index d163ba2f9b..4f9824f56a 100644 --- a/crates/core/src/operations/transaction/conflict_checker.rs +++ b/crates/core/src/operations/transaction/conflict_checker.rs @@ -411,7 +411,7 @@ impl<'a> ConflictChecker<'a> { ); if curr_read < win_read || win_write < curr_write { return Err(CommitConflictError::ProtocolChanged( - format!("reqired read/write {win_read}/{win_write}, current read/write {curr_read}/{curr_write}"), + format!("required read/write {win_read}/{win_write}, current read/write {curr_read}/{curr_write}"), )); }; } @@ -638,7 +638,7 @@ pub(super) fn can_downgrade_to_snapshot_isolation<'a>( match isolation_level { IsolationLevel::Serializable => !data_changed, IsolationLevel::WriteSerializable => !data_changed && !operation.changes_data(), - IsolationLevel::SnapshotIsolation => false, // this case should never happen, since spanpshot isolation canot be configured on table + IsolationLevel::SnapshotIsolation => false, // this case should never happen, since spanpshot isolation cannot be configured on table } } @@ -857,7 +857,7 @@ mod tests { setup_actions.push(file_part1); let result = execute_test( Some(setup_actions), - // filter matches neither exisiting nor added files + // filter matches neither existing nor added files Some(col("value").lt(lit::(0))), vec![file_part2], vec![file_part3], diff --git a/crates/core/src/operations/transaction/mod.rs b/crates/core/src/operations/transaction/mod.rs index c97a488dd3..bb173c40b7 100644 --- a/crates/core/src/operations/transaction/mod.rs +++ b/crates/core/src/operations/transaction/mod.rs @@ -1,9 +1,9 @@ //! Add a commit entry to the Delta Table. //! This module provides a unified interface for modifying commit behavior and attributes //! -//! [`CommitProperties`] provides an unified client interface for all Delta opeartions. +//! [`CommitProperties`] provides an unified client interface for all Delta operations. //! Internally this is used to initialize a [`CommitBuilder`]. -//! +//! //! For advanced use cases [`CommitBuilder`] can be used which allows //! finer control over the commit process. The builder can be converted //! into a future the yield either a [`PreparedCommit`] or a [`FinalizedCommit`]. @@ -14,64 +14,64 @@ //! //!
 //!                                          Client Interface
-//!        ┌─────────────────────────────┐                    
-//!        │      Commit Properties      │                    
-//!        │                             │                    
-//!        │ Public commit interface for │                    
-//!        │     all Delta Operations    │                    
-//!        │                             │                    
-//!        └─────────────┬───────────────┘                    
-//!                      │                                    
+//!        ┌─────────────────────────────┐
+//!        │      Commit Properties      │
+//!        │                             │
+//!        │ Public commit interface for │
+//!        │     all Delta Operations    │
+//!        │                             │
+//!        └─────────────┬───────────────┘
+//!                      │
 //! ─────────────────────┼────────────────────────────────────
-//!                      │                                    
+//!                      │
 //!                      ▼                  Advanced Interface
-//!        ┌─────────────────────────────┐                    
-//!        │       Commit Builder        │                    
-//!        │                             │                    
-//!        │   Advanced entry point for  │                    
-//!        │     creating a commit       │                    
-//!        └─────────────┬───────────────┘                    
-//!                      │                                    
-//!                      ▼                                    
-//!     ┌───────────────────────────────────┐                 
-//!     │                                   │                 
-//!     │ ┌───────────────────────────────┐ │                 
-//!     │ │        Prepared Commit        │ │                 
-//!     │ │                               │ │                 
-//!     │ │     Represents a temporary    │ │                 
-//!     │ │   commit marker written to    │ │                 
-//!     │ │           storage             │ │                 
-//!     │ └──────────────┬────────────────┘ │                 
-//!     │                │                  │                 
-//!     │                ▼                  │                 
-//!     │ ┌───────────────────────────────┐ │                 
-//!     │ │       Finalize Commit         │ │                 
-//!     │ │                               │ │                 
-//!     │ │   Convert the commit marker   │ │                 
-//!     │ │   to a commit using atomic    │ │                 
-//!     │ │         operations            │ │                 
-//!     │ │                               │ │                 
-//!     │ └───────────────────────────────┘ │                 
-//!     │                                   │                 
-//!     └────────────────┬──────────────────┘                 
-//!                      │                                    
-//!                      ▼                                    
-//!       ┌───────────────────────────────┐                   
-//!       │          Post Commit          │                   
-//!       │                               │                   
-//!       │ Commit that was materialized  │                   
-//!       │ to storage with post commit   │                   
-//!       │      hooks to be executed     │                   
-//!       └──────────────┬────────────────┘                 
-//!                      │                                    
-//!                      ▼    
-//!       ┌───────────────────────────────┐                   
-//!       │        Finalized Commit       │                   
-//!       │                               │                   
-//!       │ Commit that was materialized  │                   
-//!       │         to storage            │                   
-//!       │                               │                   
-//!       └───────────────────────────────┘           
+//!        ┌─────────────────────────────┐
+//!        │       Commit Builder        │
+//!        │                             │
+//!        │   Advanced entry point for  │
+//!        │     creating a commit       │
+//!        └─────────────┬───────────────┘
+//!                      │
+//!                      ▼
+//!     ┌───────────────────────────────────┐
+//!     │                                   │
+//!     │ ┌───────────────────────────────┐ │
+//!     │ │        Prepared Commit        │ │
+//!     │ │                               │ │
+//!     │ │     Represents a temporary    │ │
+//!     │ │   commit marker written to    │ │
+//!     │ │           storage             │ │
+//!     │ └──────────────┬────────────────┘ │
+//!     │                │                  │
+//!     │                ▼                  │
+//!     │ ┌───────────────────────────────┐ │
+//!     │ │       Finalize Commit         │ │
+//!     │ │                               │ │
+//!     │ │   Convert the commit marker   │ │
+//!     │ │   to a commit using atomic    │ │
+//!     │ │         operations            │ │
+//!     │ │                               │ │
+//!     │ └───────────────────────────────┘ │
+//!     │                                   │
+//!     └────────────────┬──────────────────┘
+//!                      │
+//!                      ▼
+//!       ┌───────────────────────────────┐
+//!       │          Post Commit          │
+//!       │                               │
+//!       │ Commit that was materialized  │
+//!       │ to storage with post commit   │
+//!       │      hooks to be executed     │
+//!       └──────────────┬────────────────┘
+//!                      │
+//!                      ▼
+//!       ┌───────────────────────────────┐
+//!       │        Finalized Commit       │
+//!       │                               │
+//!       │ Commit that was materialized  │
+//!       │         to storage            │
+//!       │                               │
+//!       └───────────────────────────────┘
 //!
use std::collections::HashMap; use std::sync::Arc; @@ -135,7 +135,7 @@ pub enum TransactionError { source: ObjectStoreError, }, - /// Error returned when a commit conflict ocurred + /// Error returned when a commit conflict occurred #[error("Failed to commit transaction: {0}")] CommitConflict(#[from] CommitConflictError), @@ -266,7 +266,7 @@ pub struct CommitData { } impl CommitData { - /// Create new data to be comitted + /// Create new data to be committed pub fn new( mut actions: Vec, operation: DeltaOperation, @@ -319,7 +319,7 @@ pub struct PostCommitHookProperties { #[derive(Clone, Debug)] /// End user facing interface to be used by operations on the table. -/// Enable controling commit behaviour and modifying metadata that is written during a commit. +/// Enable controlling commit behaviour and modifying metadata that is written during a commit. pub struct CommitProperties { pub(crate) app_metadata: HashMap, pub(crate) app_transaction: Vec, @@ -341,7 +341,7 @@ impl Default for CommitProperties { } impl CommitProperties { - /// Specify metadata the be comitted + /// Specify metadata the be committed pub fn with_metadata( mut self, metadata: impl IntoIterator, @@ -362,7 +362,7 @@ impl CommitProperties { self } - /// Add an additonal application transaction to the commit + /// Add an additional application transaction to the commit pub fn with_application_transaction(mut self, txn: Transaction) -> Self { self.app_transaction.push(txn); self @@ -446,7 +446,7 @@ impl<'a> CommitBuilder { self } - /// Propogate operation id to log store + /// Propagate operation id to log store pub fn with_operation_id(mut self, operation_id: Uuid) -> Self { self.operation_id = operation_id; self @@ -693,7 +693,7 @@ impl<'a> std::future::IntoFuture for PreparedCommit<'a> { pub struct PostCommit<'a> { /// The winning version number of the commit pub version: i64, - /// The data that was comitted to the log store + /// The data that was committed to the log store pub data: CommitData, create_checkpoint: bool, cleanup_expired_logs: Option, @@ -804,7 +804,7 @@ impl PostCommit<'_> { /// A commit that successfully completed pub struct FinalizedCommit { - /// The new table state after a commmit + /// The new table state after a commit pub snapshot: DeltaTableState, /// Version of the finalized commit @@ -812,7 +812,7 @@ pub struct FinalizedCommit { } impl FinalizedCommit { - /// The new table state after a commmit + /// The new table state after a commit pub fn snapshot(&self) -> DeltaTableState { self.snapshot.clone() } diff --git a/crates/core/src/operations/update.rs b/crates/core/src/operations/update.rs index 61623c8215..ef8e9f9784 100644 --- a/crates/core/src/operations/update.rs +++ b/crates/core/src/operations/update.rs @@ -1,4 +1,4 @@ -//! Update records from a Delta Table for records statisfy a predicate +//! Update records from a Delta Table for records satisfy a predicate //! //! When a predicate is not provided then all records are updated from the Delta //! Table. Otherwise a scan of the Delta table is performed to mark any files @@ -150,7 +150,7 @@ impl UpdateBuilder { self } - /// Perform an additional update expression during the operaton + /// Perform an additional update expression during the operation pub fn with_update, E: Into>( mut self, column: S, @@ -339,7 +339,7 @@ async fn execute( let df = DataFrame::new(state.clone(), plan); // Take advantage of how null counts are tracked in arrow arrays use the - // null count to track how many records do NOT statisfy the predicate. The + // null count to track how many records do NOT satisfy the predicate. The // count is then exposed through the metrics through the `UpdateCountExec` // execution plan let predicate_null = diff --git a/crates/core/src/operations/vacuum.rs b/crates/core/src/operations/vacuum.rs index 08caf10873..4e5c46589f 100644 --- a/crates/core/src/operations/vacuum.rs +++ b/crates/core/src/operations/vacuum.rs @@ -187,7 +187,7 @@ impl VacuumBuilder { self } - /// Determine which files can be deleted. Does not actually peform the deletion + /// Determine which files can be deleted. Does not actually perform the deletion async fn create_vacuum_plan(&self) -> Result { let min_retention = Duration::milliseconds( self.snapshot @@ -302,7 +302,7 @@ struct VacuumPlan { pub retention_check_enabled: bool, /// Default retention in milliseconds pub default_retention_millis: i64, - /// Overrided retention in milliseconds + /// Overridden retention in milliseconds pub specified_retention_millis: Option, } diff --git a/crates/core/src/operations/write.rs b/crates/core/src/operations/write.rs index 7004df74c4..8d17413c19 100644 --- a/crates/core/src/operations/write.rs +++ b/crates/core/src/operations/write.rs @@ -438,7 +438,7 @@ async fn write_execution_plan_with_predicate( let checker = if let Some(snapshot) = snapshot { DeltaDataChecker::new(snapshot) } else { - debug!("Using plan schema to derive generated columns, since no shapshot was provided. Implies first write."); + debug!("Using plan schema to derive generated columns, since no snapshot was provided. Implies first write."); let delta_schema: StructType = schema.as_ref().try_into()?; DeltaDataChecker::new_with_generated_columns( delta_schema.get_generated_columns().unwrap_or_default(), @@ -765,7 +765,7 @@ pub(crate) async fn execute_non_empty_expr_cdc( } } -// This should only be called wth a valid predicate +// This should only be called with a valid predicate #[allow(clippy::too_many_arguments)] async fn prepare_predicate_actions( predicate: Expr, @@ -2075,7 +2075,7 @@ mod tests { let table_logstore = table.log_store.clone(); let table_state = table.state.clone().unwrap(); - // An attempt to write records non comforming to predicate should fail + // An attempt to write records non conforming to predicate should fail let batch_fail = RecordBatch::try_new( Arc::clone(&schema), vec![ diff --git a/crates/core/src/protocol/mod.rs b/crates/core/src/protocol/mod.rs index 806678a496..463fb78e30 100644 --- a/crates/core/src/protocol/mod.rs +++ b/crates/core/src/protocol/mod.rs @@ -63,7 +63,7 @@ pub enum ProtocolError { source: parquet::errors::ParquetError, }, - /// Faild to serialize operation + /// Failed to serialize operation #[error("Failed to serialize operation: {source}")] SerializeOperation { #[from] @@ -819,7 +819,7 @@ mod tests { let info = serde_json::from_str::(raw); assert!(info.is_ok()); - // assert that commit info has no required filelds + // assert that commit info has no required fields let raw = "{}"; let info = serde_json::from_str::(raw); assert!(info.is_ok()); diff --git a/crates/core/src/storage/mod.rs b/crates/core/src/storage/mod.rs index 501b3fc609..a51d72b068 100644 --- a/crates/core/src/storage/mod.rs +++ b/crates/core/src/storage/mod.rs @@ -131,7 +131,7 @@ impl DeltaIOStorageBackend { } } - /// spawn taks on IO runtime + /// spawn tasks on IO runtime pub fn spawn_io_rt( &self, f: F, @@ -156,7 +156,7 @@ impl DeltaIOStorageBackend { .boxed() } - /// spawn taks on IO runtime + /// spawn tasks on IO runtime pub fn spawn_io_rt_from_to( &self, f: F, diff --git a/crates/core/src/writer/record_batch.rs b/crates/core/src/writer/record_batch.rs index a22d6f093a..96a8a215ed 100644 --- a/crates/core/src/writer/record_batch.rs +++ b/crates/core/src/writer/record_batch.rs @@ -967,7 +967,7 @@ mod tests { assert_eq!(table.version(), 0); // Hand-crafting the first RecordBatch to ensure that a write with non-nullable columns - // works properly before attepting the second write + // works properly before attempting the second write let arrow_schema = Arc::new(ArrowSchema::new(vec![ Field::new("id", DataType::Utf8, false), Field::new("value", DataType::Int32, true), diff --git a/crates/gcp/src/config.rs b/crates/gcp/src/config.rs index fbc99c7edd..b75e84e624 100644 --- a/crates/gcp/src/config.rs +++ b/crates/gcp/src/config.rs @@ -96,7 +96,7 @@ impl GcpConfigHelper { .all(|key| self.config.contains_key(key) || self.env_config.contains_key(key)) } - /// Generate a cofiguration augmented with options from the environment + /// Generate a configuration augmented with options from the environment pub fn build(mut self) -> Result> { let mut has_credential = false; @@ -110,7 +110,7 @@ impl GcpConfigHelper { } } - // try partially avaialbe credentials augmented by environment + // try partially available credentials augmented by environment if !has_credential { for cred in &self.priority { if self.has_any_config(cred) && self.has_full_config_with_env(cred) { diff --git a/crates/lakefs/src/client.rs b/crates/lakefs/src/client.rs index 1f229cfd6e..1d564d482f 100644 --- a/crates/lakefs/src/client.rs +++ b/crates/lakefs/src/client.rs @@ -34,7 +34,7 @@ pub struct LakeFSClient { /// configuration of the lakefs client config: LakeFSConfig, http_client: Client, - /// Holds the running delta lake operations, each operation propogates the operation ID into execution handler. + /// Holds the running delta lake operations, each operation propagates the operation ID into execution handler. transactions: DashMap, } diff --git a/crates/mount/src/config.rs b/crates/mount/src/config.rs index 79dbfc88d0..a3f2a3dd55 100644 --- a/crates/mount/src/config.rs +++ b/crates/mount/src/config.rs @@ -63,7 +63,7 @@ impl MountConfigHelper { }) } - /// Generate a cofiguration augmented with options from the environment + /// Generate a configuration augmented with options from the environment pub fn build(mut self) -> Result> { // Add keys from the environment to the configuration, as e.g. client configuration options. // NOTE We have to specifically configure omitting keys, since workload identity can diff --git a/crates/sql/src/logical_plan.rs b/crates/sql/src/logical_plan.rs index 27da52a96d..e43acd2d60 100644 --- a/crates/sql/src/logical_plan.rs +++ b/crates/sql/src/logical_plan.rs @@ -185,7 +185,7 @@ impl DescribeHistory { pub struct DescribeDetails { /// A reference to the table pub table: TableReference, - /// Schema for commit provenence information + /// Schema for commit provenance information pub schema: DFSchemaRef, }