From 8b3246b2a35a3d1bcfcc4194a14b532e73f9b3e5 Mon Sep 17 00:00:00 2001 From: Riccardo Busetti Date: Fri, 25 Jul 2025 10:04:09 +0200 Subject: [PATCH 01/18] feat(errors): Rework how errors work from the ground up --- etl/src/clients/bigquery.rs | 32 +- etl/src/conversions/event.rs | 3 +- etl/src/destination/bigquery.rs | 51 +-- etl/src/error.rs | 502 ++++++++++++++++++++++++++++++ etl/src/lib.rs | 4 + etl/src/pipeline.rs | 65 ++-- etl/src/replication/apply.rs | 182 +++-------- etl/src/replication/client.rs | 81 +---- etl/src/replication/common.rs | 4 +- etl/src/replication/table_sync.rs | 41 +-- etl/src/state/store/base.rs | 31 +- etl/src/state/store/memory.rs | 11 +- etl/src/state/store/notify.rs | 11 +- etl/src/state/store/postgres.rs | 33 +- etl/src/workers/apply.rs | 53 +--- etl/src/workers/base.rs | 17 +- etl/src/workers/pool.rs | 4 +- etl/src/workers/table_sync.rs | 59 +--- 18 files changed, 682 insertions(+), 502 deletions(-) create mode 100644 etl/src/error.rs diff --git a/etl/src/clients/bigquery.rs b/etl/src/clients/bigquery.rs index 394b70fcc..905646443 100644 --- a/etl/src/clients/bigquery.rs +++ b/etl/src/clients/bigquery.rs @@ -10,7 +10,7 @@ use gcp_bigquery_client::{ }; use postgres::schema::ColumnSchema; use std::fmt; -use thiserror::Error; +use crate::error::{ETLError, ETLResult, ErrorKind}; use tokio_postgres::types::Type; use tracing::info; @@ -53,7 +53,7 @@ impl fmt::Display for BigQueryOperationType { } /// Collection of row errors returned from BigQuery streaming operations. -#[derive(Debug, Error)] +#[derive(Debug)] pub struct RowErrors(pub Vec); impl fmt::Display for RowErrors { @@ -68,16 +68,6 @@ impl fmt::Display for RowErrors { } } -/// Errors that can occur when interacting with BigQuery. -#[derive(Debug, Error)] -pub enum BigQueryClientError { - #[error("An error occurred with BigQuery: {0}")] - BigQuery(#[from] BQError), - - #[error("One or multiple errors: {0}")] - AppendRowErrors(#[from] RowErrors), -} - /// A client for interacting with Google BigQuery. /// /// This client provides methods for managing tables, inserting data, @@ -95,7 +85,7 @@ impl BigQueryClient { pub async fn new_with_key_path( project_id: String, sa_key_path: &str, - ) -> Result { + ) -> ETLResult { let client = Client::from_service_account_key_file(sa_key_path).await?; Ok(BigQueryClient { project_id, client }) @@ -108,7 +98,7 @@ impl BigQueryClient { pub async fn new_with_key( project_id: String, sa_key: &str, - ) -> Result { + ) -> ETLResult { let sa_key = parse_service_account_key(sa_key).map_err(BQError::from)?; let client = Client::from_service_account_key(sa_key, false).await?; @@ -130,7 +120,7 @@ impl BigQueryClient { table_id: &str, column_schemas: &[ColumnSchema], max_staleness_mins: Option, - ) -> Result { + ) -> ETLResult { if self.table_exists(dataset_id, table_id).await? { return Ok(false); } @@ -148,7 +138,7 @@ impl BigQueryClient { table_id: &str, column_schemas: &[ColumnSchema], max_staleness_mins: Option, - ) -> Result<(), BigQueryClientError> { + ) -> ETLResult<()> { let full_table_name = self.full_table_name(dataset_id, table_id); let columns_spec = Self::create_columns_spec(column_schemas); @@ -172,7 +162,7 @@ impl BigQueryClient { &self, dataset_id: &str, table_id: &str, - ) -> Result<(), BigQueryClientError> { + ) -> ETLResult<()> { let full_table_name = self.full_table_name(dataset_id, table_id); info!("Truncating table {full_table_name} in BigQuery"); @@ -193,7 +183,7 @@ impl BigQueryClient { &self, dataset_id: &str, table_id: &str, - ) -> Result { + ) -> ETLResult { let table = self .client .table() @@ -216,7 +206,7 @@ impl BigQueryClient { table_id: String, table_descriptor: &TableDescriptor, table_rows: Vec, - ) -> Result<(), BigQueryClientError> { + ) -> ETLResult<()> { // We create a slice on table rows, which will be updated while the streaming progresses. // // Using a slice allows us to deallocate the vector only at the end of streaming, which leads @@ -242,7 +232,7 @@ impl BigQueryClient { if let Some(append_rows_response) = append_rows_stream.next().await { let append_rows_response = append_rows_response.map_err(BQError::from)?; if !append_rows_response.row_errors.is_empty() { - return Err(BigQueryClientError::AppendRowErrors(RowErrors( + return Err(ETLError::from(RowErrors( append_rows_response.row_errors, ))); } @@ -258,7 +248,7 @@ impl BigQueryClient { } /// Executes an SQL query and returns the result set. - pub async fn query(&self, request: QueryRequest) -> Result { + pub async fn query(&self, request: QueryRequest) -> ETLResult { let query_response = self.client.job().query(&self.project_id, request).await?; Ok(ResultSet::new_from_query_response(query_response)) diff --git a/etl/src/conversions/event.rs b/etl/src/conversions/event.rs index 9293fa53e..e41f9037d 100644 --- a/etl/src/conversions/event.rs +++ b/etl/src/conversions/event.rs @@ -2,7 +2,6 @@ use crate::conversions::Cell; use crate::conversions::table_row::TableRow; use crate::conversions::text::{FromTextError, TextFormatConverter}; use crate::schema::cache::SchemaCache; -use crate::state::store::base::StateStoreError; use core::str; use postgres::schema::{ColumnSchema, TableId, TableName, TableSchema}; use postgres::types::convert_type_oid_to_type; @@ -39,7 +38,7 @@ pub enum EventConversionError { Io(#[from] io::Error), #[error("An error occurred in the state store: {0}")] - StateStore(#[from] StateStoreError), + StateStore(#[from] crate::error::ETLError), } #[derive(Debug, Clone, PartialEq)] diff --git a/etl/src/destination/bigquery.rs b/etl/src/destination/bigquery.rs index 0c9741bfe..0dd826bc0 100644 --- a/etl/src/destination/bigquery.rs +++ b/etl/src/destination/bigquery.rs @@ -9,11 +9,12 @@ use tokio::sync::Mutex; use tokio_postgres::types::{PgLsn, Type}; use tracing::{debug, info, warn}; -use crate::clients::bigquery::{BigQueryClient, BigQueryClientError, BigQueryOperationType}; +use crate::clients::bigquery::{BigQueryClient, BigQueryOperationType}; use crate::conversions::Cell; use crate::conversions::event::{Event, TruncateEvent}; use crate::conversions::table_row::TableRow; use crate::destination::base::{Destination, DestinationError}; +use crate::error::{ETLError, ETLResult, ErrorKind}; use crate::schema::cache::SchemaCache; /// Table name for storing ETL table schema metadata in BigQuery. @@ -71,28 +72,6 @@ static ETL_TABLE_COLUMNS_COLUMNS: LazyLock> = LazyLock::new(|| ] }); -/// Errors that can occur when using [`BigQueryDestination`]. -/// -/// This error type covers BigQuery client failures, schema cache issues, -/// missing table schemas, and serialization problems. -#[derive(Debug, Error)] -pub enum BigQueryDestinationError { - /// Wraps errors from the underlying [`BigQueryClient`]. - #[error("An error occurred with the BigQuery client: {0}")] - BigQueryClient(#[from] BigQueryClientError), - - /// The requested table schema was not found in the schema cache. - #[error("The table schema for table id {0} was not found in the schema cache")] - MissingTableSchema(TableId), - - /// No schema cache has been injected into this destination instance. - #[error("The schema cache was not set on the destination")] - MissingSchemaCache, - - /// JSON serialization failed while processing table schema data. - #[error("Failed to serialize table schema: {0}")] - SerializationError(#[from] serde_json::Error), -} /// Internal state for [`BigQueryDestination`] wrapped in `Arc>`. /// @@ -109,7 +88,7 @@ impl Inner { /// Ensures the ETL metadata tables exist in BigQuery. /// /// Creates `etl_table_schemas` and `etl_table_columns` tables if they don't exist. - async fn ensure_schema_tables_exist(&self) -> Result<(), BigQueryDestinationError> { + async fn ensure_schema_tables_exist(&self) -> ETLResult<()> { // Create etl_table_schemas table - use ColumnSchema for compatibility self.client .create_table_if_missing( @@ -158,7 +137,7 @@ impl BigQueryDestination { dataset_id: String, sa_key: &str, max_staleness_mins: Option, - ) -> Result { + ) -> ETLResult { let client = BigQueryClient::new_with_key_path(project_id, sa_key).await?; let inner = Inner { client, @@ -181,7 +160,7 @@ impl BigQueryDestination { dataset_id: String, sa_key: &str, max_staleness_mins: Option, - ) -> Result { + ) -> ETLResult { let client = BigQueryClient::new_with_key(project_id, sa_key).await?; let inner = Inner { client, @@ -202,16 +181,16 @@ impl BigQueryDestination { inner: &I, table_id: &TableId, use_cdc_sequence_column: bool, - ) -> Result<(String, TableDescriptor), BigQueryDestinationError> { + ) -> ETLResult<(String, TableDescriptor)> { let schema_cache = inner .schema_cache .as_ref() - .ok_or(BigQueryDestinationError::MissingSchemaCache)? + .ok_or_else(|| ETLError::from((ErrorKind::ConfigError, "The schema cache was not set on the destination")))? .lock_inner() .await; let table_schema = schema_cache .get_table_schema_ref(table_id) - .ok_or(BigQueryDestinationError::MissingTableSchema(*table_id))?; + .ok_or_else(|| ETLError::from((ErrorKind::SchemaError, "Table schema not found in schema cache", format!("table_id: {}", table_id))))?; let table_id = table_schema.name.as_bigquery_table_id(); let table_descriptor = BigQueryClient::column_schemas_to_table_descriptor( @@ -228,7 +207,7 @@ impl BigQueryDestination { async fn write_table_schema( &self, table_schema: TableSchema, - ) -> Result<(), BigQueryDestinationError> { + ) -> ETLResult<()> { let mut inner = self.inner.lock().await; let dataset_id = inner.dataset_id.clone(); @@ -292,7 +271,7 @@ impl BigQueryDestination { /// Loads all table schemas from BigQuery ETL metadata tables. /// /// Reconstructs [`TableSchema`] objects by joining data from schema and column metadata tables. - async fn load_table_schemas(&self) -> Result, BigQueryDestinationError> { + async fn load_table_schemas(&self) -> ETLResult> { let inner = self.inner.lock().await; // First check if schema tables exist @@ -408,7 +387,7 @@ impl BigQueryDestination { &self, table_id: TableId, mut table_rows: Vec, - ) -> Result<(), BigQueryDestinationError> { + ) -> ETLResult<()> { let mut inner = self.inner.lock().await; // We do not use the sequence column for table rows, since we assume that table rows are always @@ -434,7 +413,7 @@ impl BigQueryDestination { /// /// Groups events by type, handles inserts/updates/deletes via streaming, and processes truncates separately. /// Adds sequence numbers to ensure proper ordering of events with the same system time. - async fn write_events(&self, events: Vec) -> Result<(), BigQueryDestinationError> { + async fn write_events(&self, events: Vec) -> ETLResult<()> { let mut event_iter = events.into_iter().peekable(); while event_iter.peek().is_some() { @@ -544,7 +523,7 @@ impl BigQueryDestination { async fn process_truncate_events( &self, truncate_events: Vec, - ) -> Result<(), BigQueryDestinationError> { + ) -> ETLResult<()> { let inner = self.inner.lock().await; for truncate_event in truncate_events { @@ -553,7 +532,7 @@ impl BigQueryDestination { let schema_cache = inner .schema_cache .as_ref() - .ok_or(BigQueryDestinationError::MissingSchemaCache)? + .ok_or_else(|| ETLError::from((ErrorKind::ConfigError, "The schema cache was not set on the destination")))? .lock_inner() .await; @@ -646,7 +625,7 @@ impl BigQueryDestination { /// /// Used when reconstructing schemas from BigQuery metadata tables. Falls back to `TEXT` for unknown types. #[allow(clippy::result_large_err)] - fn string_to_postgres_type(type_str: &str) -> Result { + fn string_to_postgres_type(type_str: &str) -> ETLResult { match type_str { "BOOL" => Ok(Type::BOOL), "CHAR" => Ok(Type::CHAR), diff --git a/etl/src/error.rs b/etl/src/error.rs new file mode 100644 index 000000000..d8c0377a7 --- /dev/null +++ b/etl/src/error.rs @@ -0,0 +1,502 @@ +use std::error; +use std::fmt; + +/// Main result type for ETL operations +pub type ETLResult = Result; + +/// Main error type for ETL operations, inspired by Redis error handling +pub struct ETLError { + pub repr: ErrorRepr, +} + +#[derive(Debug)] +pub enum ErrorRepr { + WithDescription(ErrorKind, &'static str), + WithDescriptionAndDetail(ErrorKind, &'static str, String), +} + +/// Comprehensive error kinds for ETL operations +#[derive(PartialEq, Eq, Copy, Clone, Debug)] +#[non_exhaustive] +pub enum ErrorKind { + /// Database connection failed + ConnectionFailed, + /// Authentication failed + AuthenticationFailed, + /// Query execution failed + QueryFailed, + /// Schema mismatch or validation error + SchemaError, + /// Data type conversion error + ConversionError, + /// Configuration error + ConfigError, + /// Pipeline execution error + PipelineError, + /// Resource constraint error (memory, disk, etc.) + ResourceError, + /// Network or I/O error + NetworkError, + /// Serialization/deserialization error + SerializationError, + /// Encryption/decryption error + EncryptionError, + /// Timeout error + TimeoutError, + /// Invalid state error + InvalidState, + /// Data validation error + ValidationError, + /// Worker/concurrency error + WorkerError, + /// Destination-specific error + DestinationError, + /// Source-specific error + SourceError, + /// Replication error + ReplicationError, +} + +impl ETLError { + /// Returns the kind of the error + pub fn kind(&self) -> ErrorKind { + match self.repr { + ErrorRepr::WithDescription(kind, _) | ErrorRepr::WithDescriptionAndDetail(kind, _, _) => kind, + } + } + + /// Returns the error detail if available + pub fn detail(&self) -> Option<&str> { + match self.repr { + ErrorRepr::WithDescriptionAndDetail(_, _, ref detail) => Some(detail.as_str()), + _ => None, + } + } + + /// Returns the name of the error category for display purposes + pub fn category(&self) -> &str { + match self.kind() { + ErrorKind::ConnectionFailed => "connection failed", + ErrorKind::AuthenticationFailed => "authentication failed", + ErrorKind::QueryFailed => "query failed", + ErrorKind::SchemaError => "schema error", + ErrorKind::ConversionError => "conversion error", + ErrorKind::ConfigError => "config error", + ErrorKind::PipelineError => "pipeline error", + ErrorKind::ResourceError => "resource error", + ErrorKind::NetworkError => "network error", + ErrorKind::SerializationError => "serialization error", + ErrorKind::EncryptionError => "encryption error", + ErrorKind::TimeoutError => "timeout error", + ErrorKind::InvalidState => "invalid state", + ErrorKind::ValidationError => "validation error", + ErrorKind::WorkerError => "worker error", + ErrorKind::DestinationError => "destination error", + ErrorKind::SourceError => "source error", + ErrorKind::ReplicationError => "replication error", + } + } + + /// Returns true if this is a connection-related error + pub fn is_connection_error(&self) -> bool { + matches!( + self.kind(), + ErrorKind::ConnectionFailed | ErrorKind::NetworkError | ErrorKind::TimeoutError + ) + } + + + /// Returns true if this is a data-related error + pub fn is_data_error(&self) -> bool { + matches!( + self.kind(), + ErrorKind::SchemaError | ErrorKind::ConversionError | ErrorKind::ValidationError + ) + } +} + +impl PartialEq for ETLError { + fn eq(&self, other: &ETLError) -> bool { + match (&self.repr, &other.repr) { + (ErrorRepr::WithDescription(kind_a, _), ErrorRepr::WithDescription(kind_b, _)) => kind_a == kind_b, + ( + ErrorRepr::WithDescriptionAndDetail(kind_a, _, _), + ErrorRepr::WithDescriptionAndDetail(kind_b, _, _), + ) => kind_a == kind_b, + _ => false, + } + } +} + +impl fmt::Display for ETLError { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> Result<(), fmt::Error> { + match self.repr { + ErrorRepr::WithDescription(kind, desc) => { + desc.fmt(f)?; + f.write_str(" - ")?; + fmt::Debug::fmt(&kind, f) + } + ErrorRepr::WithDescriptionAndDetail(kind, desc, ref detail) => { + desc.fmt(f)?; + f.write_str(" - ")?; + fmt::Debug::fmt(&kind, f)?; + f.write_str(": ")?; + detail.fmt(f) + } + } + } +} + +impl fmt::Debug for ETLError { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> Result<(), fmt::Error> { + fmt::Display::fmt(self, f) + } +} + +impl error::Error for ETLError { + fn description(&self) -> &str { + match self.repr { + ErrorRepr::WithDescription(_, desc) | ErrorRepr::WithDescriptionAndDetail(_, desc, _) => desc, + } + } +} + +// Ergonomic constructors following Redis pattern +impl From<(ErrorKind, &'static str)> for ETLError { + fn from((kind, desc): (ErrorKind, &'static str)) -> ETLError { + ETLError { + repr: ErrorRepr::WithDescription(kind, desc), + } + } +} + +impl From<(ErrorKind, &'static str, String)> for ETLError { + fn from((kind, desc, detail): (ErrorKind, &'static str, String)) -> ETLError { + ETLError { + repr: ErrorRepr::WithDescriptionAndDetail(kind, desc, detail), + } + } +} + +// Common standard library error conversions +impl From for ETLError { + fn from(err: std::io::Error) -> ETLError { + ETLError { + repr: ErrorRepr::WithDescriptionAndDetail( + ErrorKind::NetworkError, + "I/O error occurred", + err.to_string(), + ), + } + } +} + +impl From for ETLError { + fn from(err: serde_json::Error) -> ETLError { + ETLError { + repr: ErrorRepr::WithDescriptionAndDetail( + ErrorKind::SerializationError, + "JSON serialization failed", + err.to_string(), + ), + } + } +} + +impl From for ETLError { + fn from(err: std::str::Utf8Error) -> ETLError { + ETLError { + repr: ErrorRepr::WithDescriptionAndDetail( + ErrorKind::ConversionError, + "UTF-8 conversion failed", + err.to_string(), + ), + } + } +} + +impl From for ETLError { + fn from(err: std::string::FromUtf8Error) -> ETLError { + ETLError { + repr: ErrorRepr::WithDescriptionAndDetail( + ErrorKind::ConversionError, + "UTF-8 string conversion failed", + err.to_string(), + ), + } + } +} + +impl From for ETLError { + fn from(err: std::num::ParseIntError) -> ETLError { + ETLError { + repr: ErrorRepr::WithDescriptionAndDetail( + ErrorKind::ConversionError, + "Integer parsing failed", + err.to_string(), + ), + } + } +} + +impl From for ETLError { + fn from(err: std::num::ParseFloatError) -> ETLError { + ETLError { + repr: ErrorRepr::WithDescriptionAndDetail( + ErrorKind::ConversionError, + "Float parsing failed", + err.to_string(), + ), + } + } +} + +// PostgreSQL-specific error conversions +impl From for ETLError { + fn from(err: tokio_postgres::Error) -> ETLError { + let kind = if err.code().is_some() { + ErrorKind::QueryFailed + } else { + ErrorKind::ConnectionFailed + }; + + ETLError { + repr: ErrorRepr::WithDescriptionAndDetail( + kind, + "PostgreSQL client operation failed", + err.to_string(), + ), + } + } +} + +impl From for ETLError { + fn from(err: rustls::Error) -> ETLError { + ETLError { + repr: ErrorRepr::WithDescriptionAndDetail( + ErrorKind::EncryptionError, + "TLS configuration failed", + err.to_string(), + ), + } + } +} + +// Tokio watch error conversion +impl From> for ETLError { + fn from(err: tokio::sync::watch::error::SendError<()>) -> ETLError { + ETLError { + repr: ErrorRepr::WithDescriptionAndDetail( + ErrorKind::WorkerError, + "Worker shutdown failed", + err.to_string(), + ), + } + } +} + +// Tokio sync error conversions +impl From for ETLError { + fn from(err: tokio::sync::AcquireError) -> ETLError { + ETLError { + repr: ErrorRepr::WithDescriptionAndDetail( + ErrorKind::ResourceError, + "Failed to acquire permit", + err.to_string(), + ), + } + } +} + +// SQLx error conversion +impl From for ETLError { + fn from(err: sqlx::Error) -> ETLError { + let kind = match &err { + sqlx::Error::Database(_) => ErrorKind::QueryFailed, + sqlx::Error::Io(_) => ErrorKind::NetworkError, + sqlx::Error::PoolClosed | sqlx::Error::PoolTimedOut => ErrorKind::ConnectionFailed, + _ => ErrorKind::QueryFailed, + }; + + ETLError { + repr: ErrorRepr::WithDescriptionAndDetail( + kind, + "Database operation failed", + err.to_string(), + ), + } + } +} + +// BigQuery error conversions (feature-gated) +#[cfg(feature = "bigquery")] +impl From for ETLError { + fn from(err: gcp_bigquery_client::error::BQError) -> ETLError { + let kind = match &err { + gcp_bigquery_client::error::BQError::RequestError(_) => ErrorKind::NetworkError, + gcp_bigquery_client::error::BQError::ResponseError { .. } => ErrorKind::QueryFailed, + _ => ErrorKind::DestinationError, + }; + + ETLError { + repr: ErrorRepr::WithDescriptionAndDetail( + kind, + "BigQuery operation failed", + err.to_string(), + ), + } + } +} + +#[cfg(feature = "bigquery")] +impl From for ETLError { + fn from(err: crate::clients::bigquery::RowErrors) -> ETLError { + ETLError { + repr: ErrorRepr::WithDescriptionAndDetail( + ErrorKind::DestinationError, + "BigQuery row errors", + err.to_string(), + ), + } + } +} + +// Additional error type conversions for destination and stream errors +impl From for ETLError { + fn from(err: crate::destination::base::DestinationError) -> ETLError { + ETLError { + repr: ErrorRepr::WithDescriptionAndDetail( + ErrorKind::DestinationError, + "Destination operation failed", + err.to_string(), + ), + } + } +} + +impl From for ETLError { + fn from(err: crate::replication::stream::TableCopyStreamError) -> ETLError { + ETLError { + repr: ErrorRepr::WithDescriptionAndDetail( + ErrorKind::ReplicationError, + "Table copy stream operation failed", + err.to_string(), + ), + } + } +} + +impl From for ETLError { + fn from(err: crate::replication::stream::EventsStreamError) -> ETLError { + ETLError { + repr: ErrorRepr::WithDescriptionAndDetail( + ErrorKind::ReplicationError, + "Events stream operation failed", + err.to_string(), + ), + } + } +} + +impl From for ETLError { + fn from(err: crate::conversions::event::EventConversionError) -> ETLError { + ETLError { + repr: ErrorRepr::WithDescriptionAndDetail( + ErrorKind::ConversionError, + "Event conversion failed", + err.to_string(), + ), + } + } +} + +impl From for ETLError { + fn from(err: crate::replication::slot::SlotError) -> ETLError { + ETLError { + repr: ErrorRepr::WithDescriptionAndDetail( + ErrorKind::ReplicationError, + "Replication slot operation failed", + err.to_string(), + ), + } + } +} + +// Helpful macros for error creation (following Redis pattern) +#[macro_export] +macro_rules! etl_error { + ($kind:expr, $desc:expr) => { + ETLError::from(($kind, $desc)) + }; + ($kind:expr, $desc:expr, $detail:expr) => { + ETLError::from(($kind, $desc, $detail.to_string())) + }; +} + +#[macro_export] +macro_rules! bail { + ($kind:expr, $desc:expr) => { + return Err(etl_error!($kind, $desc)) + }; + ($kind:expr, $desc:expr, $detail:expr) => { + return Err(etl_error!($kind, $desc, $detail)) + }; +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_error_creation() { + let err = ETLError::from((ErrorKind::ConnectionFailed, "Database connection failed")); + assert_eq!(err.kind(), ErrorKind::ConnectionFailed); + assert_eq!(err.category(), "connection failed"); + assert!(err.is_connection_error()); + } + + #[test] + fn test_error_with_detail() { + let err = ETLError::from(( + ErrorKind::QueryFailed, + "SQL query execution failed", + "Table 'users' doesn't exist".to_string(), + )); + assert_eq!(err.kind(), ErrorKind::QueryFailed); + assert_eq!(err.detail(), Some("Table 'users' doesn't exist")); + } + + #[test] + fn test_from_io_error() { + let io_err = std::io::Error::new(std::io::ErrorKind::PermissionDenied, "Access denied"); + let etl_err = ETLError::from(io_err); + assert_eq!(etl_err.kind(), ErrorKind::NetworkError); + assert!(etl_err.detail().unwrap().contains("Access denied")); + } + + #[test] + fn test_macro_usage() { + let err = etl_error!(ErrorKind::ValidationError, "Invalid data format"); + assert_eq!(err.kind(), ErrorKind::ValidationError); + + let err_with_detail = etl_error!( + ErrorKind::ConversionError, + "Type conversion failed", + "Cannot convert string to integer: 'abc'" + ); + assert_eq!(err_with_detail.kind(), ErrorKind::ConversionError); + assert!(err_with_detail.detail().unwrap().contains("Cannot convert")); + } + + #[test] + fn test_error_categories() { + let connection_err = ETLError::from((ErrorKind::ConnectionFailed, "Connection failed")); + let data_err = ETLError::from((ErrorKind::SchemaError, "Schema mismatch")); + + assert!(connection_err.is_connection_error()); + assert!(!connection_err.is_data_error()); + + assert!(!data_err.is_connection_error()); + assert!(data_err.is_data_error()); + } +} \ No newline at end of file diff --git a/etl/src/lib.rs b/etl/src/lib.rs index ad6f32a24..e47a789dd 100644 --- a/etl/src/lib.rs +++ b/etl/src/lib.rs @@ -3,6 +3,7 @@ pub mod concurrency; pub mod conversions; pub mod destination; pub mod encryption; +pub mod error; #[cfg(feature = "failpoints")] pub mod failpoints; pub mod pipeline; @@ -12,3 +13,6 @@ pub mod state; #[cfg(any(test, feature = "test-utils"))] pub mod test_utils; pub mod workers; + +// Re-export main error types for convenience +pub use error::{ETLError, ETLResult, ErrorKind}; diff --git a/etl/src/pipeline.rs b/etl/src/pipeline.rs index 651d583ec..eb6a3de9e 100644 --- a/etl/src/pipeline.rs +++ b/etl/src/pipeline.rs @@ -1,44 +1,19 @@ use config::shared::PipelineConfig; use std::sync::Arc; -use thiserror::Error; -use tokio::sync::{Semaphore, watch}; +use tokio::sync::Semaphore; use tracing::{error, info}; use crate::concurrency::shutdown::{ShutdownTx, create_shutdown_channel}; -use crate::destination::base::{Destination, DestinationError}; -use crate::replication::client::{PgReplicationClient, PgReplicationError}; +use crate::destination::base::Destination; +use crate::error::{ETLError, ETLResult, ErrorKind}; +use crate::replication::client::PgReplicationClient; use crate::schema::cache::SchemaCache; -use crate::state::store::base::{StateStore, StateStoreError}; +use crate::state::store::base::StateStore; use crate::state::table::TableReplicationPhase; -use crate::workers::apply::{ApplyWorker, ApplyWorkerError, ApplyWorkerHandle}; -use crate::workers::base::{Worker, WorkerHandle, WorkerWaitErrors}; +use crate::workers::apply::{ApplyWorker, ApplyWorkerHandle}; +use crate::workers::base::{Worker, WorkerHandle}; use crate::workers::pool::TableSyncWorkerPool; -#[derive(Debug, Error)] -pub enum PipelineError { - #[error("One or more workers failed: {0}")] - OneOrMoreWorkersFailed(WorkerWaitErrors), - - #[error("PostgreSQL replication operation failed: {0}")] - PgReplicationClient(#[from] PgReplicationError), - - #[error("Apply worker failed to start in the pipeline: {0}")] - ApplyWorkerFailedOnStart(#[from] ApplyWorkerError), - - #[error("An error happened in the state store: {0}")] - StateStore(#[from] StateStoreError), - - #[error("An error occurred in the destination: {0}")] - Destination(#[from] DestinationError), - - #[error( - "An error occurred while shutting down the pipeline, likely because no workers are running: {0}" - )] - ShutdownFailed(#[from] watch::error::SendError<()>), - - #[error("The publication '{0}' does not exist in the database")] - MissingPublication(String), -} #[derive(Debug)] enum PipelineWorkers { @@ -94,7 +69,7 @@ where self.shutdown_tx.clone() } - pub async fn start(&mut self) -> Result<(), PipelineError> { + pub async fn start(&mut self) -> ETLResult<()> { info!( "starting pipeline for publication '{}' with id {}", self.config.publication_name, self.id @@ -144,7 +119,7 @@ where Ok(()) } - async fn prepare_schema_cache(&self, schema_cache: &SchemaCache) -> Result<(), PipelineError> { + async fn prepare_schema_cache(&self, schema_cache: &SchemaCache) -> ETLResult<()> { // We initialize the schema cache, which is local to a pipeline, and we try to load existing // schemas that were previously stored at the destination (if any). let table_schemas = self.destination.load_table_schemas().await?; @@ -156,7 +131,7 @@ where async fn initialize_table_states( &self, replication_client: &PgReplicationClient, - ) -> Result<(), PipelineError> { + ) -> ETLResult<()> { info!( "initializing table states for tables in publication '{}'", self.config.publication_name @@ -171,9 +146,11 @@ where "publication '{}' does not exist in the database", self.config.publication_name ); - return Err(PipelineError::MissingPublication( - self.config.publication_name.clone(), - )); + return Err(ETLError::from(( + ErrorKind::ConfigError, + "Missing publication", + format!("The publication '{}' does not exist in the database", self.config.publication_name) + ))); } let table_ids = replication_client @@ -193,7 +170,7 @@ where Ok(()) } - pub async fn wait(self) -> Result<(), PipelineError> { + pub async fn wait(self) -> ETLResult<()> { let PipelineWorkers::Started { apply_worker, pool } = self.workers else { info!("pipeline was not started, nothing to wait for"); @@ -239,8 +216,10 @@ where } if !errors.is_empty() { - return Err(PipelineError::OneOrMoreWorkersFailed(WorkerWaitErrors( - errors, + return Err(ETLError::from(( + ErrorKind::WorkerError, + "One or more workers failed", + format!("Workers failed with {} errors", errors.len()) ))); } @@ -248,7 +227,7 @@ where } #[allow(clippy::result_large_err)] - pub fn shutdown(&self) -> Result<(), PipelineError> { + pub fn shutdown(&self) -> ETLResult<()> { info!("trying to shut down the pipeline"); self.shutdown_tx.shutdown()?; info!("shut down signal successfully sent to all workers"); @@ -256,7 +235,7 @@ where Ok(()) } - pub async fn shutdown_and_wait(self) -> Result<(), PipelineError> { + pub async fn shutdown_and_wait(self) -> ETLResult<()> { self.shutdown()?; self.wait().await } diff --git a/etl/src/replication/apply.rs b/etl/src/replication/apply.rs index dea5e3109..9320e5536 100644 --- a/etl/src/replication/apply.rs +++ b/etl/src/replication/apply.rs @@ -1,14 +1,13 @@ use crate::concurrency::shutdown::ShutdownRx; -use crate::conversions::event::{Event, EventConversionError, EventType, convert_message_to_event}; -use crate::destination::base::{Destination, DestinationError}; +use crate::conversions::event::{Event, EventType, convert_message_to_event}; +use crate::destination::base::Destination; +use crate::error::{ETLError, ErrorKind, ErrorRepr}; use crate::pipeline::PipelineId; -use crate::replication::client::{PgReplicationClient, PgReplicationError}; -use crate::replication::slot::{SlotError, get_slot_name}; -use crate::replication::stream::{EventsStream, EventsStreamError}; +use crate::replication::client::PgReplicationClient; +use crate::replication::slot::get_slot_name; +use crate::replication::stream::EventsStream; use crate::schema::cache::SchemaCache; -use crate::workers::apply::ApplyWorkerHookError; use crate::workers::base::WorkerType; -use crate::workers::table_sync::TableSyncWorkerHookError; use crate::concurrency::signal::SignalRx; use config::shared::PipelineConfig; @@ -20,66 +19,14 @@ use std::future::{Future, pending}; use std::pin::Pin; use std::sync::Arc; use std::time::{Duration, Instant}; -use thiserror::Error; use tokio::pin; use tokio_postgres::types::PgLsn; -use tracing::{debug, error, info}; +use tracing::{debug, info}; /// The amount of milliseconds that pass between one refresh and the other of the system, in case no /// events or shutdown signal are received. const REFRESH_INTERVAL: Duration = Duration::from_millis(1000); -// TODO: figure out how to break the cycle and remove `Box`. -#[derive(Debug, Error)] -pub enum ApplyLoopError { - #[error("Apply worker hook operation failed: {0}")] - ApplyWorkerHook(Box), - - #[error("Table sync worker hook operation failed: {0}")] - TableSyncWorkerHook(Box), - - #[error("A Postgres replication error occurred in the apply loop: {0}")] - PgReplication(#[from] PgReplicationError), - - #[error("An error occurred while streaming logical replication changes in the apply loop: {0}")] - LogicalReplicationStreamFailed(#[from] EventsStreamError), - - #[error("Could not generate slot name in the apply loop: {0}")] - Slot(#[from] SlotError), - - #[error("An error occurred while building an event from a message in the apply loop: {0}")] - EventConversion(#[from] EventConversionError), - - #[error("An error occurred when interacting with the destination in the apply loop: {0}")] - Destination(#[from] DestinationError), - - #[error("A transaction should have started for the action ({0}) to be performed")] - InvalidTransaction(String), - - #[error("Incorrect commit LSN {0} in COMMIT message (expected {1})")] - InvalidCommitLsn(PgLsn, PgLsn), - - #[error("An invalid event {0} was received (expected {1})")] - InvalidEvent(EventType, EventType), - - #[error("The table schema for table {0} was not found in the cache")] - MissingTableSchema(TableId), - - #[error("The received table schema doesn't match the table schema loaded during table sync")] - MismatchedTableSchema, -} - -impl From for ApplyLoopError { - fn from(err: ApplyWorkerHookError) -> Self { - ApplyLoopError::ApplyWorkerHook(Box::new(err)) - } -} - -impl From for ApplyLoopError { - fn from(err: TableSyncWorkerHookError) -> Self { - ApplyLoopError::TableSyncWorkerHook(Box::new(err)) - } -} #[derive(Debug, Copy, Clone)] pub enum ApplyLoopResult { @@ -87,7 +34,7 @@ pub enum ApplyLoopResult { } pub trait ApplyLoopHook { - type Error: Into; + type Error: Into; fn before_loop( &self, @@ -262,11 +209,11 @@ pub async fn start_apply_loop( hook: T, mut shutdown_rx: ShutdownRx, mut force_syncing_tables_rx: Option, -) -> Result +) -> Result where D: Destination + Clone + Send + 'static, T: ApplyLoopHook, - ApplyLoopError: From<::Error>, + ETLError: From<::Error>, { info!( "starting apply loop in worker '{:?}' from lsn {}", @@ -389,11 +336,11 @@ async fn handle_replication_message_batch( hook: &T, max_batch_size: usize, max_batch_fill_duration: Duration, -) -> Result +) -> Result where D: Destination + Clone + Send + 'static, T: ApplyLoopHook, - ApplyLoopError: From<::Error>, + ETLError: From<::Error>, { let result = handle_replication_message(state, events_stream, message, schema_cache, hook).await?; @@ -425,11 +372,11 @@ async fn try_send_batch( hook: &T, max_batch_size: usize, max_batch_fill_duration: Duration, -) -> Result +) -> Result where D: Destination + Clone + Send + 'static, T: ApplyLoopHook, - ApplyLoopError: From<::Error>, + ETLError: From<::Error>, { let elapsed = state.last_batch_send_time.elapsed(); // `elapsed` could be zero in case current time is earlier than `last_batch_send_time`. @@ -508,10 +455,10 @@ async fn handle_replication_message( message: ReplicationMessage, schema_cache: &SchemaCache, hook: &T, -) -> Result +) -> Result where T: ApplyLoopHook, - ApplyLoopError: From<::Error>, + ETLError: From<::Error>, { match message { ReplicationMessage::XLogData(message) => { @@ -565,10 +512,10 @@ async fn handle_logical_replication_message( message: LogicalReplicationMessage, schema_cache: &SchemaCache, hook: &T, -) -> Result +) -> Result where T: ApplyLoopHook, - ApplyLoopError: From<::Error>, + ETLError: From<::Error>, { // We perform the conversion of the message to our own event format which is used downstream // by the destination. @@ -616,7 +563,7 @@ where fn get_commit_lsn( state: &ApplyLoopState, message: &LogicalReplicationMessage, -) -> Result { +) -> Result { // If we are in a `Begin` message, the `commit_lsn` is the `final_lsn` of the payload, in all the // other cases we read the `remote_final_lsn` which should be always set in case we are within or // at the end of a transaction (meaning that the event type is different from `Begin`). @@ -625,7 +572,7 @@ fn get_commit_lsn( } else { state .remote_final_lsn - .ok_or_else(|| ApplyLoopError::InvalidTransaction("get_commit_lsn".to_owned())) + .ok_or_else(|| ETLError { repr: ErrorRepr::WithDescriptionAndDetail(ErrorKind::InvalidState, "Invalid transaction", "A transaction should have started for get_commit_lsn to be performed".to_string()) }) } } @@ -633,9 +580,9 @@ async fn handle_begin_message( state: &mut ApplyLoopState, event: Event, message: &protocol::BeginBody, -) -> Result { +) -> Result { let EventType::Begin = event.event_type() else { - return Err(ApplyLoopError::InvalidEvent(event.into(), EventType::Begin)); + return Err(ETLError { repr: ErrorRepr::WithDescriptionAndDetail(ErrorKind::ValidationError, "Invalid event", format!("An invalid event {:?} was received (expected {:?})", event, EventType::Begin)) }); }; // We track the final lsn of this transaction, which should be equal to the `commit_lsn` of the @@ -656,25 +603,20 @@ async fn handle_commit_message( event: Event, message: &protocol::CommitBody, hook: &T, -) -> Result +) -> Result where T: ApplyLoopHook, - ApplyLoopError: From<::Error>, + ETLError: From<::Error>, { let EventType::Commit = event.event_type() else { - return Err(ApplyLoopError::InvalidEvent( - event.into(), - EventType::Commit, - )); + return Err(ETLError { repr: ErrorRepr::WithDescriptionAndDetail(ErrorKind::ValidationError, "Invalid event", format!("An invalid event {:?} was received (expected {:?})", event, EventType::Commit)) }); }; // We take the LSN that belongs to the current transaction, however, if there is no // LSN, it means that a `Begin` message was not received before this `Commit` which means // we are in an inconsistent state. let Some(remote_final_lsn) = state.remote_final_lsn.take() else { - return Err(ApplyLoopError::InvalidTransaction( - "handle_commit_message".to_owned(), - )); + return Err(ETLError { repr: ErrorRepr::WithDescriptionAndDetail(ErrorKind::InvalidState, "Invalid transaction", "A transaction should have started for handle_commit_message to be performed".to_string()) }); }; // If the commit lsn of the message is different from the remote final lsn, it means that the @@ -682,10 +624,7 @@ where // we want to bail assuming we are in an inconsistent state. let commit_lsn = PgLsn::from(message.commit_lsn()); if commit_lsn != remote_final_lsn { - return Err(ApplyLoopError::InvalidCommitLsn( - commit_lsn, - remote_final_lsn, - )); + return Err(ETLError { repr: ErrorRepr::WithDescriptionAndDetail(ErrorKind::ValidationError, "Invalid commit LSN", format!("Incorrect commit LSN {} in COMMIT message (expected {})", commit_lsn, remote_final_lsn)) }); } let end_lsn = PgLsn::from(message.end_lsn()); @@ -727,22 +666,17 @@ async fn handle_relation_message( message: &protocol::RelationBody, schema_cache: &SchemaCache, hook: &T, -) -> Result +) -> Result where T: ApplyLoopHook, - ApplyLoopError: From<::Error>, + ETLError: From<::Error>, { let Event::Relation(event) = event else { - return Err(ApplyLoopError::InvalidEvent( - event.into(), - EventType::Relation, - )); + return Err(ETLError { repr: ErrorRepr::WithDescriptionAndDetail(ErrorKind::ValidationError, "Invalid event", format!("An invalid event {:?} was received (expected {:?})", event, EventType::Relation)) }); }; let Some(remote_final_lsn) = state.remote_final_lsn else { - return Err(ApplyLoopError::InvalidTransaction( - "handle_relation_message".to_owned(), - )); + return Err(ETLError { repr: ErrorRepr::WithDescriptionAndDetail(ErrorKind::InvalidState, "Invalid transaction", "A transaction should have started for handle_relation_message to be performed".to_string()) }); }; if !hook @@ -757,7 +691,7 @@ where // TODO: explore how to deal with applying relation messages to the schema (creating it if missing). let schema_cache = schema_cache.lock_inner().await; let Some(existing_table_schema) = schema_cache.get_table_schema_ref(&message.rel_id()) else { - return Err(ApplyLoopError::MissingTableSchema(message.rel_id())); + return Err(ETLError { repr: ErrorRepr::WithDescriptionAndDetail(ErrorKind::SchemaError, "Missing table schema", format!("The table schema for table {} was not found in the cache", message.rel_id())) }); }; // We compare the table schema from the relation message with the existing schema (if any). @@ -784,22 +718,17 @@ async fn handle_insert_message( event: Event, message: &protocol::InsertBody, hook: &T, -) -> Result +) -> Result where T: ApplyLoopHook, - ApplyLoopError: From<::Error>, + ETLError: From<::Error>, { let Event::Insert(event) = event else { - return Err(ApplyLoopError::InvalidEvent( - event.into(), - EventType::Insert, - )); + return Err(ETLError { repr: ErrorRepr::WithDescriptionAndDetail(ErrorKind::ValidationError, "Invalid event", format!("An invalid event {:?} was received (expected {:?})", event, EventType::Insert)) }); }; let Some(remote_final_lsn) = state.remote_final_lsn else { - return Err(ApplyLoopError::InvalidTransaction( - "handle_insert_message".to_owned(), - )); + return Err(ETLError { repr: ErrorRepr::WithDescriptionAndDetail(ErrorKind::InvalidState, "Invalid transaction", "A transaction should have started for handle_insert_message to be performed".to_string()) }); }; if !hook @@ -822,22 +751,17 @@ async fn handle_update_message( event: Event, message: &protocol::UpdateBody, hook: &T, -) -> Result +) -> Result where T: ApplyLoopHook, - ApplyLoopError: From<::Error>, + ETLError: From<::Error>, { let Event::Update(event) = event else { - return Err(ApplyLoopError::InvalidEvent( - event.into(), - EventType::Update, - )); + return Err(ETLError { repr: ErrorRepr::WithDescriptionAndDetail(ErrorKind::ValidationError, "Invalid event", format!("An invalid event {:?} was received (expected {:?})", event, EventType::Update)) }); }; let Some(remote_final_lsn) = state.remote_final_lsn else { - return Err(ApplyLoopError::InvalidTransaction( - "handle_update_message".to_owned(), - )); + return Err(ETLError { repr: ErrorRepr::WithDescriptionAndDetail(ErrorKind::InvalidState, "Invalid transaction", "A transaction should have started for handle_update_message to be performed".to_string()) }); }; if !hook @@ -860,22 +784,17 @@ async fn handle_delete_message( event: Event, message: &protocol::DeleteBody, hook: &T, -) -> Result +) -> Result where T: ApplyLoopHook, - ApplyLoopError: From<::Error>, + ETLError: From<::Error>, { let Event::Delete(event) = event else { - return Err(ApplyLoopError::InvalidEvent( - event.into(), - EventType::Delete, - )); + return Err(ETLError { repr: ErrorRepr::WithDescriptionAndDetail(ErrorKind::ValidationError, "Invalid event", format!("An invalid event {:?} was received (expected {:?})", event, EventType::Delete)) }); }; let Some(remote_final_lsn) = state.remote_final_lsn else { - return Err(ApplyLoopError::InvalidTransaction( - "handle_delete_message".to_owned(), - )); + return Err(ETLError { repr: ErrorRepr::WithDescriptionAndDetail(ErrorKind::InvalidState, "Invalid transaction", "A transaction should have started for handle_delete_message to be performed".to_string()) }); }; if !hook @@ -898,22 +817,17 @@ async fn handle_truncate_message( event: Event, message: &protocol::TruncateBody, hook: &T, -) -> Result +) -> Result where T: ApplyLoopHook, - ApplyLoopError: From<::Error>, + ETLError: From<::Error>, { let Event::Truncate(mut event) = event else { - return Err(ApplyLoopError::InvalidEvent( - event.into(), - EventType::Truncate, - )); + return Err(ETLError { repr: ErrorRepr::WithDescriptionAndDetail(ErrorKind::ValidationError, "Invalid event", format!("An invalid event {:?} was received (expected {:?})", event, EventType::Truncate)) }); }; let Some(remote_final_lsn) = state.remote_final_lsn else { - return Err(ApplyLoopError::InvalidTransaction( - "handle_truncate_message".to_owned(), - )); + return Err(ETLError { repr: ErrorRepr::WithDescriptionAndDetail(ErrorKind::InvalidState, "Invalid transaction", "A transaction should have started for handle_truncate_message to be performed".to_string()) }); }; let mut rel_ids = Vec::with_capacity(message.rel_ids().len()); diff --git a/etl/src/replication/client.rs b/etl/src/replication/client.rs index d571e8037..cbcc51103 100644 --- a/etl/src/replication/client.rs +++ b/etl/src/replication/client.rs @@ -8,7 +8,7 @@ use std::collections::HashMap; use std::fmt; use std::io::BufReader; use std::sync::Arc; -use thiserror::Error; +use crate::error::{ETLError, ETLResult, ErrorKind}; use tokio_postgres::error::SqlState; use tokio_postgres::tls::MakeTlsConnect; use tokio_postgres::{ @@ -41,57 +41,6 @@ where tokio::spawn(task); } -/// Errors that can occur when using the PostgreSQL replication client. -#[derive(Debug, Error)] -pub enum PgReplicationError { - /// Errors from the underlying PostgreSQL client - #[error("PostgreSQL client operation failed: {0}")] - Client(#[from] tokio_postgres::Error), - - /// Errors related to TLS/SSL configuration - #[error("TLS configuration failed: {0}")] - Tls(#[from] rustls::Error), - - /// Errors related to replication slot operations - #[error("Failed to create replication slot")] - SlotCreationFailed, - - #[error("Replication slot '{0}' not found in database")] - SlotNotFound(String), - - #[error("Replication slot '{0}' already exists in database")] - SlotAlreadyExists(String), - - #[error("Invalid replication slot response: missing required fields in server response")] - SlotResponseInvalid, - - /// Errors related to database schema and objects - #[error("Table '{0}' not found in database")] - TableNotFound(TableName), - - #[error("Column '{0}' not found in table '{1}'")] - ColumnNotFound(String, String), - - #[error("Failed to parse value from column '{0}' in table '{1}': {2}")] - ColumnParsingFailed(String, String, String), - - #[error("Publication '{0}' not found in database")] - PublicationNotFound(String), - - /// Errors related to data type handling - #[error( - "Unsupported column type '{0}' (OID: {1}) in table '{2}': type is not supported for replication" - )] - UnsupportedColumnType(String, u32, String), - - #[error( - "Unsupported replica identity '{0}' in table: only 'default' or 'full' replica identities are supported" - )] - UnsupportedReplicaIdentity(String), - - #[error("Io error: {0}")] - Io(#[from] std::io::Error), -} #[derive(Debug, Clone)] pub struct CreateSlotResult { @@ -201,7 +150,7 @@ pub struct PgReplicationClient { } /// Update the type alias to use the new error type -pub type PgReplicationResult = Result; +pub type PgReplicationResult = ETLResult; impl PgReplicationClient { /// Establishes a connection to PostgreSQL. The connection uses TLS if configured in the @@ -308,7 +257,7 @@ impl PgReplicationClient { } } - Err(PgReplicationError::SlotNotFound(slot_name.to_string())) + Err(ETLError::from((ErrorKind::ReplicationError, "Replication slot not found", format!("Replication slot '{}' not found in database", slot_name)))) } /// Gets an existing replication slot or creates a new one if it doesn't exist. @@ -330,7 +279,7 @@ impl PgReplicationClient { Ok(GetOrCreateSlotResult::GetSlot(slot)) } - Err(PgReplicationError::SlotNotFound(_)) => { + Err(err) if err.kind() == ErrorKind::ReplicationError && err.detail().map_or(false, |d| d.contains("not found")) => { info!("creating new replication slot '{}'", slot_name); let create_result = self.create_slot_internal(slot_name, false).await?; @@ -365,7 +314,7 @@ impl PgReplicationClient { "attempted to delete non-existent replication slot '{}'", slot_name ); - return Err(PgReplicationError::SlotNotFound(slot_name.to_string())); + return Err(ETLError::from((ErrorKind::ReplicationError, "Replication slot not found", format!("Replication slot '{}' not found in database", slot_name)))); } } @@ -540,7 +489,7 @@ impl PgReplicationClient { Err(err) => { if let Some(code) = err.code() { if *code == SqlState::DUPLICATE_OBJECT { - return Err(PgReplicationError::SlotAlreadyExists(slot_name.to_string())); + return Err(ETLError::from((ErrorKind::ReplicationError, "Replication slot already exists", format!("Replication slot '{}' already exists in database", slot_name)))); } } @@ -548,7 +497,7 @@ impl PgReplicationClient { } } - Err(PgReplicationError::SlotCreationFailed) + Err(ETLError::from((ErrorKind::ReplicationError, "Failed to create replication slot"))) } /// Retrieves schema information for multiple tables. @@ -625,10 +574,7 @@ impl PgReplicationClient { } } - Err(PgReplicationError::TableNotFound(TableName { - schema: String::new(), - name: format!("oid: {table_id}"), - })) + Err(ETLError::from((ErrorKind::SchemaError, "Table not found", format!("Table not found in database (oid: {})", table_id)))) } /// Retrieves schema information for all columns in a table. @@ -753,17 +699,10 @@ impl PgReplicationClient { { let value = row .try_get(column_name)? - .ok_or(PgReplicationError::ColumnNotFound( - column_name.to_string(), - table_name.to_string(), - ))?; + .ok_or(ETLError::from((ErrorKind::SchemaError, "Column not found", format!("Column '{}' not found in table '{}'", column_name, table_name))))?; value.parse().map_err(|e: T::Err| { - PgReplicationError::ColumnParsingFailed( - column_name.to_string(), - table_name.to_string(), - format!("{e:?}"), - ) + ETLError::from((ErrorKind::ConversionError, "Column parsing failed", format!("Failed to parse value from column '{}' in table '{}': {:?}", column_name, table_name, e))) }) } } diff --git a/etl/src/replication/common.rs b/etl/src/replication/common.rs index 5e2e6665c..a91b1111d 100644 --- a/etl/src/replication/common.rs +++ b/etl/src/replication/common.rs @@ -1,9 +1,9 @@ use postgres::schema::TableId; use std::collections::HashMap; +use crate::error::ETLResult; use crate::state::store::base::StateStore; use crate::state::table::TableReplicationPhase; -use crate::workers::apply::ApplyWorkerHookError; /// Returns the table replication states that are either done or in active state. /// @@ -12,7 +12,7 @@ use crate::workers::apply::ApplyWorkerHookError; pub async fn get_table_replication_states( state_store: &S, done: bool, -) -> Result, ApplyWorkerHookError> +) -> ETLResult> where S: StateStore + Clone + Send + Sync + 'static, { diff --git a/etl/src/replication/table_sync.rs b/etl/src/replication/table_sync.rs index f28d3d50f..51ad39fc7 100644 --- a/etl/src/replication/table_sync.rs +++ b/etl/src/replication/table_sync.rs @@ -13,44 +13,19 @@ use crate::concurrency::shutdown::{ShutdownResult, ShutdownRx}; use crate::concurrency::signal::SignalTx; use crate::concurrency::stream::BatchStream; use crate::destination::base::{Destination, DestinationError}; +use crate::error::{ETLError, ETLResult, ErrorKind}; #[cfg(feature = "failpoints")] use crate::failpoints::START_TABLE_SYNC_AFTER_DATA_SYNC; use crate::pipeline::PipelineId; -use crate::replication::client::{PgReplicationClient, PgReplicationError}; +use crate::replication::client::PgReplicationClient; use crate::replication::slot::{SlotError, get_slot_name}; use crate::replication::stream::{TableCopyStream, TableCopyStreamError}; use crate::schema::cache::SchemaCache; -use crate::state::store::base::{StateStore, StateStoreError}; +use crate::state::store::base::StateStore; use crate::state::table::{TableReplicationPhase, TableReplicationPhaseType}; use crate::workers::base::WorkerType; -use crate::workers::table_sync::{TableSyncWorkerState, TableSyncWorkerStateError}; +use crate::workers::table_sync::TableSyncWorkerState; -#[derive(Debug, Error)] -pub enum TableSyncError { - #[error("Invalid replication phase '{0}': expected Init, DataSync, or FinishedCopy")] - InvalidPhase(TableReplicationPhaseType), - - #[error("Invalid replication slot name: {0}")] - InvalidSlotName(#[from] SlotError), - - #[error("PostgreSQL replication operation failed: {0}")] - PgReplication(#[from] PgReplicationError), - - #[error("An error occurred while interacting with the table sync worker state: {0}")] - TableSyncWorkerState(#[from] TableSyncWorkerStateError), - - #[error("An error occurred while writing to the destination: {0}")] - Destination(#[from] DestinationError), - - #[error("An error happened in the state store: {0}")] - StateStore(#[from] StateStoreError), - - #[error("An error happened in the table copy stream")] - TableCopyStream(#[from] TableCopyStreamError), - - #[error("table {0} has no primary key")] - MissingPrimaryKey(TableName), -} #[derive(Debug)] pub enum TableSyncResult { @@ -71,7 +46,7 @@ pub async fn start_table_sync( destination: D, shutdown_rx: ShutdownRx, force_syncing_tables_tx: SignalTx, -) -> Result +) -> Result where S: StateStore + Clone + Send + 'static, D: Destination + Clone + Send + 'static, @@ -112,7 +87,7 @@ where phase_type, table_id ); - return Err(TableSyncError::InvalidPhase(phase_type)); + return Err(ETLError::from((ErrorKind::InvalidState, "Invalid replication phase", format!("Invalid replication phase '{}': expected Init, DataSync, or FinishedCopy", phase_type)))); } phase_type @@ -146,7 +121,7 @@ where // before starting a table copy. if let Err(err) = replication_client.delete_slot(&slot_name).await { // If the slot is not found, we are safe to continue, for any other error, we bail. - if !matches!(err, PgReplicationError::SlotNotFound(_)) { + if err.kind() != ErrorKind::ReplicationError || !err.detail().map_or(false, |d| d.contains("not found")) { return Err(err.into()); } } @@ -190,7 +165,7 @@ where state_store .update_table_replication_state(table_id, TableReplicationPhase::Skipped) .await?; - return Err(TableSyncError::MissingPrimaryKey(table_schema.name)); + return Err(ETLError::from((ErrorKind::SchemaError, "Missing primary key", format!("table {} has no primary key", table_schema.name)))); } schema_cache.add_table_schema(table_schema.clone()).await; diff --git a/etl/src/state/store/base.rs b/etl/src/state/store/base.rs index 5d13fc735..9b9532018 100644 --- a/etl/src/state/store/base.rs +++ b/etl/src/state/store/base.rs @@ -1,35 +1,14 @@ use postgres::schema::TableId; use std::{collections::HashMap, future::Future}; -use thiserror::Error; +use crate::error::{ETLError, ETLResult, ErrorKind}; use crate::{ replication::slot::SlotError, state::{ - store::postgres::{FromTableStateError, ToTableStateError}, table::TableReplicationPhase, }, }; -#[derive(Debug, Error)] -pub enum StateStoreError { - #[error("Sqlx error in state store: {0}")] - Database(#[from] sqlx::Error), - - #[error("Slot error in state store: {0}")] - Slot(#[from] SlotError), - - #[error("Invalid confirmed flush lsn value in state store: {0}")] - InvalidConfirmedFlushLsn(String), - - #[error("Missing slot in state store: {0}")] - MissingSlot(String), - - #[error("Error converting from table replication phase to table state")] - ToTableState(#[from] ToTableStateError), - - #[error("Error converting from table state to table replication phase")] - FromTableState(#[from] FromTableStateError), -} /// This trait represents a state store for the replication state of all tables. /// It assumes that the implementers keep a cache of the state to avoid having @@ -41,13 +20,13 @@ pub trait StateStore { fn get_table_replication_state( &self, table_id: TableId, - ) -> impl Future, StateStoreError>> + Send; + ) -> impl Future>> + Send; /// Returns the table replication states for all the tables from the cache. /// Does not read from the persistent store. fn get_table_replication_states( &self, - ) -> impl Future, StateStoreError>> + Send; + ) -> impl Future>> + Send; /// Loads the table replication states from the persistent state into the cache. /// This should called once at program start to load the state into the cache @@ -56,7 +35,7 @@ pub trait StateStore { /// the persistent store, so no need to ever load the state again. fn load_table_replication_states( &self, - ) -> impl Future> + Send; + ) -> impl Future> + Send; /// Updates the table replicate state for a table with `table_id` in both the cache as well as /// the persistent store. @@ -64,5 +43,5 @@ pub trait StateStore { &self, table_id: TableId, state: TableReplicationPhase, - ) -> impl Future> + Send; + ) -> impl Future> + Send; } diff --git a/etl/src/state/store/memory.rs b/etl/src/state/store/memory.rs index fbba0e1a9..70d4df2e3 100644 --- a/etl/src/state/store/memory.rs +++ b/etl/src/state/store/memory.rs @@ -3,7 +3,8 @@ use std::collections::HashMap; use std::sync::Arc; use tokio::sync::Mutex; -use crate::state::store::base::{StateStore, StateStoreError}; +use crate::state::store::base::StateStore; +use crate::error::ETLResult; use crate::state::table::TableReplicationPhase; #[derive(Debug)] @@ -38,7 +39,7 @@ impl StateStore for MemoryStateStore { async fn get_table_replication_state( &self, table_id: TableId, - ) -> Result, StateStoreError> { + ) -> ETLResult> { let inner = self.inner.lock().await; Ok(inner.table_replication_states.get(&table_id).cloned()) @@ -46,13 +47,13 @@ impl StateStore for MemoryStateStore { async fn get_table_replication_states( &self, - ) -> Result, StateStoreError> { + ) -> ETLResult> { let inner = self.inner.lock().await; Ok(inner.table_replication_states.clone()) } - async fn load_table_replication_states(&self) -> Result { + async fn load_table_replication_states(&self) -> ETLResult { let inner = self.inner.lock().await; Ok(inner.table_replication_states.len()) @@ -62,7 +63,7 @@ impl StateStore for MemoryStateStore { &self, table_id: TableId, state: TableReplicationPhase, - ) -> Result<(), StateStoreError> { + ) -> ETLResult<()> { let mut inner = self.inner.lock().await; inner.table_replication_states.insert(table_id, state); diff --git a/etl/src/state/store/notify.rs b/etl/src/state/store/notify.rs index c42d79e28..aa3952046 100644 --- a/etl/src/state/store/notify.rs +++ b/etl/src/state/store/notify.rs @@ -7,9 +7,10 @@ use tokio::{ }; use crate::state::{ - store::base::{StateStore, StateStoreError}, + store::base::StateStore, table::{TableReplicationPhase, TableReplicationPhaseType}, }; +use crate::error::ETLResult; #[derive(Debug, Clone, Copy, PartialEq, Eq, Hash)] pub enum StateStoreMethod { @@ -105,7 +106,7 @@ impl StateStore for NotifyingStateStore { async fn get_table_replication_state( &self, table_id: TableId, - ) -> Result, StateStoreError> { + ) -> ETLResult> { let inner = self.inner.read().await; let result = Ok(inner.table_replication_states.get(&table_id).cloned()); @@ -118,7 +119,7 @@ impl StateStore for NotifyingStateStore { async fn get_table_replication_states( &self, - ) -> Result, StateStoreError> { + ) -> ETLResult> { let inner = self.inner.read().await; let result = Ok(inner.table_replication_states.clone()); @@ -129,7 +130,7 @@ impl StateStore for NotifyingStateStore { result } - async fn load_table_replication_states(&self) -> Result { + async fn load_table_replication_states(&self) -> ETLResult { let inner = self.inner.read().await; let result = Ok(inner.table_replication_states.clone()); @@ -144,7 +145,7 @@ impl StateStore for NotifyingStateStore { &self, table_id: TableId, state: TableReplicationPhase, - ) -> Result<(), StateStoreError> { + ) -> ETLResult<()> { let mut inner = self.inner.write().await; inner.table_replication_states.insert(table_id, state); inner.check_conditions().await; diff --git a/etl/src/state/store/postgres.rs b/etl/src/state/store/postgres.rs index 1b13169ed..886100590 100644 --- a/etl/src/state/store/postgres.rs +++ b/etl/src/state/store/postgres.rs @@ -7,7 +7,7 @@ use postgres::replication::{ }; use postgres::schema::TableId; use sqlx::PgPool; -use thiserror::Error; +use crate::error::{ETLError, ETLResult, ErrorKind}; use tokio::sync::Mutex; use tokio_postgres::types::PgLsn; use tracing::{debug, info}; @@ -15,21 +15,15 @@ use tracing::{debug, info}; use crate::{ pipeline::PipelineId, state::{ - store::base::{StateStore, StateStoreError}, + store::base::StateStore, table::TableReplicationPhase, }, }; const NUM_POOL_CONNECTIONS: u32 = 1; -#[derive(Debug, Error)] -pub enum ToTableStateError { - #[error("In-memory table replication phase can't be saved in the state store")] - InMemoryPhase, -} - impl TryFrom for (TableReplicationState, Option) { - type Error = ToTableStateError; + type Error = ETLError; fn try_from(value: TableReplicationPhase) -> Result { Ok(match value { @@ -42,17 +36,12 @@ impl TryFrom for (TableReplicationState, Option) TableReplicationPhase::Ready => (TableReplicationState::Ready, None), TableReplicationPhase::Skipped => (TableReplicationState::Skipped, None), TableReplicationPhase::SyncWait | TableReplicationPhase::Catchup { .. } => { - return Err(ToTableStateError::InMemoryPhase); + return Err(ETLError::from((ErrorKind::InvalidState, "In-memory phase error", "In-memory table replication phase can't be saved in the state store".to_string()))); } }) } } -#[derive(Debug, Error)] -pub enum FromTableStateError { - #[error("Lsn can't be missing from the state store if state is SyncDone")] - MissingSyncDoneLsn, -} #[derive(Debug)] struct Inner { @@ -118,7 +107,7 @@ impl PostgresStateStore { &self, state: &TableReplicationState, sync_done_lsn: Option, - ) -> Result { + ) -> ETLResult { Ok(match state { TableReplicationState::Init => TableReplicationPhase::Init, TableReplicationState::DataSync => TableReplicationPhase::DataSync, @@ -127,10 +116,10 @@ impl PostgresStateStore { Some(lsn_str) => { let lsn = lsn_str .parse::() - .map_err(|_| StateStoreError::InvalidConfirmedFlushLsn(lsn_str))?; + .map_err(|_| ETLError::from((ErrorKind::ValidationError, "Invalid LSN", format!("Invalid confirmed flush lsn value in state store: {}", lsn_str))))?; TableReplicationPhase::SyncDone { lsn } } - None => return Err(FromTableStateError::MissingSyncDoneLsn)?, + None => return Err(ETLError::from((ErrorKind::ValidationError, "Missing LSN", "Lsn can't be missing from the state store if state is SyncDone".to_string()))), }, TableReplicationState::Ready => TableReplicationPhase::Ready, TableReplicationState::Skipped => TableReplicationPhase::Skipped, @@ -142,19 +131,19 @@ impl StateStore for PostgresStateStore { async fn get_table_replication_state( &self, table_id: TableId, - ) -> Result, StateStoreError> { + ) -> ETLResult> { let inner = self.inner.lock().await; Ok(inner.table_states.get(&table_id).cloned()) } async fn get_table_replication_states( &self, - ) -> Result, StateStoreError> { + ) -> ETLResult> { let inner = self.inner.lock().await; Ok(inner.table_states.clone()) } - async fn load_table_replication_states(&self) -> Result { + async fn load_table_replication_states(&self) -> ETLResult { debug!("loading table replication states from postgres state store"); let pool = self.connect_to_source().await?; let replication_state_rows = self @@ -182,7 +171,7 @@ impl StateStore for PostgresStateStore { &self, table_id: TableId, state: TableReplicationPhase, - ) -> Result<(), StateStoreError> { + ) -> ETLResult<()> { let (table_state, sync_done_lsn) = state.try_into()?; self.update_replication_state(self.pipeline_id, table_id, table_state, sync_done_lsn) .await?; diff --git a/etl/src/workers/apply.rs b/etl/src/workers/apply.rs index a43553cfe..5ea5ba169 100644 --- a/etl/src/workers/apply.rs +++ b/etl/src/workers/apply.rs @@ -1,7 +1,7 @@ use config::shared::PipelineConfig; use postgres::schema::TableId; use std::sync::Arc; -use thiserror::Error; +use crate::error::{ETLError, ETLResult, ErrorKind}; use tokio::sync::Semaphore; use tokio::task::JoinHandle; use tokio_postgres::types::PgLsn; @@ -11,52 +11,23 @@ use crate::concurrency::shutdown::ShutdownRx; use crate::concurrency::signal::{SignalTx, create_signal}; use crate::destination::base::Destination; use crate::pipeline::PipelineId; -use crate::replication::apply::{ApplyLoopError, ApplyLoopHook, start_apply_loop}; -use crate::replication::client::{PgReplicationClient, PgReplicationError}; +use crate::replication::apply::{ApplyLoopHook, start_apply_loop}; +use crate::replication::client::PgReplicationClient; use crate::replication::common::get_table_replication_states; use crate::replication::slot::{SlotError, get_slot_name}; use crate::schema::cache::SchemaCache; -use crate::state::store::base::{StateStore, StateStoreError}; +use crate::state::store::base::StateStore; use crate::state::table::{TableReplicationPhase, TableReplicationPhaseType}; use crate::workers::base::{Worker, WorkerHandle, WorkerType, WorkerWaitError}; use crate::workers::pool::TableSyncWorkerPool; use crate::workers::table_sync::{ - TableSyncWorker, TableSyncWorkerError, TableSyncWorkerState, TableSyncWorkerStateError, + TableSyncWorker, TableSyncWorkerState, }; -#[derive(Debug, Error)] -pub enum ApplyWorkerError { - #[error("An error occurred while interacting with the state store: {0}")] - StateStore(#[from] StateStoreError), - - #[error("An error occurred in the apply loop: {0}")] - ApplyLoop(#[from] ApplyLoopError), - - #[error("A Postgres replication error occurred in the apply loop: {0}")] - PgReplication(#[from] PgReplicationError), - - #[error("Could not generate slot name in the apply loop: {0}")] - Slot(#[from] SlotError), -} - -#[derive(Debug, Error)] -pub enum ApplyWorkerHookError { - #[error("An error occurred while interacting with the state store: {0}")] - StateStore(#[from] StateStoreError), - - #[error("An error occurred while interacting with the table sync worker state: {0}")] - TableSyncWorkerState(#[from] TableSyncWorkerStateError), - - #[error("An error occurred while trying to start the table sync worker: {0}")] - TableSyncWorkerStartedFailed(#[from] TableSyncWorkerError), - - #[error("A Postgres replication error occurred in the apply worker: {0}")] - PgReplication(#[from] PgReplicationError), -} #[derive(Debug)] pub struct ApplyWorkerHandle { - handle: Option>>, + handle: Option>>, } impl WorkerHandle<()> for ApplyWorkerHandle { @@ -118,9 +89,9 @@ where S: StateStore + Clone + Send + Sync + 'static, D: Destination + Clone + Send + Sync + 'static, { - type Error = ApplyWorkerError; + type Error = ETLError; - async fn start(self) -> Result { + async fn start(self) -> Result { info!("starting apply worker"); let apply_worker_span = tracing::info_span!( @@ -174,7 +145,7 @@ where async fn get_start_lsn( pipeline_id: PipelineId, replication_client: &PgReplicationClient, -) -> Result { +) -> Result { let slot_name = get_slot_name(pipeline_id, WorkerType::Apply)?; // TODO: validate that we only create the slot when we first start replication which // means when all tables are in the Init state. In any other case we should raise an @@ -257,7 +228,7 @@ where &self, table_id: TableId, current_lsn: PgLsn, - ) -> Result { + ) -> Result { let mut pool = self.pool.lock().await; let table_sync_worker_state = pool.get_active_worker_state(table_id); @@ -278,7 +249,7 @@ where table_id: TableId, table_sync_worker_state: TableSyncWorkerState, current_lsn: PgLsn, - ) -> Result { + ) -> Result { let mut catchup_started = false; { let mut inner = table_sync_worker_state.get_inner().lock().await; @@ -330,7 +301,7 @@ where S: StateStore + Clone + Send + Sync + 'static, D: Destination + Clone + Send + Sync + 'static, { - type Error = ApplyWorkerHookError; + type Error = ETLError; async fn before_loop(&self, _start_lsn: PgLsn) -> Result { info!("starting table sync workers before the main apply loop"); diff --git a/etl/src/workers/base.rs b/etl/src/workers/base.rs index 79892a608..aca2ef2e3 100644 --- a/etl/src/workers/base.rs +++ b/etl/src/workers/base.rs @@ -4,8 +4,7 @@ use std::future::Future; use thiserror::Error; use tokio::task; -use crate::workers::apply::ApplyWorkerError; -use crate::workers::table_sync::TableSyncWorkerError; +use crate::error::ETLError; /// Represents all possible errors that can occur while waiting for a worker to complete. /// @@ -27,17 +26,11 @@ pub enum WorkerWaitError { #[error("Worker task failed internally and the error was silently handled: {0}")] WorkerSilentlyFailed(String), - /// The apply worker encountered an error that was propagated via the handle's return value. + /// A worker encountered an error that was propagated via the handle's return value. /// - /// This variant wraps the specific error returned by the apply worker. - #[error("Apply worker terminated with an error: {0}")] - ApplyWorkerFailed(#[from] ApplyWorkerError), - - /// The table sync worker encountered an error that was propagated via the handle's return value. - /// - /// This variant wraps the specific error returned by the table sync worker. - #[error("Table sync worker terminated with an error: {0}")] - TableSyncWorkerFailed(#[from] TableSyncWorkerError), + /// This variant wraps the specific error returned by the worker. + #[error("Worker terminated with an error: {0}")] + WorkerFailed(#[from] ETLError), } #[derive(Debug)] diff --git a/etl/src/workers/pool.rs b/etl/src/workers/pool.rs index de91718be..c1485cc6a 100644 --- a/etl/src/workers/pool.rs +++ b/etl/src/workers/pool.rs @@ -11,7 +11,7 @@ use crate::destination::base::Destination; use crate::state::store::base::StateStore; use crate::workers::base::{Worker, WorkerHandle, WorkerWaitError, WorkerWaitErrors}; use crate::workers::table_sync::{ - TableSyncWorker, TableSyncWorkerError, TableSyncWorkerHandle, TableSyncWorkerState, + TableSyncWorker, TableSyncWorkerHandle, TableSyncWorkerState, }; #[derive(Debug)] @@ -46,7 +46,7 @@ impl TableSyncWorkerPoolInner { pub async fn start_worker( &mut self, worker: TableSyncWorker, - ) -> Result + ) -> crate::error::ETLResult where S: StateStore + Clone + Send + Sync + 'static, D: Destination + Clone + Send + Sync + 'static, diff --git a/etl/src/workers/table_sync.rs b/etl/src/workers/table_sync.rs index 8167c3e04..e51d384d6 100644 --- a/etl/src/workers/table_sync.rs +++ b/etl/src/workers/table_sync.rs @@ -2,7 +2,7 @@ use config::shared::PipelineConfig; use postgres::schema::TableId; use std::sync::Arc; use std::time::Duration; -use thiserror::Error; +use crate::error::{ETLError, ETLResult, ErrorKind}; use tokio::sync::{AcquireError, Mutex, MutexGuard, Notify, Semaphore}; use tokio::task::JoinHandle; use tokio_postgres::types::PgLsn; @@ -13,12 +13,12 @@ use crate::concurrency::shutdown::{ShutdownResult, ShutdownRx}; use crate::concurrency::signal::SignalTx; use crate::destination::base::Destination; use crate::pipeline::PipelineId; -use crate::replication::apply::{ApplyLoopError, ApplyLoopHook, start_apply_loop}; -use crate::replication::client::{PgReplicationClient, PgReplicationError}; +use crate::replication::apply::{ApplyLoopHook, start_apply_loop}; +use crate::replication::client::PgReplicationClient; use crate::replication::slot::get_slot_name; -use crate::replication::table_sync::{TableSyncError, TableSyncResult, start_table_sync}; +use crate::replication::table_sync::{TableSyncResult, start_table_sync}; use crate::schema::cache::SchemaCache; -use crate::state::store::base::{StateStore, StateStoreError}; +use crate::state::store::base::StateStore; use crate::state::table::{TableReplicationPhase, TableReplicationPhaseType}; use crate::workers::base::{Worker, WorkerHandle, WorkerType, WorkerWaitError}; use crate::workers::pool::TableSyncWorkerPool; @@ -34,41 +34,6 @@ const PHASE_CHANGE_REFRESH_FREQUENCY: Duration = Duration::from_millis(100); /// of new tables. const MAX_DELETE_SLOT_WAIT: Duration = Duration::from_secs(30); -#[derive(Debug, Error)] -pub enum TableSyncWorkerError { - #[error("An error occurred while syncing a table: {0}")] - TableSync(#[from] TableSyncError), - - #[error("The replication state is missing for table {0}")] - ReplicationStateMissing(TableId), - - #[error("An error occurred while interacting with the state store: {0}")] - StateStore(#[from] StateStoreError), - - #[error("An error occurred in the apply loop: {0}")] - ApplyLoop(#[from] ApplyLoopError), - - #[error("Failed to acquire a permit to run a table sync worker")] - PermitAcquire(#[from] AcquireError), - - #[error("A Postgres replication error occurred in the table sync worker: {0}")] - PgReplication(#[from] PgReplicationError), -} - -#[derive(Debug, Error)] -pub enum TableSyncWorkerHookError { - #[error("An error occurred while updating the table sync worker state: {0}")] - TableSyncWorkerState(#[from] TableSyncWorkerStateError), - - #[error("A Postgres replication error occurred in the table sync worker: {0}")] - PgReplication(#[from] PgReplicationError), -} - -#[derive(Debug, Error)] -pub enum TableSyncWorkerStateError { - #[error("An error occurred while interacting with the state store: {0}")] - StateStore(#[from] StateStoreError), -} #[derive(Debug)] pub struct TableSyncWorkerStateInner { @@ -98,7 +63,7 @@ impl TableSyncWorkerStateInner { &mut self, phase: TableReplicationPhase, state_store: S, - ) -> Result<(), TableSyncWorkerStateError> { + ) -> ETLResult<()> { self.set_phase(phase); // If we should store this phase change, we want to do it via the supplied state store. @@ -224,7 +189,7 @@ impl TableSyncWorkerState { #[derive(Debug)] pub struct TableSyncWorkerHandle { state: TableSyncWorkerState, - handle: Option>>, + handle: Option>>, } impl WorkerHandle for TableSyncWorkerHandle { @@ -295,9 +260,9 @@ where S: StateStore + Clone + Send + Sync + 'static, D: Destination + Clone + Send + Sync + 'static, { - type Error = TableSyncWorkerError; + type Error = ETLError; - async fn start(mut self) -> Result { + async fn start(mut self) -> Result { info!("starting table sync worker for table {}", self.table_id); let Some(table_replication_phase) = self @@ -310,7 +275,7 @@ where self.table_id ); - return Err(TableSyncWorkerError::ReplicationStateMissing(self.table_id)); + return Err(ETLError::from((ErrorKind::InvalidState, "Replication state missing", format!("The replication state is missing for table {}", self.table_id)))); }; info!( @@ -476,7 +441,7 @@ where &self, current_lsn: PgLsn, update_state: bool, - ) -> Result { + ) -> Result { let mut inner = self.table_sync_worker_state.get_inner().lock().await; // If we caught up with the lsn, we mark this table as `SyncDone` and stop the worker. @@ -512,7 +477,7 @@ impl ApplyLoopHook for TableSyncWorkerHook where S: StateStore + Clone + Send + Sync + 'static, { - type Error = TableSyncWorkerHookError; + type Error = ETLError; async fn before_loop(&self, start_lsn: PgLsn) -> Result { info!("checking if the table sync worker is already caught up with the apply worker"); From 806ecc626f38fab9207165dcfc9721e79badd0f0 Mon Sep 17 00:00:00 2001 From: Riccardo Busetti Date: Fri, 25 Jul 2025 11:04:15 +0200 Subject: [PATCH 02/18] Improve --- etl/src/clients/bigquery.rs | 23 +- etl/src/concurrency/future.rs | 12 +- etl/src/destination/bigquery.rs | 35 +-- etl/src/error.rs | 275 ++++++++++++++++++++-- etl/src/lib.rs | 3 - etl/src/pipeline.rs | 20 +- etl/src/replication/apply.rs | 202 +++++++++++++--- etl/src/replication/client.rs | 58 ++++- etl/src/replication/table_sync.rs | 21 +- etl/src/state/store/base.rs | 14 +- etl/src/state/store/memory.rs | 2 +- etl/src/state/store/notify.rs | 2 +- etl/src/state/store/postgres.rs | 34 ++- etl/src/workers/apply.rs | 23 +- etl/src/workers/base.rs | 51 +--- etl/src/workers/pool.rs | 31 +-- etl/src/workers/table_sync.rs | 28 ++- etl/tests/integration/replication_test.rs | 9 +- 18 files changed, 600 insertions(+), 243 deletions(-) diff --git a/etl/src/clients/bigquery.rs b/etl/src/clients/bigquery.rs index 905646443..3a7062a6e 100644 --- a/etl/src/clients/bigquery.rs +++ b/etl/src/clients/bigquery.rs @@ -1,3 +1,4 @@ +use crate::error::{ETLError, ETLResult, ErrorKind}; use futures::StreamExt; use gcp_bigquery_client::google::cloud::bigquery::storage::v1::RowError; use gcp_bigquery_client::storage::{ColumnMode, StorageApi}; @@ -10,7 +11,6 @@ use gcp_bigquery_client::{ }; use postgres::schema::ColumnSchema; use std::fmt; -use crate::error::{ETLError, ETLResult, ErrorKind}; use tokio_postgres::types::Type; use tracing::info; @@ -95,10 +95,7 @@ impl BigQueryClient { /// /// Parses the provided service account key string to authenticate with the /// BigQuery API. - pub async fn new_with_key( - project_id: String, - sa_key: &str, - ) -> ETLResult { + pub async fn new_with_key(project_id: String, sa_key: &str) -> ETLResult { let sa_key = parse_service_account_key(sa_key).map_err(BQError::from)?; let client = Client::from_service_account_key(sa_key, false).await?; @@ -158,11 +155,7 @@ impl BigQueryClient { } /// Truncates a table in a BigQuery dataset. - pub async fn truncate_table( - &self, - dataset_id: &str, - table_id: &str, - ) -> ETLResult<()> { + pub async fn truncate_table(&self, dataset_id: &str, table_id: &str) -> ETLResult<()> { let full_table_name = self.full_table_name(dataset_id, table_id); info!("Truncating table {full_table_name} in BigQuery"); @@ -179,11 +172,7 @@ impl BigQueryClient { /// # Panics /// /// Panics if the query result does not contain the expected `table_exists` column. - pub async fn table_exists( - &self, - dataset_id: &str, - table_id: &str, - ) -> ETLResult { + pub async fn table_exists(&self, dataset_id: &str, table_id: &str) -> ETLResult { let table = self .client .table() @@ -232,9 +221,7 @@ impl BigQueryClient { if let Some(append_rows_response) = append_rows_stream.next().await { let append_rows_response = append_rows_response.map_err(BQError::from)?; if !append_rows_response.row_errors.is_empty() { - return Err(ETLError::from(RowErrors( - append_rows_response.row_errors, - ))); + return Err(ETLError::from(RowErrors(append_rows_response.row_errors))); } } diff --git a/etl/src/concurrency/future.rs b/etl/src/concurrency/future.rs index 114e86b3d..35df4562e 100644 --- a/etl/src/concurrency/future.rs +++ b/etl/src/concurrency/future.rs @@ -59,7 +59,7 @@ where /// /// The callback will be notified of either success or failure, and will receive the provided /// identifier to track which future completed. - pub fn new(future: Fut, id: I, callback_source: Arc>) -> Self { + pub fn wrap(future: Fut, id: I, callback_source: Arc>) -> Self { Self { future: AssertUnwindSafe(future).catch_unwind(), callback_fut: None, @@ -271,7 +271,7 @@ mod tests { let callback = Arc::new(Mutex::new(MockCallback::default())); let table_id = 1; - let future = ReactiveFuture::new(ImmediateFuture, table_id, callback.clone()); + let future = ReactiveFuture::wrap(ImmediateFuture, table_id, callback.clone()); future.await.unwrap(); let guard = callback.lock().await; @@ -285,7 +285,7 @@ mod tests { let table_id = 1; // Test string panic - let future = AssertUnwindSafe(ReactiveFuture::new( + let future = AssertUnwindSafe(ReactiveFuture::wrap( PanicFuture { panic_any: false }, table_id, callback.clone(), @@ -312,7 +312,7 @@ mod tests { drop(guard); // Test any panic - let future = AssertUnwindSafe(ReactiveFuture::new( + let future = AssertUnwindSafe(ReactiveFuture::wrap( PanicFuture { panic_any: true }, table_id, callback.clone(), @@ -335,7 +335,7 @@ mod tests { let table_id = 1; let pending_future = PendingFuture::new(); - let future = ReactiveFuture::new(pending_future, table_id, callback.clone()); + let future = ReactiveFuture::wrap(pending_future, table_id, callback.clone()); // First poll should return pending let mut pinned_future = Box::pin(future); @@ -364,7 +364,7 @@ mod tests { let callback = Arc::new(Mutex::new(MockCallback::default())); let table_id = 1; - let future = ReactiveFuture::new(ErrorFuture, table_id, callback.clone()); + let future = ReactiveFuture::wrap(ErrorFuture, table_id, callback.clone()); let result = future.await; // Verify the error was propagated diff --git a/etl/src/destination/bigquery.rs b/etl/src/destination/bigquery.rs index 0dd826bc0..e02df92ce 100644 --- a/etl/src/destination/bigquery.rs +++ b/etl/src/destination/bigquery.rs @@ -72,7 +72,6 @@ static ETL_TABLE_COLUMNS_COLUMNS: LazyLock> = LazyLock::new(|| ] }); - /// Internal state for [`BigQueryDestination`] wrapped in `Arc>`. /// /// Contains the BigQuery client, dataset configuration, and injected schema cache. @@ -185,12 +184,21 @@ impl BigQueryDestination { let schema_cache = inner .schema_cache .as_ref() - .ok_or_else(|| ETLError::from((ErrorKind::ConfigError, "The schema cache was not set on the destination")))? + .ok_or_else(|| { + ETLError::from(( + ErrorKind::ConfigError, + "The schema cache was not set on the destination", + )) + })? .lock_inner() .await; - let table_schema = schema_cache - .get_table_schema_ref(table_id) - .ok_or_else(|| ETLError::from((ErrorKind::SchemaError, "Table schema not found in schema cache", format!("table_id: {}", table_id))))?; + let table_schema = schema_cache.get_table_schema_ref(table_id).ok_or_else(|| { + ETLError::from(( + ErrorKind::SchemaError, + "Table schema not found in schema cache", + format!("table_id: {}", table_id), + )) + })?; let table_id = table_schema.name.as_bigquery_table_id(); let table_descriptor = BigQueryClient::column_schemas_to_table_descriptor( @@ -204,10 +212,7 @@ impl BigQueryDestination { /// Writes a table schema to BigQuery, creating the data table and storing metadata. /// /// This method creates the actual data table and inserts schema information into the ETL metadata tables. - async fn write_table_schema( - &self, - table_schema: TableSchema, - ) -> ETLResult<()> { + async fn write_table_schema(&self, table_schema: TableSchema) -> ETLResult<()> { let mut inner = self.inner.lock().await; let dataset_id = inner.dataset_id.clone(); @@ -520,10 +525,7 @@ impl BigQueryDestination { /// /// Maps PostgreSQL table OIDs to BigQuery table names and issues truncate commands. #[allow(dead_code)] - async fn process_truncate_events( - &self, - truncate_events: Vec, - ) -> ETLResult<()> { + async fn process_truncate_events(&self, truncate_events: Vec) -> ETLResult<()> { let inner = self.inner.lock().await; for truncate_event in truncate_events { @@ -532,7 +534,12 @@ impl BigQueryDestination { let schema_cache = inner .schema_cache .as_ref() - .ok_or_else(|| ETLError::from((ErrorKind::ConfigError, "The schema cache was not set on the destination")))? + .ok_or_else(|| { + ETLError::from(( + ErrorKind::ConfigError, + "The schema cache was not set on the destination", + )) + })? .lock_inner() .await; diff --git a/etl/src/error.rs b/etl/src/error.rs index d8c0377a7..f4deade4f 100644 --- a/etl/src/error.rs +++ b/etl/src/error.rs @@ -6,13 +6,14 @@ pub type ETLResult = Result; /// Main error type for ETL operations, inspired by Redis error handling pub struct ETLError { - pub repr: ErrorRepr, + repr: ErrorRepr, } #[derive(Debug)] pub enum ErrorRepr { WithDescription(ErrorKind, &'static str), WithDescriptionAndDetail(ErrorKind, &'static str, String), + Many(Vec), } /// Comprehensive error kinds for ETL operations @@ -53,15 +54,38 @@ pub enum ErrorKind { DestinationError, /// Source-specific error SourceError, - /// Replication error - ReplicationError, + /// Replication slot not found + ReplicationSlotNotFound, + /// Replication slot already exists + ReplicationSlotAlreadyExists, + /// Replication slot name is invalid or too long + ReplicationSlotInvalid, + /// Table synchronization failed + TableSyncFailed, + /// Logical replication stream error + LogicalReplicationFailed, } impl ETLError { + /// Creates a new ETLError that contains multiple errors + pub fn many(errors: Vec) -> ETLError { + ETLError { + repr: ErrorRepr::Many(errors), + } + } + /// Returns the kind of the error pub fn kind(&self) -> ErrorKind { match self.repr { - ErrorRepr::WithDescription(kind, _) | ErrorRepr::WithDescriptionAndDetail(kind, _, _) => kind, + ErrorRepr::WithDescription(kind, _) + | ErrorRepr::WithDescriptionAndDetail(kind, _, _) => kind, + ErrorRepr::Many(ref errors) => { + // For multiple errors, return the kind of the first error, or WorkerError if empty + errors + .first() + .map(|e| e.kind()) + .unwrap_or(ErrorKind::WorkerError) + } } } @@ -69,6 +93,10 @@ impl ETLError { pub fn detail(&self) -> Option<&str> { match self.repr { ErrorRepr::WithDescriptionAndDetail(_, _, ref detail) => Some(detail.as_str()), + ErrorRepr::Many(ref errors) => { + // For multiple errors, return the detail of the first error that has one + errors.iter().find_map(|e| e.detail()) + } _ => None, } } @@ -93,7 +121,11 @@ impl ETLError { ErrorKind::WorkerError => "worker error", ErrorKind::DestinationError => "destination error", ErrorKind::SourceError => "source error", - ErrorKind::ReplicationError => "replication error", + ErrorKind::ReplicationSlotNotFound => "replication slot not found", + ErrorKind::ReplicationSlotAlreadyExists => "replication slot already exists", + ErrorKind::ReplicationSlotInvalid => "replication slot invalid", + ErrorKind::TableSyncFailed => "table sync failed", + ErrorKind::LogicalReplicationFailed => "logical replication failed", } } @@ -105,7 +137,6 @@ impl ETLError { ) } - /// Returns true if this is a data-related error pub fn is_data_error(&self) -> bool { matches!( @@ -113,16 +144,112 @@ impl ETLError { ErrorKind::SchemaError | ErrorKind::ConversionError | ErrorKind::ValidationError ) } + + /// Returns true if this is a replication-related error + pub fn is_replication_error(&self) -> bool { + matches!( + self.kind(), + ErrorKind::ReplicationSlotNotFound + | ErrorKind::ReplicationSlotAlreadyExists + | ErrorKind::ReplicationSlotInvalid + | ErrorKind::TableSyncFailed + | ErrorKind::LogicalReplicationFailed + ) + } + + /// Returns true if this is a replication slot error + pub fn is_replication_slot_error(&self) -> bool { + matches!( + self.kind(), + ErrorKind::ReplicationSlotNotFound + | ErrorKind::ReplicationSlotAlreadyExists + | ErrorKind::ReplicationSlotInvalid + ) + } + + /// Returns true if this error contains multiple errors + pub fn is_many(&self) -> bool { + matches!(self.repr, ErrorRepr::Many(_)) + } + + /// Returns the number of errors contained in this error + pub fn error_count(&self) -> usize { + match self.repr { + ErrorRepr::Many(ref errors) => errors.len(), + _ => 1, + } + } + + /// Returns an iterator over all errors (including nested ones) + pub fn iter_errors(&self) -> impl Iterator { + ErrorIterator::new(self) + } + + /// Flattens multiple errors into a single vector + pub fn flatten_errors(&self) -> Vec<&ETLError> { + let mut errors = Vec::new(); + self.collect_errors(&mut errors); + errors + } + + fn collect_errors<'a>(&'a self, errors: &mut Vec<&'a ETLError>) { + match self.repr { + ErrorRepr::Many(ref nested_errors) => { + for error in nested_errors { + error.collect_errors(errors); + } + } + _ => errors.push(self), + } + } +} + +/// Iterator over all errors in an ETLError tree +pub struct ErrorIterator<'a> { + stack: Vec<&'a ETLError>, +} + +impl<'a> ErrorIterator<'a> { + fn new(error: &'a ETLError) -> Self { + let mut stack = Vec::new(); + stack.push(error); + ErrorIterator { stack } + } +} + +impl<'a> Iterator for ErrorIterator<'a> { + type Item = &'a ETLError; + + fn next(&mut self) -> Option { + while let Some(error) = self.stack.pop() { + match &error.repr { + ErrorRepr::Many(ref errors) => { + // Add all errors to the stack in reverse order so they're processed in order + for error in errors.iter().rev() { + self.stack.push(error); + } + } + _ => return Some(error), + } + } + None + } } impl PartialEq for ETLError { fn eq(&self, other: &ETLError) -> bool { match (&self.repr, &other.repr) { - (ErrorRepr::WithDescription(kind_a, _), ErrorRepr::WithDescription(kind_b, _)) => kind_a == kind_b, + (ErrorRepr::WithDescription(kind_a, _), ErrorRepr::WithDescription(kind_b, _)) => { + kind_a == kind_b + } ( ErrorRepr::WithDescriptionAndDetail(kind_a, _, _), ErrorRepr::WithDescriptionAndDetail(kind_b, _, _), ) => kind_a == kind_b, + (ErrorRepr::Many(errors_a), ErrorRepr::Many(errors_b)) => { + errors_a.len() == errors_b.len() + && errors_a.iter().zip(errors_b.iter()).all(|(a, b)| a == b) + } _ => false, } } @@ -143,6 +270,20 @@ impl fmt::Display for ETLError { f.write_str(": ")?; detail.fmt(f) } + ErrorRepr::Many(ref errors) => { + if errors.is_empty() { + write!(f, "Multiple errors occurred (empty)")?; + } else if errors.len() == 1 { + // If there's only one error, just display it directly + errors[0].fmt(f)?; + } else { + write!(f, "Multiple errors occurred ({} total):", errors.len())?; + for (i, error) in errors.iter().enumerate() { + write!(f, "\n {}: {}", i + 1, error)?; + } + } + Ok(()) + } } } } @@ -156,7 +297,16 @@ impl fmt::Debug for ETLError { impl error::Error for ETLError { fn description(&self) -> &str { match self.repr { - ErrorRepr::WithDescription(_, desc) | ErrorRepr::WithDescriptionAndDetail(_, desc, _) => desc, + ErrorRepr::WithDescription(_, desc) + | ErrorRepr::WithDescriptionAndDetail(_, desc, _) => desc, + ErrorRepr::Many(ref errors) => { + if errors.is_empty() { + "Multiple errors occurred (empty)" + } else { + // Return the description of the first error + errors[0].description() + } + } } } } @@ -244,7 +394,7 @@ impl From for ETLError { ETLError { repr: ErrorRepr::WithDescriptionAndDetail( ErrorKind::ConversionError, - "Float parsing failed", + "Float parsing failed", err.to_string(), ), } @@ -259,7 +409,7 @@ impl From for ETLError { } else { ErrorKind::ConnectionFailed }; - + ETLError { repr: ErrorRepr::WithDescriptionAndDetail( kind, @@ -308,6 +458,18 @@ impl From for ETLError { } } +impl From for ETLError { + fn from(err: tokio::task::JoinError) -> ETLError { + ETLError { + repr: ErrorRepr::WithDescriptionAndDetail( + ErrorKind::WorkerError, + "Failed to join tokio task", + err.to_string(), + ), + } + } +} + // SQLx error conversion impl From for ETLError { fn from(err: sqlx::Error) -> ETLError { @@ -317,7 +479,7 @@ impl From for ETLError { sqlx::Error::PoolClosed | sqlx::Error::PoolTimedOut => ErrorKind::ConnectionFailed, _ => ErrorKind::QueryFailed, }; - + ETLError { repr: ErrorRepr::WithDescriptionAndDetail( kind, @@ -337,7 +499,7 @@ impl From for ETLError { gcp_bigquery_client::error::BQError::ResponseError { .. } => ErrorKind::QueryFailed, _ => ErrorKind::DestinationError, }; - + ETLError { repr: ErrorRepr::WithDescriptionAndDetail( kind, @@ -378,7 +540,7 @@ impl From for ETLError { fn from(err: crate::replication::stream::TableCopyStreamError) -> ETLError { ETLError { repr: ErrorRepr::WithDescriptionAndDetail( - ErrorKind::ReplicationError, + ErrorKind::TableSyncFailed, "Table copy stream operation failed", err.to_string(), ), @@ -390,7 +552,7 @@ impl From for ETLError { fn from(err: crate::replication::stream::EventsStreamError) -> ETLError { ETLError { repr: ErrorRepr::WithDescriptionAndDetail( - ErrorKind::ReplicationError, + ErrorKind::LogicalReplicationFailed, "Events stream operation failed", err.to_string(), ), @@ -414,7 +576,7 @@ impl From for ETLError { fn from(err: crate::replication::slot::SlotError) -> ETLError { ETLError { repr: ErrorRepr::WithDescriptionAndDetail( - ErrorKind::ReplicationError, + ErrorKind::ReplicationSlotInvalid, "Replication slot operation failed", err.to_string(), ), @@ -422,7 +584,67 @@ impl From for ETLError { } } -// Helpful macros for error creation (following Redis pattern) +// Missing From implementations for conversion error types +impl From for ETLError { + fn from(err: crate::conversions::hex::ByteaHexParseError) -> ETLError { + ETLError { + repr: ErrorRepr::WithDescriptionAndDetail( + ErrorKind::ConversionError, + "Hex parsing failed", + err.to_string(), + ), + } + } +} + +impl From for ETLError { + fn from(err: crate::conversions::table_row::TableRowConversionError) -> ETLError { + ETLError { + repr: ErrorRepr::WithDescriptionAndDetail( + ErrorKind::ConversionError, + "Table row conversion failed", + err.to_string(), + ), + } + } +} + +impl From for ETLError { + fn from(err: crate::conversions::text::FromTextError) -> ETLError { + ETLError { + repr: ErrorRepr::WithDescriptionAndDetail( + ErrorKind::ConversionError, + "Text conversion failed", + err.to_string(), + ), + } + } +} + +impl From for ETLError { + fn from(err: crate::conversions::text::ArrayParseError) -> ETLError { + ETLError { + repr: ErrorRepr::WithDescriptionAndDetail( + ErrorKind::ConversionError, + "Array parsing failed", + err.to_string(), + ), + } + } +} + +impl From for ETLError { + fn from(err: crate::conversions::bool::ParseBoolError) -> ETLError { + ETLError { + repr: ErrorRepr::WithDescriptionAndDetail( + ErrorKind::ConversionError, + "Boolean parsing failed", + err.to_string(), + ), + } + } +} + #[macro_export] macro_rules! etl_error { ($kind:expr, $desc:expr) => { @@ -433,7 +655,7 @@ macro_rules! etl_error { }; } -#[macro_export] +#[macro_export] macro_rules! bail { ($kind:expr, $desc:expr) => { return Err(etl_error!($kind, $desc)) @@ -492,11 +714,24 @@ mod tests { fn test_error_categories() { let connection_err = ETLError::from((ErrorKind::ConnectionFailed, "Connection failed")); let data_err = ETLError::from((ErrorKind::SchemaError, "Schema mismatch")); - + let replication_err = + ETLError::from((ErrorKind::ReplicationSlotNotFound, "Slot not found")); + let slot_err = ETLError::from((ErrorKind::ReplicationSlotAlreadyExists, "Slot exists")); + assert!(connection_err.is_connection_error()); assert!(!connection_err.is_data_error()); - + assert!(!connection_err.is_replication_error()); + assert!(!data_err.is_connection_error()); assert!(data_err.is_data_error()); + assert!(!data_err.is_replication_error()); + + assert!(replication_err.is_replication_error()); + assert!(replication_err.is_replication_slot_error()); + assert!(!replication_err.is_connection_error()); + + assert!(slot_err.is_replication_error()); + assert!(slot_err.is_replication_slot_error()); + assert!(!slot_err.is_data_error()); } -} \ No newline at end of file +} diff --git a/etl/src/lib.rs b/etl/src/lib.rs index e47a789dd..6282a29fd 100644 --- a/etl/src/lib.rs +++ b/etl/src/lib.rs @@ -13,6 +13,3 @@ pub mod state; #[cfg(any(test, feature = "test-utils"))] pub mod test_utils; pub mod workers; - -// Re-export main error types for convenience -pub use error::{ETLError, ETLResult, ErrorKind}; diff --git a/etl/src/pipeline.rs b/etl/src/pipeline.rs index eb6a3de9e..499386d1c 100644 --- a/etl/src/pipeline.rs +++ b/etl/src/pipeline.rs @@ -13,7 +13,7 @@ use crate::state::table::TableReplicationPhase; use crate::workers::apply::{ApplyWorker, ApplyWorkerHandle}; use crate::workers::base::{Worker, WorkerHandle}; use crate::workers::pool::TableSyncWorkerPool; - +use crate::{bail, etl_error}; #[derive(Debug)] enum PipelineWorkers { @@ -146,11 +146,15 @@ where "publication '{}' does not exist in the database", self.config.publication_name ); - return Err(ETLError::from(( + + bail!( ErrorKind::ConfigError, "Missing publication", - format!("The publication '{}' does not exist in the database", self.config.publication_name) - ))); + format!( + "The publication '{}' does not exist in the database", + self.config.publication_name + ) + ); } let table_ids = replication_client @@ -208,7 +212,7 @@ where // We wait for all table sync workers to finish. let table_sync_workers_result = pool.wait_all().await; if let Err(err) = table_sync_workers_result { - errors.extend(err.0); + errors.push(err); info!("one or more table sync workers failed with an error"); } else { @@ -216,11 +220,7 @@ where } if !errors.is_empty() { - return Err(ETLError::from(( - ErrorKind::WorkerError, - "One or more workers failed", - format!("Workers failed with {} errors", errors.len()) - ))); + return Err(errors.into()); } Ok(()) diff --git a/etl/src/replication/apply.rs b/etl/src/replication/apply.rs index 9320e5536..6121e7fae 100644 --- a/etl/src/replication/apply.rs +++ b/etl/src/replication/apply.rs @@ -1,7 +1,7 @@ use crate::concurrency::shutdown::ShutdownRx; use crate::conversions::event::{Event, EventType, convert_message_to_event}; use crate::destination::base::Destination; -use crate::error::{ETLError, ErrorKind, ErrorRepr}; +use crate::error::{ETLError, ETLResult, ErrorKind, ErrorRepr}; use crate::pipeline::PipelineId; use crate::replication::client::PgReplicationClient; use crate::replication::slot::get_slot_name; @@ -27,7 +27,6 @@ use tracing::{debug, info}; /// events or shutdown signal are received. const REFRESH_INTERVAL: Duration = Duration::from_millis(1000); - #[derive(Debug, Copy, Clone)] pub enum ApplyLoopResult { ApplyStopped, @@ -209,7 +208,7 @@ pub async fn start_apply_loop( hook: T, mut shutdown_rx: ShutdownRx, mut force_syncing_tables_rx: Option, -) -> Result +) -> ETLResult where D: Destination + Clone + Send + 'static, T: ApplyLoopHook, @@ -336,7 +335,7 @@ async fn handle_replication_message_batch( hook: &T, max_batch_size: usize, max_batch_fill_duration: Duration, -) -> Result +) -> ETLResult where D: Destination + Clone + Send + 'static, T: ApplyLoopHook, @@ -372,7 +371,7 @@ async fn try_send_batch( hook: &T, max_batch_size: usize, max_batch_fill_duration: Duration, -) -> Result +) -> ETLResult where D: Destination + Clone + Send + 'static, T: ApplyLoopHook, @@ -455,7 +454,7 @@ async fn handle_replication_message( message: ReplicationMessage, schema_cache: &SchemaCache, hook: &T, -) -> Result +) -> ETLResult where T: ApplyLoopHook, ETLError: From<::Error>, @@ -512,7 +511,7 @@ async fn handle_logical_replication_message( message: LogicalReplicationMessage, schema_cache: &SchemaCache, hook: &T, -) -> Result +) -> ETLResult where T: ApplyLoopHook, ETLError: From<::Error>, @@ -560,19 +559,20 @@ where } #[allow(clippy::result_large_err)] -fn get_commit_lsn( - state: &ApplyLoopState, - message: &LogicalReplicationMessage, -) -> Result { +fn get_commit_lsn(state: &ApplyLoopState, message: &LogicalReplicationMessage) -> ETLResult { // If we are in a `Begin` message, the `commit_lsn` is the `final_lsn` of the payload, in all the // other cases we read the `remote_final_lsn` which should be always set in case we are within or // at the end of a transaction (meaning that the event type is different from `Begin`). if let LogicalReplicationMessage::Begin(message) = message { Ok(PgLsn::from(message.final_lsn())) } else { - state - .remote_final_lsn - .ok_or_else(|| ETLError { repr: ErrorRepr::WithDescriptionAndDetail(ErrorKind::InvalidState, "Invalid transaction", "A transaction should have started for get_commit_lsn to be performed".to_string()) }) + state.remote_final_lsn.ok_or_else(|| ETLError { + repr: ErrorRepr::WithDescriptionAndDetail( + ErrorKind::InvalidState, + "Invalid transaction", + "A transaction should have started for get_commit_lsn to be performed".to_string(), + ), + }) } } @@ -580,9 +580,19 @@ async fn handle_begin_message( state: &mut ApplyLoopState, event: Event, message: &protocol::BeginBody, -) -> Result { +) -> ETLResult { let EventType::Begin = event.event_type() else { - return Err(ETLError { repr: ErrorRepr::WithDescriptionAndDetail(ErrorKind::ValidationError, "Invalid event", format!("An invalid event {:?} was received (expected {:?})", event, EventType::Begin)) }); + return Err(ETLError { + repr: ErrorRepr::WithDescriptionAndDetail( + ErrorKind::ValidationError, + "Invalid event", + format!( + "An invalid event {:?} was received (expected {:?})", + event, + EventType::Begin + ), + ), + }); }; // We track the final lsn of this transaction, which should be equal to the `commit_lsn` of the @@ -603,20 +613,37 @@ async fn handle_commit_message( event: Event, message: &protocol::CommitBody, hook: &T, -) -> Result +) -> ETLResult where T: ApplyLoopHook, ETLError: From<::Error>, { let EventType::Commit = event.event_type() else { - return Err(ETLError { repr: ErrorRepr::WithDescriptionAndDetail(ErrorKind::ValidationError, "Invalid event", format!("An invalid event {:?} was received (expected {:?})", event, EventType::Commit)) }); + return Err(ETLError { + repr: ErrorRepr::WithDescriptionAndDetail( + ErrorKind::ValidationError, + "Invalid event", + format!( + "An invalid event {:?} was received (expected {:?})", + event, + EventType::Commit + ), + ), + }); }; // We take the LSN that belongs to the current transaction, however, if there is no // LSN, it means that a `Begin` message was not received before this `Commit` which means // we are in an inconsistent state. let Some(remote_final_lsn) = state.remote_final_lsn.take() else { - return Err(ETLError { repr: ErrorRepr::WithDescriptionAndDetail(ErrorKind::InvalidState, "Invalid transaction", "A transaction should have started for handle_commit_message to be performed".to_string()) }); + return Err(ETLError { + repr: ErrorRepr::WithDescriptionAndDetail( + ErrorKind::InvalidState, + "Invalid transaction", + "A transaction should have started for handle_commit_message to be performed" + .to_string(), + ), + }); }; // If the commit lsn of the message is different from the remote final lsn, it means that the @@ -624,7 +651,16 @@ where // we want to bail assuming we are in an inconsistent state. let commit_lsn = PgLsn::from(message.commit_lsn()); if commit_lsn != remote_final_lsn { - return Err(ETLError { repr: ErrorRepr::WithDescriptionAndDetail(ErrorKind::ValidationError, "Invalid commit LSN", format!("Incorrect commit LSN {} in COMMIT message (expected {})", commit_lsn, remote_final_lsn)) }); + return Err(ETLError { + repr: ErrorRepr::WithDescriptionAndDetail( + ErrorKind::ValidationError, + "Invalid commit LSN", + format!( + "Incorrect commit LSN {} in COMMIT message (expected {})", + commit_lsn, remote_final_lsn + ), + ), + }); } let end_lsn = PgLsn::from(message.end_lsn()); @@ -666,17 +702,34 @@ async fn handle_relation_message( message: &protocol::RelationBody, schema_cache: &SchemaCache, hook: &T, -) -> Result +) -> ETLResult where T: ApplyLoopHook, ETLError: From<::Error>, { let Event::Relation(event) = event else { - return Err(ETLError { repr: ErrorRepr::WithDescriptionAndDetail(ErrorKind::ValidationError, "Invalid event", format!("An invalid event {:?} was received (expected {:?})", event, EventType::Relation)) }); + return Err(ETLError { + repr: ErrorRepr::WithDescriptionAndDetail( + ErrorKind::ValidationError, + "Invalid event", + format!( + "An invalid event {:?} was received (expected {:?})", + event, + EventType::Relation + ), + ), + }); }; let Some(remote_final_lsn) = state.remote_final_lsn else { - return Err(ETLError { repr: ErrorRepr::WithDescriptionAndDetail(ErrorKind::InvalidState, "Invalid transaction", "A transaction should have started for handle_relation_message to be performed".to_string()) }); + return Err(ETLError { + repr: ErrorRepr::WithDescriptionAndDetail( + ErrorKind::InvalidState, + "Invalid transaction", + "A transaction should have started for handle_relation_message to be performed" + .to_string(), + ), + }); }; if !hook @@ -691,7 +744,16 @@ where // TODO: explore how to deal with applying relation messages to the schema (creating it if missing). let schema_cache = schema_cache.lock_inner().await; let Some(existing_table_schema) = schema_cache.get_table_schema_ref(&message.rel_id()) else { - return Err(ETLError { repr: ErrorRepr::WithDescriptionAndDetail(ErrorKind::SchemaError, "Missing table schema", format!("The table schema for table {} was not found in the cache", message.rel_id())) }); + return Err(ETLError { + repr: ErrorRepr::WithDescriptionAndDetail( + ErrorKind::SchemaError, + "Missing table schema", + format!( + "The table schema for table {} was not found in the cache", + message.rel_id() + ), + ), + }); }; // We compare the table schema from the relation message with the existing schema (if any). @@ -718,17 +780,34 @@ async fn handle_insert_message( event: Event, message: &protocol::InsertBody, hook: &T, -) -> Result +) -> ETLResult where T: ApplyLoopHook, ETLError: From<::Error>, { let Event::Insert(event) = event else { - return Err(ETLError { repr: ErrorRepr::WithDescriptionAndDetail(ErrorKind::ValidationError, "Invalid event", format!("An invalid event {:?} was received (expected {:?})", event, EventType::Insert)) }); + return Err(ETLError { + repr: ErrorRepr::WithDescriptionAndDetail( + ErrorKind::ValidationError, + "Invalid event", + format!( + "An invalid event {:?} was received (expected {:?})", + event, + EventType::Insert + ), + ), + }); }; let Some(remote_final_lsn) = state.remote_final_lsn else { - return Err(ETLError { repr: ErrorRepr::WithDescriptionAndDetail(ErrorKind::InvalidState, "Invalid transaction", "A transaction should have started for handle_insert_message to be performed".to_string()) }); + return Err(ETLError { + repr: ErrorRepr::WithDescriptionAndDetail( + ErrorKind::InvalidState, + "Invalid transaction", + "A transaction should have started for handle_insert_message to be performed" + .to_string(), + ), + }); }; if !hook @@ -751,17 +830,34 @@ async fn handle_update_message( event: Event, message: &protocol::UpdateBody, hook: &T, -) -> Result +) -> ETLResult where T: ApplyLoopHook, ETLError: From<::Error>, { let Event::Update(event) = event else { - return Err(ETLError { repr: ErrorRepr::WithDescriptionAndDetail(ErrorKind::ValidationError, "Invalid event", format!("An invalid event {:?} was received (expected {:?})", event, EventType::Update)) }); + return Err(ETLError { + repr: ErrorRepr::WithDescriptionAndDetail( + ErrorKind::ValidationError, + "Invalid event", + format!( + "An invalid event {:?} was received (expected {:?})", + event, + EventType::Update + ), + ), + }); }; let Some(remote_final_lsn) = state.remote_final_lsn else { - return Err(ETLError { repr: ErrorRepr::WithDescriptionAndDetail(ErrorKind::InvalidState, "Invalid transaction", "A transaction should have started for handle_update_message to be performed".to_string()) }); + return Err(ETLError { + repr: ErrorRepr::WithDescriptionAndDetail( + ErrorKind::InvalidState, + "Invalid transaction", + "A transaction should have started for handle_update_message to be performed" + .to_string(), + ), + }); }; if !hook @@ -784,17 +880,34 @@ async fn handle_delete_message( event: Event, message: &protocol::DeleteBody, hook: &T, -) -> Result +) -> ETLResult where T: ApplyLoopHook, ETLError: From<::Error>, { let Event::Delete(event) = event else { - return Err(ETLError { repr: ErrorRepr::WithDescriptionAndDetail(ErrorKind::ValidationError, "Invalid event", format!("An invalid event {:?} was received (expected {:?})", event, EventType::Delete)) }); + return Err(ETLError { + repr: ErrorRepr::WithDescriptionAndDetail( + ErrorKind::ValidationError, + "Invalid event", + format!( + "An invalid event {:?} was received (expected {:?})", + event, + EventType::Delete + ), + ), + }); }; let Some(remote_final_lsn) = state.remote_final_lsn else { - return Err(ETLError { repr: ErrorRepr::WithDescriptionAndDetail(ErrorKind::InvalidState, "Invalid transaction", "A transaction should have started for handle_delete_message to be performed".to_string()) }); + return Err(ETLError { + repr: ErrorRepr::WithDescriptionAndDetail( + ErrorKind::InvalidState, + "Invalid transaction", + "A transaction should have started for handle_delete_message to be performed" + .to_string(), + ), + }); }; if !hook @@ -817,17 +930,34 @@ async fn handle_truncate_message( event: Event, message: &protocol::TruncateBody, hook: &T, -) -> Result +) -> ETLResult where T: ApplyLoopHook, ETLError: From<::Error>, { let Event::Truncate(mut event) = event else { - return Err(ETLError { repr: ErrorRepr::WithDescriptionAndDetail(ErrorKind::ValidationError, "Invalid event", format!("An invalid event {:?} was received (expected {:?})", event, EventType::Truncate)) }); + return Err(ETLError { + repr: ErrorRepr::WithDescriptionAndDetail( + ErrorKind::ValidationError, + "Invalid event", + format!( + "An invalid event {:?} was received (expected {:?})", + event, + EventType::Truncate + ), + ), + }); }; let Some(remote_final_lsn) = state.remote_final_lsn else { - return Err(ETLError { repr: ErrorRepr::WithDescriptionAndDetail(ErrorKind::InvalidState, "Invalid transaction", "A transaction should have started for handle_truncate_message to be performed".to_string()) }); + return Err(ETLError { + repr: ErrorRepr::WithDescriptionAndDetail( + ErrorKind::InvalidState, + "Invalid transaction", + "A transaction should have started for handle_truncate_message to be performed" + .to_string(), + ), + }); }; let mut rel_ids = Vec::with_capacity(message.rel_ids().len()); diff --git a/etl/src/replication/client.rs b/etl/src/replication/client.rs index cbcc51103..95bb00ab4 100644 --- a/etl/src/replication/client.rs +++ b/etl/src/replication/client.rs @@ -1,3 +1,5 @@ +use crate::error::{ETLError, ETLResult, ErrorKind}; +use crate::{bail, etl_error}; use config::shared::{IntoConnectOptions, PgConnectionConfig}; use pg_escape::{quote_identifier, quote_literal}; use postgres::schema::{ColumnSchema, TableId, TableName, TableSchema}; @@ -8,7 +10,6 @@ use std::collections::HashMap; use std::fmt; use std::io::BufReader; use std::sync::Arc; -use crate::error::{ETLError, ETLResult, ErrorKind}; use tokio_postgres::error::SqlState; use tokio_postgres::tls::MakeTlsConnect; use tokio_postgres::{ @@ -41,7 +42,6 @@ where tokio::spawn(task); } - #[derive(Debug, Clone)] pub struct CreateSlotResult { pub consistent_point: PgLsn, @@ -257,7 +257,11 @@ impl PgReplicationClient { } } - Err(ETLError::from((ErrorKind::ReplicationError, "Replication slot not found", format!("Replication slot '{}' not found in database", slot_name)))) + Err(etl_error!( + ErrorKind::ReplicationSlotNotFound, + "Replication slot not found", + format!("Replication slot '{}' not found in database", slot_name) + )) } /// Gets an existing replication slot or creates a new one if it doesn't exist. @@ -279,7 +283,7 @@ impl PgReplicationClient { Ok(GetOrCreateSlotResult::GetSlot(slot)) } - Err(err) if err.kind() == ErrorKind::ReplicationError && err.detail().map_or(false, |d| d.contains("not found")) => { + Err(err) if err.kind() == ErrorKind::ReplicationSlotNotFound => { info!("creating new replication slot '{}'", slot_name); let create_result = self.create_slot_internal(slot_name, false).await?; @@ -314,7 +318,11 @@ impl PgReplicationClient { "attempted to delete non-existent replication slot '{}'", slot_name ); - return Err(ETLError::from((ErrorKind::ReplicationError, "Replication slot not found", format!("Replication slot '{}' not found in database", slot_name)))); + bail!( + ErrorKind::ReplicationSlotNotFound, + "Replication slot not found", + format!("Replication slot '{}' not found in database", slot_name) + ); } } @@ -489,7 +497,14 @@ impl PgReplicationClient { Err(err) => { if let Some(code) = err.code() { if *code == SqlState::DUPLICATE_OBJECT { - return Err(ETLError::from((ErrorKind::ReplicationError, "Replication slot already exists", format!("Replication slot '{}' already exists in database", slot_name)))); + bail!( + ErrorKind::ReplicationSlotAlreadyExists, + "Replication slot already exists", + format!( + "Replication slot '{}' already exists in database", + slot_name + ) + ); } } @@ -497,7 +512,10 @@ impl PgReplicationClient { } } - Err(ETLError::from((ErrorKind::ReplicationError, "Failed to create replication slot"))) + Err(etl_error!( + ErrorKind::ReplicationSlotAlreadyExists, + "Failed to create replication slot" + )) } /// Retrieves schema information for multiple tables. @@ -574,7 +592,11 @@ impl PgReplicationClient { } } - Err(ETLError::from((ErrorKind::SchemaError, "Table not found", format!("Table not found in database (oid: {})", table_id)))) + Err(etl_error!( + ErrorKind::SchemaError, + "Table not found", + format!("Table not found in database (oid: {})", table_id) + )) } /// Retrieves schema information for all columns in a table. @@ -697,12 +719,24 @@ impl PgReplicationClient { where T::Err: fmt::Debug, { - let value = row - .try_get(column_name)? - .ok_or(ETLError::from((ErrorKind::SchemaError, "Column not found", format!("Column '{}' not found in table '{}'", column_name, table_name))))?; + let value = row.try_get(column_name)?.ok_or(etl_error!( + ErrorKind::SchemaError, + "Column not found", + format!( + "Column '{}' not found in table '{}'", + column_name, table_name + ) + ))?; value.parse().map_err(|e: T::Err| { - ETLError::from((ErrorKind::ConversionError, "Column parsing failed", format!("Failed to parse value from column '{}' in table '{}': {:?}", column_name, table_name, e))) + etl_error!( + ErrorKind::ConversionError, + "Column parsing failed", + format!( + "Failed to parse value from column '{}' in table '{}': {:?}", + column_name, table_name, e + ) + ) }) } } diff --git a/etl/src/replication/table_sync.rs b/etl/src/replication/table_sync.rs index 51ad39fc7..81482e319 100644 --- a/etl/src/replication/table_sync.rs +++ b/etl/src/replication/table_sync.rs @@ -25,7 +25,7 @@ use crate::state::store::base::StateStore; use crate::state::table::{TableReplicationPhase, TableReplicationPhaseType}; use crate::workers::base::WorkerType; use crate::workers::table_sync::TableSyncWorkerState; - +use crate::{bail, etl_error}; #[derive(Debug)] pub enum TableSyncResult { @@ -46,7 +46,7 @@ pub async fn start_table_sync( destination: D, shutdown_rx: ShutdownRx, force_syncing_tables_tx: SignalTx, -) -> Result +) -> ETLResult where S: StateStore + Clone + Send + 'static, D: Destination + Clone + Send + 'static, @@ -87,7 +87,14 @@ where phase_type, table_id ); - return Err(ETLError::from((ErrorKind::InvalidState, "Invalid replication phase", format!("Invalid replication phase '{}': expected Init, DataSync, or FinishedCopy", phase_type)))); + bail!( + ErrorKind::InvalidState, + "Invalid replication phase", + format!( + "Invalid replication phase '{}': expected Init, DataSync, or FinishedCopy", + phase_type + ) + ); } phase_type @@ -121,7 +128,7 @@ where // before starting a table copy. if let Err(err) = replication_client.delete_slot(&slot_name).await { // If the slot is not found, we are safe to continue, for any other error, we bail. - if err.kind() != ErrorKind::ReplicationError || !err.detail().map_or(false, |d| d.contains("not found")) { + if err.kind() != ErrorKind::ReplicationSlotNotFound { return Err(err.into()); } } @@ -165,7 +172,11 @@ where state_store .update_table_replication_state(table_id, TableReplicationPhase::Skipped) .await?; - return Err(ETLError::from((ErrorKind::SchemaError, "Missing primary key", format!("table {} has no primary key", table_schema.name)))); + bail!( + ErrorKind::SchemaError, + "Missing primary key", + format!("table {} has no primary key", table_schema.name) + ); } schema_cache.add_table_schema(table_schema.clone()).await; diff --git a/etl/src/state/store/base.rs b/etl/src/state/store/base.rs index 9b9532018..c43bd7a74 100644 --- a/etl/src/state/store/base.rs +++ b/etl/src/state/store/base.rs @@ -1,14 +1,8 @@ +use crate::error::{ETLError, ETLResult, ErrorKind}; use postgres::schema::TableId; use std::{collections::HashMap, future::Future}; -use crate::error::{ETLError, ETLResult, ErrorKind}; - -use crate::{ - replication::slot::SlotError, - state::{ - table::TableReplicationPhase, - }, -}; +use crate::{replication::slot::SlotError, state::table::TableReplicationPhase}; /// This trait represents a state store for the replication state of all tables. /// It assumes that the implementers keep a cache of the state to avoid having @@ -33,9 +27,7 @@ pub trait StateStore { /// and then use only the `get_X` methods to access the state. Updating the state /// by calling the `update_table_replication_state` updates in both the cache and /// the persistent store, so no need to ever load the state again. - fn load_table_replication_states( - &self, - ) -> impl Future> + Send; + fn load_table_replication_states(&self) -> impl Future> + Send; /// Updates the table replicate state for a table with `table_id` in both the cache as well as /// the persistent store. diff --git a/etl/src/state/store/memory.rs b/etl/src/state/store/memory.rs index 70d4df2e3..74d1218eb 100644 --- a/etl/src/state/store/memory.rs +++ b/etl/src/state/store/memory.rs @@ -3,8 +3,8 @@ use std::collections::HashMap; use std::sync::Arc; use tokio::sync::Mutex; -use crate::state::store::base::StateStore; use crate::error::ETLResult; +use crate::state::store::base::StateStore; use crate::state::table::TableReplicationPhase; #[derive(Debug)] diff --git a/etl/src/state/store/notify.rs b/etl/src/state/store/notify.rs index aa3952046..befcd8bfd 100644 --- a/etl/src/state/store/notify.rs +++ b/etl/src/state/store/notify.rs @@ -6,11 +6,11 @@ use tokio::{ sync::{Notify, RwLock}, }; +use crate::error::ETLResult; use crate::state::{ store::base::StateStore, table::{TableReplicationPhase, TableReplicationPhaseType}, }; -use crate::error::ETLResult; #[derive(Debug, Clone, Copy, PartialEq, Eq, Hash)] pub enum StateStoreMethod { diff --git a/etl/src/state/store/postgres.rs b/etl/src/state/store/postgres.rs index 886100590..b9a95b317 100644 --- a/etl/src/state/store/postgres.rs +++ b/etl/src/state/store/postgres.rs @@ -1,5 +1,7 @@ use std::{collections::HashMap, sync::Arc}; +use crate::error::{ETLError, ETLResult, ErrorKind}; +use crate::{bail, etl_error}; use config::shared::PgConnectionConfig; use postgres::replication::{ TableReplicationState, TableReplicationStateRow, connect_to_source_database, @@ -7,17 +9,13 @@ use postgres::replication::{ }; use postgres::schema::TableId; use sqlx::PgPool; -use crate::error::{ETLError, ETLResult, ErrorKind}; use tokio::sync::Mutex; use tokio_postgres::types::PgLsn; use tracing::{debug, info}; use crate::{ pipeline::PipelineId, - state::{ - store::base::StateStore, - table::TableReplicationPhase, - }, + state::{store::base::StateStore, table::TableReplicationPhase}, }; const NUM_POOL_CONNECTIONS: u32 = 1; @@ -36,13 +34,16 @@ impl TryFrom for (TableReplicationState, Option) TableReplicationPhase::Ready => (TableReplicationState::Ready, None), TableReplicationPhase::Skipped => (TableReplicationState::Skipped, None), TableReplicationPhase::SyncWait | TableReplicationPhase::Catchup { .. } => { - return Err(ETLError::from((ErrorKind::InvalidState, "In-memory phase error", "In-memory table replication phase can't be saved in the state store".to_string()))); + bail!( + ErrorKind::InvalidState, + "In-memory phase error", + "In-memory table replication phase can't be saved in the state store" + ); } }) } } - #[derive(Debug)] struct Inner { table_states: HashMap, @@ -114,12 +115,23 @@ impl PostgresStateStore { TableReplicationState::FinishedCopy => TableReplicationPhase::FinishedCopy, TableReplicationState::SyncDone => match sync_done_lsn { Some(lsn_str) => { - let lsn = lsn_str - .parse::() - .map_err(|_| ETLError::from((ErrorKind::ValidationError, "Invalid LSN", format!("Invalid confirmed flush lsn value in state store: {}", lsn_str))))?; + let lsn = lsn_str.parse::().map_err(|_| { + etl_error!( + ErrorKind::ValidationError, + "Invalid LSN", + format!( + "Invalid confirmed flush lsn value in state store: {}", + lsn_str + ) + ) + })?; TableReplicationPhase::SyncDone { lsn } } - None => return Err(ETLError::from((ErrorKind::ValidationError, "Missing LSN", "Lsn can't be missing from the state store if state is SyncDone".to_string()))), + None => bail!( + ErrorKind::ValidationError, + "Missing LSN", + "Lsn can't be missing from the state store if state is SyncDone" + ), }, TableReplicationState::Ready => TableReplicationPhase::Ready, TableReplicationState::Skipped => TableReplicationPhase::Skipped, diff --git a/etl/src/workers/apply.rs b/etl/src/workers/apply.rs index 5ea5ba169..f74e0b7d0 100644 --- a/etl/src/workers/apply.rs +++ b/etl/src/workers/apply.rs @@ -1,7 +1,7 @@ +use crate::error::{ETLError, ETLResult}; use config::shared::PipelineConfig; use postgres::schema::TableId; use std::sync::Arc; -use crate::error::{ETLError, ETLResult, ErrorKind}; use tokio::sync::Semaphore; use tokio::task::JoinHandle; use tokio_postgres::types::PgLsn; @@ -18,12 +18,9 @@ use crate::replication::slot::{SlotError, get_slot_name}; use crate::schema::cache::SchemaCache; use crate::state::store::base::StateStore; use crate::state::table::{TableReplicationPhase, TableReplicationPhaseType}; -use crate::workers::base::{Worker, WorkerHandle, WorkerType, WorkerWaitError}; +use crate::workers::base::{Worker, WorkerHandle, WorkerType}; use crate::workers::pool::TableSyncWorkerPool; -use crate::workers::table_sync::{ - TableSyncWorker, TableSyncWorkerState, -}; - +use crate::workers::table_sync::{TableSyncWorker, TableSyncWorkerState}; #[derive(Debug)] pub struct ApplyWorkerHandle { @@ -33,7 +30,7 @@ pub struct ApplyWorkerHandle { impl WorkerHandle<()> for ApplyWorkerHandle { fn state(&self) {} - async fn wait(mut self) -> Result<(), WorkerWaitError> { + async fn wait(mut self) -> ETLResult<()> { let Some(handle) = self.handle.take() else { return Ok(()); }; @@ -91,7 +88,7 @@ where { type Error = ETLError; - async fn start(self) -> Result { + async fn start(self) -> ETLResult { info!("starting apply worker"); let apply_worker_span = tracing::info_span!( @@ -145,7 +142,7 @@ where async fn get_start_lsn( pipeline_id: PipelineId, replication_client: &PgReplicationClient, -) -> Result { +) -> ETLResult { let slot_name = get_slot_name(pipeline_id, WorkerType::Apply)?; // TODO: validate that we only create the slot when we first start replication which // means when all tables are in the Init state. In any other case we should raise an @@ -224,11 +221,7 @@ where ) } - async fn handle_syncing_table( - &self, - table_id: TableId, - current_lsn: PgLsn, - ) -> Result { + async fn handle_syncing_table(&self, table_id: TableId, current_lsn: PgLsn) -> ETLResult { let mut pool = self.pool.lock().await; let table_sync_worker_state = pool.get_active_worker_state(table_id); @@ -249,7 +242,7 @@ where table_id: TableId, table_sync_worker_state: TableSyncWorkerState, current_lsn: PgLsn, - ) -> Result { + ) -> ETLResult { let mut catchup_started = false; { let mut inner = table_sync_worker_state.get_inner().lock().await; diff --git a/etl/src/workers/base.rs b/etl/src/workers/base.rs index aca2ef2e3..ef105bdb9 100644 --- a/etl/src/workers/base.rs +++ b/etl/src/workers/base.rs @@ -1,54 +1,7 @@ use postgres::schema::TableId; -use std::fmt; use std::future::Future; -use thiserror::Error; -use tokio::task; -use crate::error::ETLError; - -/// Represents all possible errors that can occur while waiting for a worker to complete. -/// -/// This enum is used to distinguish between different failure modes that may arise during -/// the execution of asynchronous worker tasks, including panics, explicit errors, and -/// silent failures that are internally handled but still indicate abnormal termination. -#[derive(Debug, Error)] -pub enum WorkerWaitError { - /// The worker's task panicked, causing its join handle to return an error. - /// - /// This typically indicates a bug or unrecoverable condition in the worker's logic. - #[error("Worker task panicked or was forcefully aborted: {0}")] - WorkerPanicked(#[from] task::JoinError), - - /// The worker's task failed internally, but the error was caught and not propagated. - /// - /// This can occur when using abstractions like `ReactiveFuture`, which catch both panics and - /// `Result::Err` values, and then notify the pool about the failure without letting it bubble up. - #[error("Worker task failed internally and the error was silently handled: {0}")] - WorkerSilentlyFailed(String), - - /// A worker encountered an error that was propagated via the handle's return value. - /// - /// This variant wraps the specific error returned by the worker. - #[error("Worker terminated with an error: {0}")] - WorkerFailed(#[from] ETLError), -} - -#[derive(Debug)] -pub struct WorkerWaitErrors(pub Vec); - -impl fmt::Display for WorkerWaitErrors { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - if self.0.is_empty() { - write!(f, "no worker failed.") - } else { - writeln!(f, "the workers failed with the following errors:")?; - for (i, err) in self.0.iter().enumerate() { - writeln!(f, " {}: {}", i + 1, err)?; - } - Ok(()) - } - } -} +use crate::error::ETLResult; /// The type of worker that is currently running. /// @@ -92,5 +45,5 @@ pub trait WorkerHandle { /// /// The future resolves to a [`Result`] indicating whether the worker completed successfully /// or encountered an error. - fn wait(self) -> impl Future> + Send; + fn wait(self) -> impl Future> + Send; } diff --git a/etl/src/workers/pool.rs b/etl/src/workers/pool.rs index c1485cc6a..5102c313e 100644 --- a/etl/src/workers/pool.rs +++ b/etl/src/workers/pool.rs @@ -8,11 +8,11 @@ use tracing::{debug, warn}; use crate::concurrency::future::ReactiveFutureCallback; use crate::destination::base::Destination; +use crate::error::{ETLResult, ErrorKind}; +use crate::etl_error; use crate::state::store::base::StateStore; -use crate::workers::base::{Worker, WorkerHandle, WorkerWaitError, WorkerWaitErrors}; -use crate::workers::table_sync::{ - TableSyncWorker, TableSyncWorkerHandle, TableSyncWorkerState, -}; +use crate::workers::base::{Worker, WorkerHandle}; +use crate::workers::table_sync::{TableSyncWorker, TableSyncWorkerHandle, TableSyncWorkerState}; #[derive(Debug)] pub enum TableSyncWorkerInactiveReason { @@ -43,10 +43,7 @@ impl TableSyncWorkerPoolInner { } } - pub async fn start_worker( - &mut self, - worker: TableSyncWorker, - ) -> crate::error::ETLResult + pub async fn start_worker(&mut self, worker: TableSyncWorker) -> ETLResult where S: StateStore + Clone + Send + Sync + 'static, D: Destination + Clone + Send + Sync + 'static, @@ -95,7 +92,7 @@ impl TableSyncWorkerPoolInner { } } - pub async fn wait_all(&mut self) -> Result>, WorkerWaitErrors> { + pub async fn wait_all(&mut self) -> ETLResult>> { // If there are active workers, we return the notify, signaling that not all of them are // ready. // @@ -124,16 +121,20 @@ impl TableSyncWorkerPoolInner { // This should not happen since right now the `ReactiveFuture` is configured to // re-propagate the error after marking a table sync worker as finished. if let TableSyncWorkerInactiveReason::Errored(err) = finish { - errors.push(WorkerWaitError::WorkerSilentlyFailed(err)); + errors.push(etl_error!( + ErrorKind::WorkerError, + "Table sync worker failed", + err + )); } } } - if errors.is_empty() { - Ok(None) - } else { - Err(WorkerWaitErrors(errors)) + if !errors.is_empty() { + return Err(errors.into()); } + + Ok(None) } } @@ -163,7 +164,7 @@ impl TableSyncWorkerPool { self.inner.clone() } - pub async fn wait_all(&self) -> Result<(), WorkerWaitErrors> { + pub async fn wait_all(&self) -> ETLResult<()> { loop { // We try first to wait for all workers to be finished, in case there are still active // workers, we get back a `Notify` which we will use to try again once new workers reported diff --git a/etl/src/workers/table_sync.rs b/etl/src/workers/table_sync.rs index e51d384d6..1319c66f6 100644 --- a/etl/src/workers/table_sync.rs +++ b/etl/src/workers/table_sync.rs @@ -1,8 +1,9 @@ +use crate::error::{ETLError, ETLResult, ErrorKind}; +use crate::{bail, etl_error}; use config::shared::PipelineConfig; use postgres::schema::TableId; use std::sync::Arc; use std::time::Duration; -use crate::error::{ETLError, ETLResult, ErrorKind}; use tokio::sync::{AcquireError, Mutex, MutexGuard, Notify, Semaphore}; use tokio::task::JoinHandle; use tokio_postgres::types::PgLsn; @@ -20,7 +21,7 @@ use crate::replication::table_sync::{TableSyncResult, start_table_sync}; use crate::schema::cache::SchemaCache; use crate::state::store::base::StateStore; use crate::state::table::{TableReplicationPhase, TableReplicationPhaseType}; -use crate::workers::base::{Worker, WorkerHandle, WorkerType, WorkerWaitError}; +use crate::workers::base::{Worker, WorkerHandle, WorkerType}; use crate::workers::pool::TableSyncWorkerPool; /// Maximum time to wait for a phase change before trying again. @@ -34,7 +35,6 @@ const PHASE_CHANGE_REFRESH_FREQUENCY: Duration = Duration::from_millis(100); /// of new tables. const MAX_DELETE_SLOT_WAIT: Duration = Duration::from_secs(30); - #[derive(Debug)] pub struct TableSyncWorkerStateInner { table_id: TableId, @@ -197,7 +197,7 @@ impl WorkerHandle for TableSyncWorkerHandle { self.state.clone() } - async fn wait(mut self) -> Result<(), WorkerWaitError> { + async fn wait(mut self) -> ETLResult<()> { let Some(handle) = self.handle.take() else { return Ok(()); }; @@ -262,7 +262,7 @@ where { type Error = ETLError; - async fn start(mut self) -> Result { + async fn start(mut self) -> ETLResult { info!("starting table sync worker for table {}", self.table_id); let Some(table_replication_phase) = self @@ -275,7 +275,14 @@ where self.table_id ); - return Err(ETLError::from((ErrorKind::InvalidState, "Replication state missing", format!("The replication state is missing for table {}", self.table_id)))); + bail!( + ErrorKind::InvalidState, + "Replication state missing", + format!( + "The replication state is missing for table {}", + self.table_id + ) + ); }; info!( @@ -397,7 +404,8 @@ where // We spawn the table sync worker with a safe future, so that we can have controlled teardown // on completion or error. - let fut = ReactiveFuture::new(table_sync_worker, self.table_id, self.pool.get_inner()) + // TODO: we want to implement a custom callback source which can skip tables and react to panics. + let fut = ReactiveFuture::wrap(table_sync_worker, self.table_id, self.pool.get_inner()) .instrument(table_sync_worker_span); let handle = tokio::spawn(fut); @@ -437,11 +445,7 @@ where /// /// Returns `Ok(false)` when the worker is done with its work, signaling the caller that the apply /// loop should be stopped. - async fn try_advance_phase( - &self, - current_lsn: PgLsn, - update_state: bool, - ) -> Result { + async fn try_advance_phase(&self, current_lsn: PgLsn, update_state: bool) -> ETLResult { let mut inner = self.table_sync_worker_state.get_inner().lock().await; // If we caught up with the lsn, we mark this table as `SyncDone` and stop the worker. diff --git a/etl/tests/integration/replication_test.rs b/etl/tests/integration/replication_test.rs index 33e74e1ab..c9c9d87c0 100644 --- a/etl/tests/integration/replication_test.rs +++ b/etl/tests/integration/replication_test.rs @@ -1,4 +1,5 @@ -use etl::replication::client::{PgReplicationClient, PgReplicationError}; +use etl::error::{ETLError, ErrorKind}; +use etl::replication::client::PgReplicationClient; use futures::StreamExt; use postgres::schema::ColumnSchema; use postgres::tokio::test_utils::{TableModification, id_column_schema}; @@ -120,7 +121,7 @@ async fn test_create_and_delete_slot() { // Verify the slot no longer exists let result = client.get_slot(&slot_name).await; - assert!(matches!(result, Err(PgReplicationError::SlotNotFound(_)))); + assert!(matches!(result, Err(ref err) if err.kind() == ErrorKind::ReplicationSlotNotFound)); } #[tokio::test(flavor = "multi_thread")] @@ -136,7 +137,7 @@ async fn test_delete_nonexistent_slot() { // Attempt to delete a slot that doesn't exist let result = client.delete_slot(&slot_name).await; - assert!(matches!(result, Err(PgReplicationError::SlotNotFound(_)))); + assert!(matches!(result, Err(ref err) if err.kind() == ErrorKind::ReplicationSlotNotFound)); } #[tokio::test(flavor = "multi_thread")] @@ -152,7 +153,7 @@ async fn test_replication_client_doesnt_recreate_slot() { assert!(client.create_slot(&slot_name).await.is_ok()); assert!(matches!( client.create_slot(&slot_name).await, - Err(PgReplicationError::SlotAlreadyExists(_)) + Err(ref err) if err.kind() == ErrorKind::ReplicationSlotAlreadyExists )); } From e695dd6494f7d9657b4dd0e8993c8cf1cd3ff514 Mon Sep 17 00:00:00 2001 From: Riccardo Busetti Date: Fri, 25 Jul 2025 12:01:48 +0200 Subject: [PATCH 03/18] Improve --- etl/src/destination/bigquery.rs | 2 +- etl/src/error.rs | 10 +- etl/src/replication/apply.rs | 288 +++++++++++++----------------- etl/src/replication/client.rs | 103 +++++------ etl/src/replication/slot.rs | 21 +-- etl/src/replication/table_sync.rs | 5 +- etl/src/state/store/base.rs | 2 +- etl/src/state/store/postgres.rs | 21 ++- etl/src/workers/table_sync.rs | 4 +- 9 files changed, 213 insertions(+), 243 deletions(-) diff --git a/etl/src/destination/bigquery.rs b/etl/src/destination/bigquery.rs index e02df92ce..283ef4d23 100644 --- a/etl/src/destination/bigquery.rs +++ b/etl/src/destination/bigquery.rs @@ -194,7 +194,7 @@ impl BigQueryDestination { .await; let table_schema = schema_cache.get_table_schema_ref(table_id).ok_or_else(|| { ETLError::from(( - ErrorKind::SchemaError, + ErrorKind::DestinationSchemaError, "Table schema not found in schema cache", format!("table_id: {}", table_id), )) diff --git a/etl/src/error.rs b/etl/src/error.rs index f4deade4f..8370930e8 100644 --- a/etl/src/error.rs +++ b/etl/src/error.rs @@ -26,8 +26,12 @@ pub enum ErrorKind { AuthenticationFailed, /// Query execution failed QueryFailed, - /// Schema mismatch or validation error - SchemaError, + /// Source schema mismatch or validation error + SourceSchemaError, + /// Destination schema mismatch or validation error + DestinationSchemaError, + /// Missing table schema + MissingTableSchema, /// Data type conversion error ConversionError, /// Configuration error @@ -58,6 +62,8 @@ pub enum ErrorKind { ReplicationSlotNotFound, /// Replication slot already exists ReplicationSlotAlreadyExists, + /// Replication slot could not be created + ReplicationSlotNotCreated, /// Replication slot name is invalid or too long ReplicationSlotInvalid, /// Table synchronization failed diff --git a/etl/src/replication/apply.rs b/etl/src/replication/apply.rs index 6121e7fae..1b0ebd3e0 100644 --- a/etl/src/replication/apply.rs +++ b/etl/src/replication/apply.rs @@ -1,15 +1,3 @@ -use crate::concurrency::shutdown::ShutdownRx; -use crate::conversions::event::{Event, EventType, convert_message_to_event}; -use crate::destination::base::Destination; -use crate::error::{ETLError, ETLResult, ErrorKind, ErrorRepr}; -use crate::pipeline::PipelineId; -use crate::replication::client::PgReplicationClient; -use crate::replication::slot::get_slot_name; -use crate::replication::stream::EventsStream; -use crate::schema::cache::SchemaCache; -use crate::workers::base::WorkerType; - -use crate::concurrency::signal::SignalRx; use config::shared::PipelineConfig; use futures::{FutureExt, StreamExt}; use postgres::schema::TableId; @@ -23,6 +11,19 @@ use tokio::pin; use tokio_postgres::types::PgLsn; use tracing::{debug, info}; +use crate::concurrency::shutdown::ShutdownRx; +use crate::concurrency::signal::SignalRx; +use crate::conversions::event::{Event, EventType, convert_message_to_event}; +use crate::destination::base::Destination; +use crate::error::{ETLError, ETLResult, ErrorKind, ErrorRepr}; +use crate::pipeline::PipelineId; +use crate::replication::client::PgReplicationClient; +use crate::replication::slot::get_slot_name; +use crate::replication::stream::EventsStream; +use crate::schema::cache::SchemaCache; +use crate::workers::base::WorkerType; +use crate::{bail, etl_error}; + /// The amount of milliseconds that pass between one refresh and the other of the system, in case no /// events or shutdown signal are received. const REFRESH_INTERVAL: Duration = Duration::from_millis(1000); @@ -302,6 +303,7 @@ where // sync at the next transaction boundary. if !state.handling_transaction() { debug!("forcefully processing syncing tables"); + let continue_loop = hook.process_syncing_tables(state.next_status_update.flush_lsn, true).await?; if !continue_loop { return Ok(ApplyLoopResult::ApplyStopped); @@ -566,12 +568,12 @@ fn get_commit_lsn(state: &ApplyLoopState, message: &LogicalReplicationMessage) - if let LogicalReplicationMessage::Begin(message) = message { Ok(PgLsn::from(message.final_lsn())) } else { - state.remote_final_lsn.ok_or_else(|| ETLError { - repr: ErrorRepr::WithDescriptionAndDetail( + state.remote_final_lsn.ok_or_else(|| { + etl_error!( ErrorKind::InvalidState, "Invalid transaction", - "A transaction should have started for get_commit_lsn to be performed".to_string(), - ), + "A transaction should have started for get_commit_lsn to be performed" + ) }) } } @@ -582,17 +584,15 @@ async fn handle_begin_message( message: &protocol::BeginBody, ) -> ETLResult { let EventType::Begin = event.event_type() else { - return Err(ETLError { - repr: ErrorRepr::WithDescriptionAndDetail( - ErrorKind::ValidationError, - "Invalid event", - format!( - "An invalid event {:?} was received (expected {:?})", - event, - EventType::Begin - ), - ), - }); + bail!( + ErrorKind::ValidationError, + "Invalid event", + format!( + "An invalid event {:?} was received (expected {:?})", + event, + EventType::Begin + ) + ); }; // We track the final lsn of this transaction, which should be equal to the `commit_lsn` of the @@ -619,31 +619,26 @@ where ETLError: From<::Error>, { let EventType::Commit = event.event_type() else { - return Err(ETLError { - repr: ErrorRepr::WithDescriptionAndDetail( - ErrorKind::ValidationError, - "Invalid event", - format!( - "An invalid event {:?} was received (expected {:?})", - event, - EventType::Commit - ), - ), - }); + bail!( + ErrorKind::ValidationError, + "Invalid event", + format!( + "An invalid event {:?} was received (expected {:?})", + event, + EventType::Commit + ) + ); }; // We take the LSN that belongs to the current transaction, however, if there is no // LSN, it means that a `Begin` message was not received before this `Commit` which means // we are in an inconsistent state. let Some(remote_final_lsn) = state.remote_final_lsn.take() else { - return Err(ETLError { - repr: ErrorRepr::WithDescriptionAndDetail( - ErrorKind::InvalidState, - "Invalid transaction", - "A transaction should have started for handle_commit_message to be performed" - .to_string(), - ), - }); + bail!( + ErrorKind::InvalidState, + "Invalid transaction", + "A transaction should have started for handle_commit_message to be performed" + ); }; // If the commit lsn of the message is different from the remote final lsn, it means that the @@ -651,16 +646,14 @@ where // we want to bail assuming we are in an inconsistent state. let commit_lsn = PgLsn::from(message.commit_lsn()); if commit_lsn != remote_final_lsn { - return Err(ETLError { - repr: ErrorRepr::WithDescriptionAndDetail( - ErrorKind::ValidationError, - "Invalid commit LSN", - format!( - "Incorrect commit LSN {} in COMMIT message (expected {})", - commit_lsn, remote_final_lsn - ), - ), - }); + bail!( + ErrorKind::ValidationError, + "Invalid commit LSN", + format!( + "Incorrect commit LSN {} in COMMIT message (expected {})", + commit_lsn, remote_final_lsn + ) + ); } let end_lsn = PgLsn::from(message.end_lsn()); @@ -708,28 +701,23 @@ where ETLError: From<::Error>, { let Event::Relation(event) = event else { - return Err(ETLError { - repr: ErrorRepr::WithDescriptionAndDetail( - ErrorKind::ValidationError, - "Invalid event", - format!( - "An invalid event {:?} was received (expected {:?})", - event, - EventType::Relation - ), - ), - }); + bail!( + ErrorKind::ValidationError, + "Invalid event", + format!( + "An invalid event {:?} was received (expected {:?})", + event, + EventType::Relation + ) + ); }; let Some(remote_final_lsn) = state.remote_final_lsn else { - return Err(ETLError { - repr: ErrorRepr::WithDescriptionAndDetail( - ErrorKind::InvalidState, - "Invalid transaction", - "A transaction should have started for handle_relation_message to be performed" - .to_string(), - ), - }); + bail!( + ErrorKind::InvalidState, + "Invalid transaction", + "A transaction should have started for handle_relation_message to be performed" + ); }; if !hook @@ -744,16 +732,14 @@ where // TODO: explore how to deal with applying relation messages to the schema (creating it if missing). let schema_cache = schema_cache.lock_inner().await; let Some(existing_table_schema) = schema_cache.get_table_schema_ref(&message.rel_id()) else { - return Err(ETLError { - repr: ErrorRepr::WithDescriptionAndDetail( - ErrorKind::SchemaError, - "Missing table schema", - format!( - "The table schema for table {} was not found in the cache", - message.rel_id() - ), - ), - }); + bail!( + ErrorKind::MissingTableSchema, + "Missing table schema", + format!( + "The table schema for table {} was not found in the cache", + message.rel_id() + ) + ); }; // We compare the table schema from the relation message with the existing schema (if any). @@ -786,28 +772,23 @@ where ETLError: From<::Error>, { let Event::Insert(event) = event else { - return Err(ETLError { - repr: ErrorRepr::WithDescriptionAndDetail( - ErrorKind::ValidationError, - "Invalid event", - format!( - "An invalid event {:?} was received (expected {:?})", - event, - EventType::Insert - ), - ), - }); + bail!( + ErrorKind::ValidationError, + "Invalid event", + format!( + "An invalid event {:?} was received (expected {:?})", + event, + EventType::Insert + ) + ); }; let Some(remote_final_lsn) = state.remote_final_lsn else { - return Err(ETLError { - repr: ErrorRepr::WithDescriptionAndDetail( - ErrorKind::InvalidState, - "Invalid transaction", - "A transaction should have started for handle_insert_message to be performed" - .to_string(), - ), - }); + bail!( + ErrorKind::InvalidState, + "Invalid transaction", + "A transaction should have started for handle_insert_message to be performed" + ); }; if !hook @@ -836,28 +817,23 @@ where ETLError: From<::Error>, { let Event::Update(event) = event else { - return Err(ETLError { - repr: ErrorRepr::WithDescriptionAndDetail( - ErrorKind::ValidationError, - "Invalid event", - format!( - "An invalid event {:?} was received (expected {:?})", - event, - EventType::Update - ), - ), - }); + bail!( + ErrorKind::ValidationError, + "Invalid event", + format!( + "An invalid event {:?} was received (expected {:?})", + event, + EventType::Update + ) + ); }; let Some(remote_final_lsn) = state.remote_final_lsn else { - return Err(ETLError { - repr: ErrorRepr::WithDescriptionAndDetail( - ErrorKind::InvalidState, - "Invalid transaction", - "A transaction should have started for handle_update_message to be performed" - .to_string(), - ), - }); + bail!( + ErrorKind::InvalidState, + "Invalid transaction", + "A transaction should have started for handle_update_message to be performed" + ); }; if !hook @@ -886,28 +862,23 @@ where ETLError: From<::Error>, { let Event::Delete(event) = event else { - return Err(ETLError { - repr: ErrorRepr::WithDescriptionAndDetail( - ErrorKind::ValidationError, - "Invalid event", - format!( - "An invalid event {:?} was received (expected {:?})", - event, - EventType::Delete - ), - ), - }); + bail!( + ErrorKind::ValidationError, + "Invalid event", + format!( + "An invalid event {:?} was received (expected {:?})", + event, + EventType::Delete + ) + ); }; let Some(remote_final_lsn) = state.remote_final_lsn else { - return Err(ETLError { - repr: ErrorRepr::WithDescriptionAndDetail( - ErrorKind::InvalidState, - "Invalid transaction", - "A transaction should have started for handle_delete_message to be performed" - .to_string(), - ), - }); + bail!( + ErrorKind::InvalidState, + "Invalid transaction", + "A transaction should have started for handle_delete_message to be performed" + ); }; if !hook @@ -936,28 +907,23 @@ where ETLError: From<::Error>, { let Event::Truncate(mut event) = event else { - return Err(ETLError { - repr: ErrorRepr::WithDescriptionAndDetail( - ErrorKind::ValidationError, - "Invalid event", - format!( - "An invalid event {:?} was received (expected {:?})", - event, - EventType::Truncate - ), - ), - }); + bail!( + ErrorKind::ValidationError, + "Invalid event", + format!( + "An invalid event {:?} was received (expected {:?})", + event, + EventType::Truncate + ) + ); }; let Some(remote_final_lsn) = state.remote_final_lsn else { - return Err(ETLError { - repr: ErrorRepr::WithDescriptionAndDetail( - ErrorKind::InvalidState, - "Invalid transaction", - "A transaction should have started for handle_truncate_message to be performed" - .to_string(), - ), - }); + bail!( + ErrorKind::InvalidState, + "Invalid transaction", + "A transaction should have started for handle_truncate_message to be performed" + ); }; let mut rel_ids = Vec::with_capacity(message.rel_ids().len()); diff --git a/etl/src/replication/client.rs b/etl/src/replication/client.rs index 95bb00ab4..be82aa6d9 100644 --- a/etl/src/replication/client.rs +++ b/etl/src/replication/client.rs @@ -84,7 +84,7 @@ impl PgReplicationSlotTransaction { /// /// The transaction is started with a repeatable read isolation level and uses the /// snapshot associated with the provided slot. - async fn new(client: PgReplicationClient) -> PgReplicationResult { + async fn new(client: PgReplicationClient) -> ETLResult { client.begin_tx().await?; Ok(Self { client }) @@ -98,7 +98,7 @@ impl PgReplicationSlotTransaction { &self, table_ids: &[TableId], publication_name: Option<&str>, - ) -> PgReplicationResult> { + ) -> ETLResult> { self.client .get_table_schemas(table_ids, publication_name) .await @@ -112,7 +112,7 @@ impl PgReplicationSlotTransaction { &self, table_id: TableId, publication: Option<&str>, - ) -> PgReplicationResult { + ) -> ETLResult { self.client.get_table_schema(table_id, publication).await } @@ -123,19 +123,19 @@ impl PgReplicationSlotTransaction { &self, table_id: TableId, column_schemas: &[ColumnSchema], - ) -> PgReplicationResult { + ) -> ETLResult { self.client .get_table_copy_stream(table_id, column_schemas) .await } /// Commits the current transaction. - pub async fn commit(self) -> PgReplicationResult<()> { + pub async fn commit(self) -> ETLResult<()> { self.client.commit_tx().await } /// Rolls back the current transaction. - pub async fn rollback(self) -> PgReplicationResult<()> { + pub async fn rollback(self) -> ETLResult<()> { self.client.rollback_tx().await } } @@ -149,15 +149,12 @@ pub struct PgReplicationClient { client: Arc, } -/// Update the type alias to use the new error type -pub type PgReplicationResult = ETLResult; - impl PgReplicationClient { /// Establishes a connection to PostgreSQL. The connection uses TLS if configured in the - /// passed [`PgConnectionConfig`]. + /// supplied [`PgConnectionConfig`]. /// /// The connection is configured for logical replication mode - pub async fn connect(pg_connection_config: PgConnectionConfig) -> PgReplicationResult { + pub async fn connect(pg_connection_config: PgConnectionConfig) -> ETLResult { match pg_connection_config.tls.enabled { true => PgReplicationClient::connect_tls(pg_connection_config).await, false => PgReplicationClient::connect_no_tls(pg_connection_config).await, @@ -167,7 +164,7 @@ impl PgReplicationClient { /// Establishes a connection to PostgreSQL without TLS encryption. /// /// The connection is configured for logical replication mode. - async fn connect_no_tls(pg_connection_config: PgConnectionConfig) -> PgReplicationResult { + async fn connect_no_tls(pg_connection_config: PgConnectionConfig) -> ETLResult { let mut config: Config = pg_connection_config.clone().with_db(); config.replication_mode(ReplicationMode::Logical); @@ -184,7 +181,7 @@ impl PgReplicationClient { /// Establishes a TLS-encrypted connection to PostgreSQL. /// /// The connection is configured for logical replication mode - async fn connect_tls(pg_connection_config: PgConnectionConfig) -> PgReplicationResult { + async fn connect_tls(pg_connection_config: PgConnectionConfig) -> ETLResult { let mut config: Config = pg_connection_config.clone().with_db(); config.replication_mode(ReplicationMode::Logical); @@ -217,7 +214,7 @@ impl PgReplicationClient { pub async fn create_slot_with_transaction( &self, slot_name: &str, - ) -> PgReplicationResult<(PgReplicationSlotTransaction, CreateSlotResult)> { + ) -> ETLResult<(PgReplicationSlotTransaction, CreateSlotResult)> { // TODO: check if we want to consume the client and return it on commit to avoid any other // operations on a connection that has started a transaction. let transaction = PgReplicationSlotTransaction::new(self.clone()).await?; @@ -227,14 +224,14 @@ impl PgReplicationClient { } /// Creates a new logical replication slot with the specified name and no snapshot. - pub async fn create_slot(&self, slot_name: &str) -> PgReplicationResult { + pub async fn create_slot(&self, slot_name: &str) -> ETLResult { self.create_slot_internal(slot_name, false).await } /// Gets the slot by `slot_name`. /// /// Returns an error in case of failure or missing slot. - pub async fn get_slot(&self, slot_name: &str) -> PgReplicationResult { + pub async fn get_slot(&self, slot_name: &str) -> ETLResult { let query = format!( r#"select confirmed_flush_lsn from pg_replication_slots where slot_name = {};"#, quote_literal(slot_name) @@ -257,11 +254,11 @@ impl PgReplicationClient { } } - Err(etl_error!( + bail!( ErrorKind::ReplicationSlotNotFound, "Replication slot not found", format!("Replication slot '{}' not found in database", slot_name) - )) + ); } /// Gets an existing replication slot or creates a new one if it doesn't exist. @@ -273,10 +270,7 @@ impl PgReplicationClient { /// - A boolean indicating whether the slot was created (true) or already existed (false) /// - The slot result containing either the confirmed_flush_lsn (for existing slots) /// or the consistent_point (for newly created slots) - pub async fn get_or_create_slot( - &self, - slot_name: &str, - ) -> PgReplicationResult { + pub async fn get_or_create_slot(&self, slot_name: &str) -> ETLResult { match self.get_slot(slot_name).await { Ok(slot) => { info!("using existing replication slot '{}'", slot_name); @@ -297,7 +291,7 @@ impl PgReplicationClient { /// Deletes a replication slot with the specified name. /// /// Returns an error if the slot doesn't exist or if there are any issues with the deletion. - pub async fn delete_slot(&self, slot_name: &str) -> PgReplicationResult<()> { + pub async fn delete_slot(&self, slot_name: &str) -> ETLResult<()> { info!("deleting replication slot '{}'", slot_name); // Do not convert the query or the options to lowercase, see comment in `create_slot_internal`. let query = format!( @@ -318,22 +312,27 @@ impl PgReplicationClient { "attempted to delete non-existent replication slot '{}'", slot_name ); + bail!( ErrorKind::ReplicationSlotNotFound, "Replication slot not found", - format!("Replication slot '{}' not found in database", slot_name) + format!( + "Replication slot '{}' not found in database while attempting its deletion", + slot_name + ) ); } } error!("failed to delete replication slot '{}': {}", slot_name, err); + Err(err.into()) } } } /// Checks if a publication with the given name exists. - pub async fn publication_exists(&self, publication: &str) -> PgReplicationResult { + pub async fn publication_exists(&self, publication: &str) -> ETLResult { let publication_exists_query = format!( "select 1 as exists from pg_publication where pubname = {};", quote_literal(publication) @@ -343,6 +342,7 @@ impl PgReplicationClient { return Ok(true); } } + Ok(false) } @@ -350,7 +350,7 @@ impl PgReplicationClient { pub async fn get_publication_table_names( &self, publication_name: &str, - ) -> PgReplicationResult> { + ) -> ETLResult> { let publication_query = format!( "select schemaname, tablename from pg_publication_tables where pubname = {};", quote_literal(publication_name) @@ -377,7 +377,7 @@ impl PgReplicationClient { pub async fn get_publication_table_ids( &self, publication_name: &str, - ) -> PgReplicationResult> { + ) -> ETLResult> { let publication_query = format!( "select c.oid from pg_publication_tables pt join pg_class c on c.relname = pt.tablename @@ -386,16 +386,16 @@ impl PgReplicationClient { quote_literal(publication_name) ); - let mut table_oids = vec![]; + let mut table_ids = vec![]; for msg in self.client.simple_query(&publication_query).await? { if let SimpleQueryMessage::Row(row) = msg { - let oid = Self::get_row_value::(&row, "oid", "pg_class").await?; - - table_oids.push(oid); + // For the sake of simplicity, we refer to the table oid as table id. + let table_id = Self::get_row_value::(&row, "oid", "pg_class").await?; + table_ids.push(table_id); } } - Ok(table_oids) + Ok(table_ids) } /// Starts a logical replication stream from the specified publication and slot. @@ -406,7 +406,7 @@ impl PgReplicationClient { publication_name: &str, slot_name: &str, start_lsn: PgLsn, - ) -> PgReplicationResult { + ) -> ETLResult { info!( "starting logical replication from publication '{}' with slot named '{}' at lsn {}", publication_name, slot_name, start_lsn @@ -426,7 +426,6 @@ impl PgReplicationClient { ); let copy_stream = self.client.copy_both_simple::(&query).await?; - let stream = LogicalReplicationStream::new(copy_stream); Ok(stream) @@ -436,7 +435,7 @@ impl PgReplicationClient { /// /// The transaction doesn't make any assumptions about the snapshot in use, since this is a /// concern of the statements issued within the transaction. - async fn begin_tx(&self) -> PgReplicationResult<()> { + async fn begin_tx(&self) -> ETLResult<()> { self.client .simple_query("begin read only isolation level repeatable read;") .await?; @@ -445,14 +444,16 @@ impl PgReplicationClient { } /// Commits the current transaction. - async fn commit_tx(&self) -> PgReplicationResult<()> { + async fn commit_tx(&self) -> ETLResult<()> { self.client.simple_query("commit;").await?; + Ok(()) } /// Rolls back the current transaction. - async fn rollback_tx(&self) -> PgReplicationResult<()> { + async fn rollback_tx(&self) -> ETLResult<()> { self.client.simple_query("rollback;").await?; + Ok(()) } @@ -463,7 +464,7 @@ impl PgReplicationClient { &self, slot_name: &str, use_snapshot: bool, - ) -> PgReplicationResult { + ) -> ETLResult { // Do not convert the query or the options to lowercase, since the lexer for // replication commands (repl_scanner.l) in Postgres code expects the commands // in uppercase. This probably should be fixed in upstream, but for now we will @@ -513,7 +514,7 @@ impl PgReplicationClient { } Err(etl_error!( - ErrorKind::ReplicationSlotAlreadyExists, + ErrorKind::ReplicationSlotNotCreated, "Failed to create replication slot" )) } @@ -525,7 +526,7 @@ impl PgReplicationClient { &self, table_ids: &[TableId], publication_name: Option<&str>, - ) -> PgReplicationResult> { + ) -> ETLResult> { let mut table_schemas = HashMap::new(); // TODO: consider if we want to fail when at least one table was missing or not. @@ -533,7 +534,7 @@ impl PgReplicationClient { let table_schema = self.get_table_schema(*table_id, publication_name).await?; // TODO: this warning and skipping should not happen in this method, - // but rather higher in the stack. + // but rather higher in the stack. if !table_schema.has_primary_keys() { warn!( "table {} with id {} will not be copied because it has no primary key", @@ -556,7 +557,7 @@ impl PgReplicationClient { &self, table_id: TableId, publication: Option<&str>, - ) -> PgReplicationResult { + ) -> ETLResult { let table_name = self.get_table_name(table_id).await?; let column_schemas = self.get_column_schemas(table_id, publication).await?; @@ -570,7 +571,7 @@ impl PgReplicationClient { /// Loads the table name and schema information for a given table OID. /// /// Returns a `TableName` containing both the schema and table name. - async fn get_table_name(&self, table_id: TableId) -> PgReplicationResult { + async fn get_table_name(&self, table_id: TableId) -> ETLResult { let table_info_query = format!( "select n.nspname as schema_name, c.relname as table_name from pg_class c @@ -592,11 +593,11 @@ impl PgReplicationClient { } } - Err(etl_error!( - ErrorKind::SchemaError, + bail!( + ErrorKind::SourceSchemaError, "Table not found", - format!("Table not found in database (oid: {})", table_id) - )) + format!("Table not found in database (table id: {})", table_id) + ); } /// Retrieves schema information for all columns in a table. @@ -607,7 +608,7 @@ impl PgReplicationClient { &self, table_id: TableId, publication: Option<&str>, - ) -> PgReplicationResult> { + ) -> ETLResult> { let (pub_cte, pub_pred) = if let Some(publication) = publication { ( format!( @@ -687,7 +688,7 @@ impl PgReplicationClient { &self, table_id: TableId, column_schemas: &[ColumnSchema], - ) -> PgReplicationResult { + ) -> ETLResult { let column_list = column_schemas .iter() .map(|col| quote_identifier(&col.name)) @@ -715,12 +716,12 @@ impl PgReplicationClient { row: &SimpleQueryRow, column_name: &str, table_name: &str, - ) -> PgReplicationResult + ) -> ETLResult where T::Err: fmt::Debug, { let value = row.try_get(column_name)?.ok_or(etl_error!( - ErrorKind::SchemaError, + ErrorKind::SourceSchemaError, "Column not found", format!( "Column '{}' not found in table '{}'", diff --git a/etl/src/replication/slot.rs b/etl/src/replication/slot.rs index 2ce3e7e86..7c2d62e87 100644 --- a/etl/src/replication/slot.rs +++ b/etl/src/replication/slot.rs @@ -1,5 +1,7 @@ use thiserror::Error; +use crate::bail; +use crate::error::ETLResult; use crate::pipeline::PipelineId; use crate::workers::base::WorkerType; @@ -10,20 +12,8 @@ const MAX_SLOT_NAME_LENGTH: usize = 63; const APPLY_WORKER_PREFIX: &str = "supabase_etl_apply"; const TABLE_SYNC_PREFIX: &str = "supabase_etl_table_sync"; -/// Error types that can occur when working with replication slots -#[derive(Debug, Error)] -pub enum SlotError { - #[error( - "Replication slot name exceeds maximum length of {MAX_SLOT_NAME_LENGTH} characters: name must be shorter" - )] - NameTooLong, -} - /// Generates a replication slot name. -pub fn get_slot_name( - pipeline_id: PipelineId, - worker_type: WorkerType, -) -> Result { +pub fn get_slot_name(pipeline_id: PipelineId, worker_type: WorkerType) -> ETLResult { let slot_name = match worker_type { WorkerType::Apply => { format!("{APPLY_WORKER_PREFIX}_{pipeline_id}") @@ -34,7 +24,10 @@ pub fn get_slot_name( }; if slot_name.len() > MAX_SLOT_NAME_LENGTH { - return Err(SlotError::NameTooLong); + bail!( + ErrorKind::ValidationError, + "Invalid slot name length: {slot_name}" + ); } Ok(slot_name) diff --git a/etl/src/replication/table_sync.rs b/etl/src/replication/table_sync.rs index 81482e319..9561460ea 100644 --- a/etl/src/replication/table_sync.rs +++ b/etl/src/replication/table_sync.rs @@ -18,7 +18,7 @@ use crate::error::{ETLError, ETLResult, ErrorKind}; use crate::failpoints::START_TABLE_SYNC_AFTER_DATA_SYNC; use crate::pipeline::PipelineId; use crate::replication::client::PgReplicationClient; -use crate::replication::slot::{SlotError, get_slot_name}; +use crate::replication::slot::get_slot_name; use crate::replication::stream::{TableCopyStream, TableCopyStreamError}; use crate::schema::cache::SchemaCache; use crate::state::store::base::StateStore; @@ -172,8 +172,9 @@ where state_store .update_table_replication_state(table_id, TableReplicationPhase::Skipped) .await?; + bail!( - ErrorKind::SchemaError, + ErrorKind::SourceSchemaError, "Missing primary key", format!("table {} has no primary key", table_schema.name) ); diff --git a/etl/src/state/store/base.rs b/etl/src/state/store/base.rs index c43bd7a74..b775aee59 100644 --- a/etl/src/state/store/base.rs +++ b/etl/src/state/store/base.rs @@ -23,7 +23,7 @@ pub trait StateStore { ) -> impl Future>> + Send; /// Loads the table replication states from the persistent state into the cache. - /// This should called once at program start to load the state into the cache + /// This should be called once at program start to load the state into the cache /// and then use only the `get_X` methods to access the state. Updating the state /// by calling the `update_table_replication_state` updates in both the cache and /// the persistent store, so no need to ever load the state again. diff --git a/etl/src/state/store/postgres.rs b/etl/src/state/store/postgres.rs index b9a95b317..72c80ef7c 100644 --- a/etl/src/state/store/postgres.rs +++ b/etl/src/state/store/postgres.rs @@ -1,7 +1,5 @@ use std::{collections::HashMap, sync::Arc}; -use crate::error::{ETLError, ETLResult, ErrorKind}; -use crate::{bail, etl_error}; use config::shared::PgConnectionConfig; use postgres::replication::{ TableReplicationState, TableReplicationStateRow, connect_to_source_database, @@ -13,10 +11,11 @@ use tokio::sync::Mutex; use tokio_postgres::types::PgLsn; use tracing::{debug, info}; -use crate::{ - pipeline::PipelineId, - state::{store::base::StateStore, table::TableReplicationPhase}, -}; +use crate::error::{ETLError, ETLResult, ErrorKind}; +use crate::pipeline::PipelineId; +use crate::state::store::base::StateStore; +use crate::state::table::TableReplicationPhase; +use crate::{bail, etl_error}; const NUM_POOL_CONNECTIONS: u32 = 1; @@ -59,11 +58,12 @@ pub struct PostgresStateStore { } impl PostgresStateStore { - pub fn new(pipeline_id: PipelineId, source_config: PgConnectionConfig) -> PostgresStateStore { + pub fn new(pipeline_id: PipelineId, source_config: PgConnectionConfig) -> Self { let inner = Inner { table_states: HashMap::new(), }; - PostgresStateStore { + + Self { pipeline_id, source_config, inner: Arc::new(Mutex::new(inner)), @@ -130,7 +130,7 @@ impl PostgresStateStore { None => bail!( ErrorKind::ValidationError, "Missing LSN", - "Lsn can't be missing from the state store if state is SyncDone" + "Lsn can't be missing from the state store if state is 'SyncDone'" ), }, TableReplicationState::Ready => TableReplicationPhase::Ready, @@ -157,6 +157,7 @@ impl StateStore for PostgresStateStore { async fn load_table_replication_states(&self) -> ETLResult { debug!("loading table replication states from postgres state store"); + let pool = self.connect_to_source().await?; let replication_state_rows = self .get_all_replication_state_rows(&pool, self.pipeline_id) @@ -187,8 +188,10 @@ impl StateStore for PostgresStateStore { let (table_state, sync_done_lsn) = state.try_into()?; self.update_replication_state(self.pipeline_id, table_id, table_state, sync_done_lsn) .await?; + let mut inner = self.inner.lock().await; inner.table_states.insert(table_id, state); + Ok(()) } } diff --git a/etl/src/workers/table_sync.rs b/etl/src/workers/table_sync.rs index 1319c66f6..18384a623 100644 --- a/etl/src/workers/table_sync.rs +++ b/etl/src/workers/table_sync.rs @@ -1,5 +1,3 @@ -use crate::error::{ETLError, ETLResult, ErrorKind}; -use crate::{bail, etl_error}; use config::shared::PipelineConfig; use postgres::schema::TableId; use std::sync::Arc; @@ -9,10 +7,12 @@ use tokio::task::JoinHandle; use tokio_postgres::types::PgLsn; use tracing::{Instrument, debug, error, info, warn}; +use crate::bail; use crate::concurrency::future::ReactiveFuture; use crate::concurrency::shutdown::{ShutdownResult, ShutdownRx}; use crate::concurrency::signal::SignalTx; use crate::destination::base::Destination; +use crate::error::{ETLError, ETLResult, ErrorKind}; use crate::pipeline::PipelineId; use crate::replication::apply::{ApplyLoopHook, start_apply_loop}; use crate::replication::client::PgReplicationClient; From cdbbf169190050369291c70d1fa373c3d336b5ae Mon Sep 17 00:00:00 2001 From: Riccardo Busetti Date: Fri, 25 Jul 2025 12:32:12 +0200 Subject: [PATCH 04/18] Improve --- etl/src/conversions/bool.rs | 16 ++- etl/src/conversions/event.rs | 62 ++------- etl/src/conversions/hex.rs | 28 ++-- etl/src/conversions/table_row.rs | 39 ++---- etl/src/conversions/text.rs | 69 ++-------- etl/src/destination/base.rs | 29 +--- etl/src/destination/bigquery.rs | 13 +- etl/src/destination/memory.rs | 11 +- etl/src/error.rs | 214 ++---------------------------- etl/src/replication/table_sync.rs | 2 +- 10 files changed, 94 insertions(+), 389 deletions(-) diff --git a/etl/src/conversions/bool.rs b/etl/src/conversions/bool.rs index 040d8c4a0..24dcf21db 100644 --- a/etl/src/conversions/bool.rs +++ b/etl/src/conversions/bool.rs @@ -1,17 +1,19 @@ +use std::io::ErrorKind; use thiserror::Error; -#[derive(Debug, Error)] -pub enum ParseBoolError { - #[error("invalid input value: {0}")] - InvalidInput(String), -} +use crate::bail; +use crate::error::ETLResult; -pub fn parse_bool(s: &str) -> Result { +pub fn parse_bool(s: &str) -> ETLResult { if s == "t" { Ok(true) } else if s == "f" { Ok(false) } else { - Err(ParseBoolError::InvalidInput(s.to_string())) + bail!( + ErrorKind::InvalidData, + "Invalid boolean value", + format!("Boolean value must be 't' or 'f' (received: {s})") + ); } } diff --git a/etl/src/conversions/event.rs b/etl/src/conversions/event.rs index e41f9037d..0b1de4758 100644 --- a/etl/src/conversions/event.rs +++ b/etl/src/conversions/event.rs @@ -1,45 +1,16 @@ -use crate::conversions::Cell; -use crate::conversions::table_row::TableRow; -use crate::conversions::text::{FromTextError, TextFormatConverter}; -use crate::schema::cache::SchemaCache; use core::str; use postgres::schema::{ColumnSchema, TableId, TableName, TableSchema}; use postgres::types::convert_type_oid_to_type; use postgres_replication::protocol; use postgres_replication::protocol::LogicalReplicationMessage; -use std::{fmt, io, str::Utf8Error}; -use thiserror::Error; +use std::fmt; use tokio_postgres::types::PgLsn; -#[derive(Debug, Error)] -pub enum EventConversionError { - #[error("An unknown replication message type was encountered")] - UnknownReplicationMessage, - - #[error("Binary format is not supported for data conversion")] - BinaryFormatNotSupported, - - #[error("The tuple data was not found for column {0} at index {0}")] - TupleDataNotFound(String, usize), - - #[error("Missing tuple data in delete body")] - MissingTupleInDeleteBody, - - #[error("Table schema not found for table id {0}")] - MissingSchema(TableId), - - #[error("Error converting from bytes: {0}")] - FromBytes(#[from] FromTextError), - - #[error("Invalid string value encountered: {0}")] - InvalidStr(#[from] Utf8Error), - - #[error("IO error encountered: {0}")] - Io(#[from] io::Error), - - #[error("An error occurred in the state store: {0}")] - StateStore(#[from] crate::error::ETLError), -} +use crate::conversions::Cell; +use crate::conversions::table_row::TableRow; +use crate::conversions::text::TextFormatConverter; +use crate::error::ETLResult; +use crate::schema::cache::SchemaCache; #[derive(Debug, Clone, PartialEq)] pub struct BeginEvent { @@ -101,7 +72,7 @@ impl RelationEvent { start_lsn: PgLsn, commit_lsn: PgLsn, relation_body: &protocol::RelationBody, - ) -> Result { + ) -> ETLResult { let table_name = TableName::new( relation_body.namespace()?.to_string(), relation_body.name()?.to_string(), @@ -120,9 +91,7 @@ impl RelationEvent { }) } - fn build_column_schema( - column: &protocol::Column, - ) -> Result { + fn build_column_schema(column: &protocol::Column) -> ETLResult { Ok(ColumnSchema::new( column.name()?.to_string(), convert_type_oid_to_type(column.type_id() as u32), @@ -263,10 +232,7 @@ impl From for EventType { } } -async fn get_table_schema( - schema_cache: &SchemaCache, - table_id: TableId, -) -> Result { +async fn get_table_schema(schema_cache: &SchemaCache, table_id: TableId) -> ETLResult { schema_cache .get_table_schema(&table_id) .await @@ -276,7 +242,7 @@ async fn get_table_schema( fn convert_tuple_to_row( column_schemas: &[ColumnSchema], tuple_data: &[protocol::TupleData], -) -> Result { +) -> ETLResult { let mut values = Vec::with_capacity(column_schemas.len()); for (i, column_schema) in column_schemas.iter().enumerate() { @@ -316,7 +282,7 @@ async fn convert_insert_to_event( start_lsn: PgLsn, commit_lsn: PgLsn, insert_body: &protocol::InsertBody, -) -> Result { +) -> ETLResult { let table_id = insert_body.rel_id(); let table_schema = get_table_schema(schema_cache, table_id).await?; @@ -338,7 +304,7 @@ async fn convert_update_to_event( start_lsn: PgLsn, commit_lsn: PgLsn, update_body: &protocol::UpdateBody, -) -> Result { +) -> ETLResult { let table_id = update_body.rel_id(); let table_schema = get_table_schema(schema_cache, table_id).await?; @@ -374,7 +340,7 @@ async fn convert_delete_to_event( start_lsn: PgLsn, commit_lsn: PgLsn, delete_body: &protocol::DeleteBody, -) -> Result { +) -> ETLResult { let table_id = delete_body.rel_id(); let table_schema = get_table_schema(schema_cache, table_id).await?; @@ -404,7 +370,7 @@ pub async fn convert_message_to_event( start_lsn: PgLsn, commit_lsn: PgLsn, message: &LogicalReplicationMessage, -) -> Result { +) -> ETLResult { match message { LogicalReplicationMessage::Begin(begin_body) => Ok(Event::Begin( BeginEvent::from_protocol(start_lsn, commit_lsn, begin_body), diff --git a/etl/src/conversions/hex.rs b/etl/src/conversions/hex.rs index 153858814..234078423 100644 --- a/etl/src/conversions/hex.rs +++ b/etl/src/conversions/hex.rs @@ -1,29 +1,27 @@ use std::num::ParseIntError; +use crate::bail; +use crate::error::ETLResult; use thiserror::Error; -#[derive(Debug, Error)] -pub enum ByteaHexParseError { - #[error("missing prefix '\\x'")] - InvalidPrefix, - - #[error("invalid byte")] - OddNumerOfDigits, - - #[error("parse int result: {0}")] - ParseInt(#[from] ParseIntError), -} - -pub fn from_bytea_hex(s: &str) -> Result, ByteaHexParseError> { +pub fn from_bytea_hex(s: &str) -> ETLResult> { if s.len() < 2 || &s[..2] != "\\x" { - return Err(ByteaHexParseError::InvalidPrefix); + bail!( + ErrorKind::ConversionError, + "Could not convert from bytea hex string to byte array", + "The prefix '\\x' is missing" + ); } let mut result = Vec::with_capacity((s.len() - 2) / 2); let s = &s[2..]; if s.len() % 2 != 0 { - return Err(ByteaHexParseError::OddNumerOfDigits); + bail!( + ErrorKind::ConversionError, + "Could not convert from bytea hex string to byte array", + "The number of digits is odd" + ); } for i in (0..s.len()).step_by(2) { diff --git a/etl/src/conversions/table_row.rs b/etl/src/conversions/table_row.rs index d5a0ec8df..9e0692046 100644 --- a/etl/src/conversions/table_row.rs +++ b/etl/src/conversions/table_row.rs @@ -1,3 +1,7 @@ +use super::Cell; +use crate::bail; +use crate::conversions::text::TextFormatConverter; +use crate::error::ETLResult; use core::str; use postgres::schema::ColumnSchema; use std::str::Utf8Error; @@ -5,10 +9,6 @@ use thiserror::Error; use tokio_postgres::types::Type; use tracing::error; -use crate::conversions::text::TextFormatConverter; - -use super::{Cell, text::FromTextError}; - #[derive(Debug, Clone, PartialEq)] pub struct TableRow { pub values: Vec, @@ -64,32 +64,11 @@ impl prost::Message for TableRow { } } -#[derive(Debug, Error)] -pub enum TableRowConversionError { - #[error("unsupported type {0}")] - UnsupportedType(Type), - - #[error("invalid string: {0}")] - InvalidString(#[from] Utf8Error), - - #[error("mismatch in num of columns in schema and row")] - NumColsMismatch, - - #[error("unterminated row")] - UnterminatedRow, - - #[error("invalid value: {0}")] - InvalidValue(#[from] FromTextError), -} - pub struct TableRowConverter; impl TableRowConverter { // parses text produced by this code in Postgres: https://github.com/postgres/postgres/blob/263a3f5f7f508167dbeafc2aefd5835b41d77481/src/backend/commands/copyto.c#L988-L1134 - pub fn try_from( - row: &[u8], - column_schemas: &[ColumnSchema], - ) -> Result { + pub fn try_from(row: &[u8], column_schemas: &[ColumnSchema]) -> ETLResult { let mut values = Vec::with_capacity(column_schemas.len()); let row_str = str::from_utf8(row)?; @@ -139,9 +118,10 @@ impl TableRowConverter { }, None => { if !row_terminated { - return Err(TableRowConversionError::UnterminatedRow); + bail!(ErrorKind::ConversionError, "The row is not terminated"); } done = true; + break; } } @@ -149,7 +129,10 @@ impl TableRowConverter { if !done { let Some(column_schema) = column_schemas_iter.next() else { - return Err(TableRowConversionError::NumColsMismatch); + bail!( + ErrorKind::ConversionError, + "The number of columns in the schema and row is mismatched" + ); }; let value = if val_str == "\\N" { diff --git a/etl/src/conversions/text.rs b/etl/src/conversions/text.rs index 0b43519ef..1f91d13c7 100644 --- a/etl/src/conversions/text.rs +++ b/etl/src/conversions/text.rs @@ -1,60 +1,14 @@ -use core::str; -use std::num::{ParseFloatError, ParseIntError}; - -use bigdecimal::ParseBigDecimalError; +use super::{ArrayCell, Cell, numeric::PgNumeric}; +use crate::bail; +use crate::conversions::{bool::parse_bool, hex}; +use crate::error::ETLResult; use chrono::{DateTime, FixedOffset, NaiveDate, NaiveDateTime, NaiveTime, Utc}; -use thiserror::Error; +use core::str; use tokio_postgres::types::Type; use uuid::Uuid; -use crate::conversions::{bool::parse_bool, hex}; - -use super::{ArrayCell, Cell, bool::ParseBoolError, hex::ByteaHexParseError, numeric::PgNumeric}; - -#[derive(Debug, Error)] -pub enum FromTextError { - #[error("invalid bool value")] - InvalidBool(#[from] ParseBoolError), - - #[error("invalid int value")] - InvalidInt(#[from] ParseIntError), - - #[error("invalid float value")] - InvalidFloat(#[from] ParseFloatError), - - #[error("invalid numeric: {0}")] - InvalidNumeric(#[from] ParseBigDecimalError), - - #[error("invalid bytea: {0}")] - InvalidBytea(#[from] ByteaHexParseError), - - #[error("invalid uuid: {0}")] - InvalidUuid(#[from] uuid::Error), - - #[error("invalid json: {0}")] - InvalidJson(#[from] serde_json::Error), - - #[error("invalid timestamp: {0} ")] - InvalidTimestamp(#[from] chrono::ParseError), - - #[error("invalid array: {0}")] - InvalidArray(#[from] ArrayParseError), - - #[error("row get error: {0:?}")] - RowGetError(#[from] Box), -} - pub struct TextFormatConverter; -#[derive(Debug, Error)] -pub enum ArrayParseError { - #[error("input too short")] - InputTooShort, - - #[error("missing braces")] - MissingBraces, -} - impl TextFormatConverter { pub fn default_value(typ: &Type) -> Cell { match *typ { @@ -108,7 +62,7 @@ impl TextFormatConverter { } } - pub fn try_from_str(typ: &Type, str: &str) -> Result { + pub fn try_from_str(typ: &Type, str: &str) -> ETLResult { match *typ { Type::BOOL => Ok(Cell::Bool(parse_bool(str)?)), Type::BOOL_ARRAY => TextFormatConverter::parse_array( @@ -266,17 +220,20 @@ impl TextFormatConverter { } } - fn parse_array(str: &str, mut parse: P, m: M) -> Result + fn parse_array(str: &str, mut parse: P, m: M) -> ETLResult where - P: FnMut(&str) -> Result, FromTextError>, + P: FnMut(&str) -> ETLResult>, M: FnOnce(Vec>) -> ArrayCell, { if str.len() < 2 { - return Err(ArrayParseError::InputTooShort.into()); + bail!(ErrorKind::ConversionError, "The array input is too short"); } if !str.starts_with('{') || !str.ends_with('}') { - return Err(ArrayParseError::MissingBraces.into()); + bail!( + ErrorKind::ConversionError, + "The array input is missing braces" + ); } let mut res = vec![]; diff --git a/etl/src/destination/base.rs b/etl/src/destination/base.rs index 1adda4fa9..53d0cc89b 100644 --- a/etl/src/destination/base.rs +++ b/etl/src/destination/base.rs @@ -1,25 +1,13 @@ use postgres::schema::{TableId, TableSchema}; use std::future::Future; -use thiserror::Error; use crate::conversions::event::Event; use crate::conversions::table_row::TableRow; -#[cfg(feature = "bigquery")] -use crate::destination::bigquery::BigQueryDestinationError; +use crate::error::ETLResult; use crate::schema::cache::SchemaCache; -#[derive(Debug, Error)] -pub enum DestinationError { - #[cfg(feature = "bigquery")] - #[error(transparent)] - BigQuery(#[from] BigQueryDestinationError), -} - pub trait Destination { - fn inject( - &self, - _schema_cache: SchemaCache, - ) -> impl Future> + Send { + fn inject(&self, _schema_cache: SchemaCache) -> impl Future> + Send { // By default, the injection code is a noop, since not all destinations need dependencies // to be injected. async move { Ok(()) } @@ -28,20 +16,15 @@ pub trait Destination { fn write_table_schema( &self, table_schema: TableSchema, - ) -> impl Future> + Send; + ) -> impl Future> + Send; - fn load_table_schemas( - &self, - ) -> impl Future, DestinationError>> + Send; + fn load_table_schemas(&self) -> impl Future>> + Send; fn write_table_rows( &self, table_id: TableId, table_rows: Vec, - ) -> impl Future> + Send; + ) -> impl Future> + Send; - fn write_events( - &self, - events: Vec, - ) -> impl Future> + Send; + fn write_events(&self, events: Vec) -> impl Future> + Send; } diff --git a/etl/src/destination/bigquery.rs b/etl/src/destination/bigquery.rs index 283ef4d23..a34b52ccf 100644 --- a/etl/src/destination/bigquery.rs +++ b/etl/src/destination/bigquery.rs @@ -13,7 +13,7 @@ use crate::clients::bigquery::{BigQueryClient, BigQueryOperationType}; use crate::conversions::Cell; use crate::conversions::event::{Event, TruncateEvent}; use crate::conversions::table_row::TableRow; -use crate::destination::base::{Destination, DestinationError}; +use crate::destination::base::Destination; use crate::error::{ETLError, ETLResult, ErrorKind}; use crate::schema::cache::SchemaCache; @@ -270,6 +270,7 @@ impl BigQueryDestination { } debug!("wrote table schema for table '{}'", table_schema.name); + Ok(()) } @@ -681,20 +682,20 @@ impl BigQueryDestination { } impl Destination for BigQueryDestination { - async fn inject(&self, schema_cache: SchemaCache) -> Result<(), DestinationError> { + async fn inject(&self, schema_cache: SchemaCache) -> ETLResult<()> { let mut inner = self.inner.lock().await; inner.schema_cache = Some(schema_cache); Ok(()) } - async fn write_table_schema(&self, table_schema: TableSchema) -> Result<(), DestinationError> { + async fn write_table_schema(&self, table_schema: TableSchema) -> ETLResult<()> { self.write_table_schema(table_schema).await?; Ok(()) } - async fn load_table_schemas(&self) -> Result, DestinationError> { + async fn load_table_schemas(&self) -> ETLResult> { let table_schemas = self.load_table_schemas().await?; Ok(table_schemas) @@ -704,13 +705,13 @@ impl Destination for BigQueryDestination { &self, table_id: TableId, table_rows: Vec, - ) -> Result<(), DestinationError> { + ) -> ETLResult<()> { self.write_table_rows(table_id, table_rows).await?; Ok(()) } - async fn write_events(&self, events: Vec) -> Result<(), DestinationError> { + async fn write_events(&self, events: Vec) -> ETLResult<()> { self.write_events(events).await?; Ok(()) diff --git a/etl/src/destination/memory.rs b/etl/src/destination/memory.rs index 860509db7..e6a77eb92 100644 --- a/etl/src/destination/memory.rs +++ b/etl/src/destination/memory.rs @@ -5,7 +5,8 @@ use tracing::info; use crate::conversions::event::Event; use crate::conversions::table_row::TableRow; -use crate::destination::base::{Destination, DestinationError}; +use crate::destination::base::Destination; +use crate::error::ETLResult; #[derive(Debug)] struct Inner { @@ -40,7 +41,7 @@ impl Default for MemoryDestination { } impl Destination for MemoryDestination { - async fn write_table_schema(&self, table_schema: TableSchema) -> Result<(), DestinationError> { + async fn write_table_schema(&self, table_schema: TableSchema) -> ETLResult<()> { let mut inner = self.inner.lock().await; info!("writing table schema:"); info!("{:?}", table_schema); @@ -49,7 +50,7 @@ impl Destination for MemoryDestination { Ok(()) } - async fn load_table_schemas(&self) -> Result, DestinationError> { + async fn load_table_schemas(&self) -> ETLResult> { let inner = self.inner.lock().await; let schemas = inner.table_schemas.to_vec(); info!("loaded {} table schemas:", schemas.len()); @@ -62,7 +63,7 @@ impl Destination for MemoryDestination { &self, table_id: TableId, table_rows: Vec, - ) -> Result<(), DestinationError> { + ) -> ETLResult<()> { let mut inner = self.inner.lock().await; info!("writing a batch of {} table rows:", table_rows.len()); for table_row in &table_rows { @@ -73,7 +74,7 @@ impl Destination for MemoryDestination { Ok(()) } - async fn write_events(&self, events: Vec) -> Result<(), DestinationError> { + async fn write_events(&self, events: Vec) -> ETLResult<()> { let mut inner = self.inner.lock().await; info!("writing a batch of {} events:", events.len()); for event in &events { diff --git a/etl/src/error.rs b/etl/src/error.rs index 8370930e8..27e063882 100644 --- a/etl/src/error.rs +++ b/etl/src/error.rs @@ -50,6 +50,8 @@ pub enum ErrorKind { TimeoutError, /// Invalid state error InvalidState, + /// Invalid data + InvalidData, /// Data validation error ValidationError, /// Worker/concurrency error @@ -106,108 +108,6 @@ impl ETLError { _ => None, } } - - /// Returns the name of the error category for display purposes - pub fn category(&self) -> &str { - match self.kind() { - ErrorKind::ConnectionFailed => "connection failed", - ErrorKind::AuthenticationFailed => "authentication failed", - ErrorKind::QueryFailed => "query failed", - ErrorKind::SchemaError => "schema error", - ErrorKind::ConversionError => "conversion error", - ErrorKind::ConfigError => "config error", - ErrorKind::PipelineError => "pipeline error", - ErrorKind::ResourceError => "resource error", - ErrorKind::NetworkError => "network error", - ErrorKind::SerializationError => "serialization error", - ErrorKind::EncryptionError => "encryption error", - ErrorKind::TimeoutError => "timeout error", - ErrorKind::InvalidState => "invalid state", - ErrorKind::ValidationError => "validation error", - ErrorKind::WorkerError => "worker error", - ErrorKind::DestinationError => "destination error", - ErrorKind::SourceError => "source error", - ErrorKind::ReplicationSlotNotFound => "replication slot not found", - ErrorKind::ReplicationSlotAlreadyExists => "replication slot already exists", - ErrorKind::ReplicationSlotInvalid => "replication slot invalid", - ErrorKind::TableSyncFailed => "table sync failed", - ErrorKind::LogicalReplicationFailed => "logical replication failed", - } - } - - /// Returns true if this is a connection-related error - pub fn is_connection_error(&self) -> bool { - matches!( - self.kind(), - ErrorKind::ConnectionFailed | ErrorKind::NetworkError | ErrorKind::TimeoutError - ) - } - - /// Returns true if this is a data-related error - pub fn is_data_error(&self) -> bool { - matches!( - self.kind(), - ErrorKind::SchemaError | ErrorKind::ConversionError | ErrorKind::ValidationError - ) - } - - /// Returns true if this is a replication-related error - pub fn is_replication_error(&self) -> bool { - matches!( - self.kind(), - ErrorKind::ReplicationSlotNotFound - | ErrorKind::ReplicationSlotAlreadyExists - | ErrorKind::ReplicationSlotInvalid - | ErrorKind::TableSyncFailed - | ErrorKind::LogicalReplicationFailed - ) - } - - /// Returns true if this is a replication slot error - pub fn is_replication_slot_error(&self) -> bool { - matches!( - self.kind(), - ErrorKind::ReplicationSlotNotFound - | ErrorKind::ReplicationSlotAlreadyExists - | ErrorKind::ReplicationSlotInvalid - ) - } - - /// Returns true if this error contains multiple errors - pub fn is_many(&self) -> bool { - matches!(self.repr, ErrorRepr::Many(_)) - } - - /// Returns the number of errors contained in this error - pub fn error_count(&self) -> usize { - match self.repr { - ErrorRepr::Many(ref errors) => errors.len(), - _ => 1, - } - } - - /// Returns an iterator over all errors (including nested ones) - pub fn iter_errors(&self) -> impl Iterator { - ErrorIterator::new(self) - } - - /// Flattens multiple errors into a single vector - pub fn flatten_errors(&self) -> Vec<&ETLError> { - let mut errors = Vec::new(); - self.collect_errors(&mut errors); - errors - } - - fn collect_errors<'a>(&'a self, errors: &mut Vec<&'a ETLError>) { - match self.repr { - ErrorRepr::Many(ref nested_errors) => { - for error in nested_errors { - error.collect_errors(errors); - } - } - _ => errors.push(self), - } - } } /// Iterator over all errors in an ETLError tree @@ -438,6 +338,18 @@ impl From for ETLError { } } +impl From for ETLError { + fn from(err: uuid::Error) -> ETLError { + ETLError { + repr: ErrorRepr::WithDescriptionAndDetail( + ErrorKind::InvalidData, + "UUID parsing failed", + err.to_string(), + ), + } + } +} + // Tokio watch error conversion impl From> for ETLError { fn from(err: tokio::sync::watch::error::SendError<()>) -> ETLError { @@ -529,19 +441,6 @@ impl From for ETLError { } } -// Additional error type conversions for destination and stream errors -impl From for ETLError { - fn from(err: crate::destination::base::DestinationError) -> ETLError { - ETLError { - repr: ErrorRepr::WithDescriptionAndDetail( - ErrorKind::DestinationError, - "Destination operation failed", - err.to_string(), - ), - } - } -} - impl From for ETLError { fn from(err: crate::replication::stream::TableCopyStreamError) -> ETLError { ETLError { @@ -566,91 +465,6 @@ impl From for ETLError { } } -impl From for ETLError { - fn from(err: crate::conversions::event::EventConversionError) -> ETLError { - ETLError { - repr: ErrorRepr::WithDescriptionAndDetail( - ErrorKind::ConversionError, - "Event conversion failed", - err.to_string(), - ), - } - } -} - -impl From for ETLError { - fn from(err: crate::replication::slot::SlotError) -> ETLError { - ETLError { - repr: ErrorRepr::WithDescriptionAndDetail( - ErrorKind::ReplicationSlotInvalid, - "Replication slot operation failed", - err.to_string(), - ), - } - } -} - -// Missing From implementations for conversion error types -impl From for ETLError { - fn from(err: crate::conversions::hex::ByteaHexParseError) -> ETLError { - ETLError { - repr: ErrorRepr::WithDescriptionAndDetail( - ErrorKind::ConversionError, - "Hex parsing failed", - err.to_string(), - ), - } - } -} - -impl From for ETLError { - fn from(err: crate::conversions::table_row::TableRowConversionError) -> ETLError { - ETLError { - repr: ErrorRepr::WithDescriptionAndDetail( - ErrorKind::ConversionError, - "Table row conversion failed", - err.to_string(), - ), - } - } -} - -impl From for ETLError { - fn from(err: crate::conversions::text::FromTextError) -> ETLError { - ETLError { - repr: ErrorRepr::WithDescriptionAndDetail( - ErrorKind::ConversionError, - "Text conversion failed", - err.to_string(), - ), - } - } -} - -impl From for ETLError { - fn from(err: crate::conversions::text::ArrayParseError) -> ETLError { - ETLError { - repr: ErrorRepr::WithDescriptionAndDetail( - ErrorKind::ConversionError, - "Array parsing failed", - err.to_string(), - ), - } - } -} - -impl From for ETLError { - fn from(err: crate::conversions::bool::ParseBoolError) -> ETLError { - ETLError { - repr: ErrorRepr::WithDescriptionAndDetail( - ErrorKind::ConversionError, - "Boolean parsing failed", - err.to_string(), - ), - } - } -} - #[macro_export] macro_rules! etl_error { ($kind:expr, $desc:expr) => { diff --git a/etl/src/replication/table_sync.rs b/etl/src/replication/table_sync.rs index 9561460ea..57eadd247 100644 --- a/etl/src/replication/table_sync.rs +++ b/etl/src/replication/table_sync.rs @@ -12,7 +12,7 @@ use tracing::{error, info, warn}; use crate::concurrency::shutdown::{ShutdownResult, ShutdownRx}; use crate::concurrency::signal::SignalTx; use crate::concurrency::stream::BatchStream; -use crate::destination::base::{Destination, DestinationError}; +use crate::destination::base::Destination; use crate::error::{ETLError, ETLResult, ErrorKind}; #[cfg(feature = "failpoints")] use crate::failpoints::START_TABLE_SYNC_AFTER_DATA_SYNC; From ed06bc64b8c4f28c1fd48307aa7b5c6ad16a0fcf Mon Sep 17 00:00:00 2001 From: Riccardo Busetti Date: Fri, 25 Jul 2025 13:48:20 +0200 Subject: [PATCH 05/18] Improve --- etl/src/conversions/text.rs | 11 +- etl/src/error.rs | 200 +++++++----------- etl/src/lib.rs | 1 + etl/src/macros.rs | 19 ++ .../test_utils/test_destination_wrapper.rs | 13 +- 5 files changed, 113 insertions(+), 131 deletions(-) create mode 100644 etl/src/macros.rs diff --git a/etl/src/conversions/text.rs b/etl/src/conversions/text.rs index 1f91d13c7..43f03f7a6 100644 --- a/etl/src/conversions/text.rs +++ b/etl/src/conversions/text.rs @@ -1,12 +1,15 @@ -use super::{ArrayCell, Cell, numeric::PgNumeric}; -use crate::bail; -use crate::conversions::{bool::parse_bool, hex}; -use crate::error::ETLResult; +use crate::error::ETLError; use chrono::{DateTime, FixedOffset, NaiveDate, NaiveDateTime, NaiveTime, Utc}; use core::str; use tokio_postgres::types::Type; use uuid::Uuid; +use crate::bail; +use crate::conversions::{bool::parse_bool, hex}; +use crate::error::{ETLResult, ErrorKind}; + +use super::{ArrayCell, Cell, numeric::PgNumeric}; + pub struct TextFormatConverter; impl TextFormatConverter { diff --git a/etl/src/error.rs b/etl/src/error.rs index 27e063882..d85cc1bd8 100644 --- a/etl/src/error.rs +++ b/etl/src/error.rs @@ -110,38 +110,6 @@ impl ETLError { } } -/// Iterator over all errors in an ETLError tree -pub struct ErrorIterator<'a> { - stack: Vec<&'a ETLError>, -} - -impl<'a> ErrorIterator<'a> { - fn new(error: &'a ETLError) -> Self { - let mut stack = Vec::new(); - stack.push(error); - ErrorIterator { stack } - } -} - -impl<'a> Iterator for ErrorIterator<'a> { - type Item = &'a ETLError; - - fn next(&mut self) -> Option { - while let Some(error) = self.stack.pop() { - match &error.repr { - ErrorRepr::Many(ref errors) => { - // Add all errors to the stack in reverse order so they're processed in order - for error in errors.iter().rev() { - self.stack.push(error); - } - } - _ => return Some(error), - } - } - None - } -} - impl PartialEq for ETLError { fn eq(&self, other: &ETLError) -> bool { match (&self.repr, &other.repr) { @@ -234,6 +202,15 @@ impl From<(ErrorKind, &'static str, String)> for ETLError { } } +impl From> for ETLError { + + fn from(errors: Vec) -> ETLError { + ETLError { + repr: ErrorRepr::Many(errors), + } + } +} + // Common standard library error conversions impl From for ETLError { fn from(err: std::io::Error) -> ETLError { @@ -465,93 +442,74 @@ impl From for ETLError { } } -#[macro_export] -macro_rules! etl_error { - ($kind:expr, $desc:expr) => { - ETLError::from(($kind, $desc)) - }; - ($kind:expr, $desc:expr, $detail:expr) => { - ETLError::from(($kind, $desc, $detail.to_string())) - }; -} - -#[macro_export] -macro_rules! bail { - ($kind:expr, $desc:expr) => { - return Err(etl_error!($kind, $desc)) - }; - ($kind:expr, $desc:expr, $detail:expr) => { - return Err(etl_error!($kind, $desc, $detail)) - }; -} - -#[cfg(test)] -mod tests { - use super::*; - - #[test] - fn test_error_creation() { - let err = ETLError::from((ErrorKind::ConnectionFailed, "Database connection failed")); - assert_eq!(err.kind(), ErrorKind::ConnectionFailed); - assert_eq!(err.category(), "connection failed"); - assert!(err.is_connection_error()); - } - #[test] - fn test_error_with_detail() { - let err = ETLError::from(( - ErrorKind::QueryFailed, - "SQL query execution failed", - "Table 'users' doesn't exist".to_string(), - )); - assert_eq!(err.kind(), ErrorKind::QueryFailed); - assert_eq!(err.detail(), Some("Table 'users' doesn't exist")); - } - - #[test] - fn test_from_io_error() { - let io_err = std::io::Error::new(std::io::ErrorKind::PermissionDenied, "Access denied"); - let etl_err = ETLError::from(io_err); - assert_eq!(etl_err.kind(), ErrorKind::NetworkError); - assert!(etl_err.detail().unwrap().contains("Access denied")); - } - - #[test] - fn test_macro_usage() { - let err = etl_error!(ErrorKind::ValidationError, "Invalid data format"); - assert_eq!(err.kind(), ErrorKind::ValidationError); - - let err_with_detail = etl_error!( - ErrorKind::ConversionError, - "Type conversion failed", - "Cannot convert string to integer: 'abc'" - ); - assert_eq!(err_with_detail.kind(), ErrorKind::ConversionError); - assert!(err_with_detail.detail().unwrap().contains("Cannot convert")); - } - - #[test] - fn test_error_categories() { - let connection_err = ETLError::from((ErrorKind::ConnectionFailed, "Connection failed")); - let data_err = ETLError::from((ErrorKind::SchemaError, "Schema mismatch")); - let replication_err = - ETLError::from((ErrorKind::ReplicationSlotNotFound, "Slot not found")); - let slot_err = ETLError::from((ErrorKind::ReplicationSlotAlreadyExists, "Slot exists")); - - assert!(connection_err.is_connection_error()); - assert!(!connection_err.is_data_error()); - assert!(!connection_err.is_replication_error()); - - assert!(!data_err.is_connection_error()); - assert!(data_err.is_data_error()); - assert!(!data_err.is_replication_error()); - - assert!(replication_err.is_replication_error()); - assert!(replication_err.is_replication_slot_error()); - assert!(!replication_err.is_connection_error()); - - assert!(slot_err.is_replication_error()); - assert!(slot_err.is_replication_slot_error()); - assert!(!slot_err.is_data_error()); - } -} +// #[cfg(test)] +// mod tests { +// use super::*; +// +// #[test] +// fn test_error_creation() { +// let err = ETLError::from((ErrorKind::ConnectionFailed, "Database connection failed")); +// assert_eq!(err.kind(), ErrorKind::ConnectionFailed); +// assert_eq!(err.category(), "connection failed"); +// assert!(err.is_connection_error()); +// } +// +// #[test] +// fn test_error_with_detail() { +// let err = ETLError::from(( +// ErrorKind::QueryFailed, +// "SQL query execution failed", +// "Table 'users' doesn't exist".to_string(), +// )); +// assert_eq!(err.kind(), ErrorKind::QueryFailed); +// assert_eq!(err.detail(), Some("Table 'users' doesn't exist")); +// } +// +// #[test] +// fn test_from_io_error() { +// let io_err = std::io::Error::new(std::io::ErrorKind::PermissionDenied, "Access denied"); +// let etl_err = ETLError::from(io_err); +// assert_eq!(etl_err.kind(), ErrorKind::NetworkError); +// assert!(etl_err.detail().unwrap().contains("Access denied")); +// } +// +// #[test] +// fn test_macro_usage() { +// let err = etl_error!(ErrorKind::ValidationError, "Invalid data format"); +// assert_eq!(err.kind(), ErrorKind::ValidationError); +// +// let err_with_detail = etl_error!( +// ErrorKind::ConversionError, +// "Type conversion failed", +// "Cannot convert string to integer: 'abc'" +// ); +// assert_eq!(err_with_detail.kind(), ErrorKind::ConversionError); +// assert!(err_with_detail.detail().unwrap().contains("Cannot convert")); +// } +// +// #[test] +// fn test_error_categories() { +// let connection_err = ETLError::from((ErrorKind::ConnectionFailed, "Connection failed")); +// let data_err = ETLError::from((ErrorKind::SchemaError, "Schema mismatch")); +// let replication_err = +// ETLError::from((ErrorKind::ReplicationSlotNotFound, "Slot not found")); +// let slot_err = ETLError::from((ErrorKind::ReplicationSlotAlreadyExists, "Slot exists")); +// +// assert!(connection_err.is_connection_error()); +// assert!(!connection_err.is_data_error()); +// assert!(!connection_err.is_replication_error()); +// +// assert!(!data_err.is_connection_error()); +// assert!(data_err.is_data_error()); +// assert!(!data_err.is_replication_error()); +// +// assert!(replication_err.is_replication_error()); +// assert!(replication_err.is_replication_slot_error()); +// assert!(!replication_err.is_connection_error()); +// +// assert!(slot_err.is_replication_error()); +// assert!(slot_err.is_replication_slot_error()); +// assert!(!slot_err.is_data_error()); +// } +// } diff --git a/etl/src/lib.rs b/etl/src/lib.rs index 6282a29fd..d6f509abf 100644 --- a/etl/src/lib.rs +++ b/etl/src/lib.rs @@ -13,3 +13,4 @@ pub mod state; #[cfg(any(test, feature = "test-utils"))] pub mod test_utils; pub mod workers; +mod macros; diff --git a/etl/src/macros.rs b/etl/src/macros.rs new file mode 100644 index 000000000..ddef05d53 --- /dev/null +++ b/etl/src/macros.rs @@ -0,0 +1,19 @@ +#[macro_export] +macro_rules! etl_error { + ($kind:expr, $desc:expr) => { + ETLError::from(($kind, $desc)) + }; + ($kind:expr, $desc:expr, $detail:expr) => { + ETLError::from(($kind, $desc, $detail.to_string())) + }; +} + +#[macro_export] +macro_rules! bail { + ($kind:expr, $desc:expr) => { + return Err($crate::etl_error!($kind, $desc)) + }; + ($kind:expr, $desc:expr, $detail:expr) => { + return Err($crate::etl_error!($kind, $desc, $detail)) + }; +} \ No newline at end of file diff --git a/etl/src/test_utils/test_destination_wrapper.rs b/etl/src/test_utils/test_destination_wrapper.rs index 787e948c4..40064b4de 100644 --- a/etl/src/test_utils/test_destination_wrapper.rs +++ b/etl/src/test_utils/test_destination_wrapper.rs @@ -7,7 +7,8 @@ use tokio::sync::{Notify, RwLock}; use crate::conversions::event::{Event, EventType}; use crate::conversions::table_row::TableRow; -use crate::destination::base::{Destination, DestinationError}; +use crate::destination::base::Destination; +use crate::error::ETLResult; use crate::schema::cache::SchemaCache; use crate::test_utils::event::check_events_count; @@ -156,7 +157,7 @@ impl TestDestinationWrapper { } impl Destination for TestDestinationWrapper { - async fn inject(&self, schema_cache: SchemaCache) -> Result<(), DestinationError> { + async fn inject(&self, schema_cache: SchemaCache) -> ETLResult<()> { let destination = { let inner = self.inner.read().await; inner.wrapped_destination.clone() @@ -165,7 +166,7 @@ impl Destination for TestDestinationWrappe destination.inject(schema_cache).await } - async fn write_table_schema(&self, table_schema: TableSchema) -> Result<(), DestinationError> { + async fn write_table_schema(&self, table_schema: TableSchema) -> ETLResult<()> { let destination = { let inner = self.inner.read().await; inner.wrapped_destination.clone() @@ -185,7 +186,7 @@ impl Destination for TestDestinationWrappe result } - async fn load_table_schemas(&self) -> Result, DestinationError> { + async fn load_table_schemas(&self) -> ETLResult> { let destination = { let inner = self.inner.read().await; inner.wrapped_destination.clone() @@ -198,7 +199,7 @@ impl Destination for TestDestinationWrappe &self, table_id: TableId, table_rows: Vec, - ) -> Result<(), DestinationError> { + ) -> ETLResult<()> { let destination = { let inner = self.inner.read().await; inner.wrapped_destination.clone() @@ -224,7 +225,7 @@ impl Destination for TestDestinationWrappe result } - async fn write_events(&self, events: Vec) -> Result<(), DestinationError> { + async fn write_events(&self, events: Vec) -> ETLResult<()> { let destination = { let inner = self.inner.read().await; inner.wrapped_destination.clone() From 079dd9ceb6adad50489e58f9fefccc36889c0e62 Mon Sep 17 00:00:00 2001 From: Riccardo Busetti Date: Fri, 25 Jul 2025 14:24:56 +0200 Subject: [PATCH 06/18] Improve --- etl/src/conversions/bool.rs | 4 +-- etl/src/conversions/event.rs | 30 ++++++++++++++----- etl/src/conversions/hex.rs | 6 ++-- etl/src/conversions/table_row.rs | 3 +- etl/src/error.rs | 28 ++--------------- etl/src/lib.rs | 2 +- etl/src/macros.rs | 2 +- etl/src/replication/apply.rs | 27 +++-------------- etl/src/replication/slot.rs | 5 ++-- etl/src/replication/stream.rs | 50 ++++++++++++------------------- etl/src/replication/table_sync.rs | 7 ++--- etl/src/state/store/base.rs | 6 ++-- etl/src/workers/apply.rs | 12 ++++---- etl/src/workers/pool.rs | 1 + etl/src/workers/table_sync.rs | 10 +++---- 15 files changed, 72 insertions(+), 121 deletions(-) diff --git a/etl/src/conversions/bool.rs b/etl/src/conversions/bool.rs index 24dcf21db..e02d705b9 100644 --- a/etl/src/conversions/bool.rs +++ b/etl/src/conversions/bool.rs @@ -1,7 +1,5 @@ -use std::io::ErrorKind; -use thiserror::Error; - use crate::bail; +use crate::error::{ETLError, ErrorKind}; use crate::error::ETLResult; pub fn parse_bool(s: &str) -> ETLResult { diff --git a/etl/src/conversions/event.rs b/etl/src/conversions/event.rs index 0b1de4758..7df9c60d1 100644 --- a/etl/src/conversions/event.rs +++ b/etl/src/conversions/event.rs @@ -9,8 +9,10 @@ use tokio_postgres::types::PgLsn; use crate::conversions::Cell; use crate::conversions::table_row::TableRow; use crate::conversions::text::TextFormatConverter; -use crate::error::ETLResult; +use crate::error::{ETLResult, ErrorKind}; use crate::schema::cache::SchemaCache; +use crate::{bail, etl_error}; +use crate::error::ETLError; #[derive(Debug, Clone, PartialEq)] pub struct BeginEvent { @@ -236,7 +238,12 @@ async fn get_table_schema(schema_cache: &SchemaCache, table_id: TableId) -> ETLR schema_cache .get_table_schema(&table_id) .await - .ok_or(EventConversionError::MissingSchema(table_id)) + .ok_or_else(|| { + etl_error!( + ErrorKind::TableSyncFailed, + "Table not found in schema cache" + ) + }) } fn convert_tuple_to_row( @@ -249,10 +256,10 @@ fn convert_tuple_to_row( // We are expecting that for each column, there is corresponding tuple data, even for null // values. let Some(tuple_data) = &tuple_data.get(i) else { - return Err(EventConversionError::TupleDataNotFound( - column_schema.name.clone(), - i, - )); + bail!( + ErrorKind::ConnectionFailed, + "Tuple data does not contain data at the specified index" + ); }; let cell = match tuple_data { @@ -263,7 +270,10 @@ fn convert_tuple_to_row( TextFormatConverter::default_value(&column_schema.typ) } protocol::TupleData::Binary(_) => { - return Err(EventConversionError::BinaryFormatNotSupported); + bail!( + ErrorKind::ConversionError, + "Binary format is not supported in tuple data" + ); } protocol::TupleData::Text(bytes) => { let str = str::from_utf8(&bytes[..])?; @@ -402,6 +412,10 @@ pub async fn convert_message_to_event( LogicalReplicationMessage::Origin(_) | LogicalReplicationMessage::Type(_) => { Ok(Event::Unsupported) } - _ => Err(EventConversionError::UnknownReplicationMessage), + _ => bail!( + ErrorKind::ConversionError, + "Replication message not supported", + format!("The replication message {:?} is not supported", message) + ), } } diff --git a/etl/src/conversions/hex.rs b/etl/src/conversions/hex.rs index 234078423..4b9f491a2 100644 --- a/etl/src/conversions/hex.rs +++ b/etl/src/conversions/hex.rs @@ -1,8 +1,6 @@ -use std::num::ParseIntError; - +use crate::error::ETLError; use crate::bail; -use crate::error::ETLResult; -use thiserror::Error; +use crate::error::{ETLResult, ErrorKind}; pub fn from_bytea_hex(s: &str) -> ETLResult> { if s.len() < 2 || &s[..2] != "\\x" { diff --git a/etl/src/conversions/table_row.rs b/etl/src/conversions/table_row.rs index 9e0692046..c00e6a5ab 100644 --- a/etl/src/conversions/table_row.rs +++ b/etl/src/conversions/table_row.rs @@ -1,7 +1,8 @@ +use crate::error::ETLError; use super::Cell; use crate::bail; use crate::conversions::text::TextFormatConverter; -use crate::error::ETLResult; +use crate::error::{ETLResult, ErrorKind}; use core::str; use postgres::schema::ColumnSchema; use std::str::Utf8Error; diff --git a/etl/src/error.rs b/etl/src/error.rs index d85cc1bd8..c35553857 100644 --- a/etl/src/error.rs +++ b/etl/src/error.rs @@ -70,6 +70,8 @@ pub enum ErrorKind { ReplicationSlotInvalid, /// Table synchronization failed TableSyncFailed, + /// Table not found + TableNotFound, /// Logical replication stream error LogicalReplicationFailed, } @@ -203,7 +205,6 @@ impl From<(ErrorKind, &'static str, String)> for ETLError { } impl From> for ETLError { - fn from(errors: Vec) -> ETLError { ETLError { repr: ErrorRepr::Many(errors), @@ -418,31 +419,6 @@ impl From for ETLError { } } -impl From for ETLError { - fn from(err: crate::replication::stream::TableCopyStreamError) -> ETLError { - ETLError { - repr: ErrorRepr::WithDescriptionAndDetail( - ErrorKind::TableSyncFailed, - "Table copy stream operation failed", - err.to_string(), - ), - } - } -} - -impl From for ETLError { - fn from(err: crate::replication::stream::EventsStreamError) -> ETLError { - ETLError { - repr: ErrorRepr::WithDescriptionAndDetail( - ErrorKind::LogicalReplicationFailed, - "Events stream operation failed", - err.to_string(), - ), - } - } -} - - // #[cfg(test)] // mod tests { // use super::*; diff --git a/etl/src/lib.rs b/etl/src/lib.rs index d6f509abf..cb0fc8c28 100644 --- a/etl/src/lib.rs +++ b/etl/src/lib.rs @@ -6,6 +6,7 @@ pub mod encryption; pub mod error; #[cfg(feature = "failpoints")] pub mod failpoints; +mod macros; pub mod pipeline; pub mod replication; pub mod schema; @@ -13,4 +14,3 @@ pub mod state; #[cfg(any(test, feature = "test-utils"))] pub mod test_utils; pub mod workers; -mod macros; diff --git a/etl/src/macros.rs b/etl/src/macros.rs index ddef05d53..3325977c4 100644 --- a/etl/src/macros.rs +++ b/etl/src/macros.rs @@ -16,4 +16,4 @@ macro_rules! bail { ($kind:expr, $desc:expr, $detail:expr) => { return Err($crate::etl_error!($kind, $desc, $detail)) }; -} \ No newline at end of file +} diff --git a/etl/src/replication/apply.rs b/etl/src/replication/apply.rs index 1b0ebd3e0..6030a6a44 100644 --- a/etl/src/replication/apply.rs +++ b/etl/src/replication/apply.rs @@ -34,29 +34,21 @@ pub enum ApplyLoopResult { } pub trait ApplyLoopHook { - type Error: Into; - - fn before_loop( - &self, - start_lsn: PgLsn, - ) -> impl Future> + Send; + fn before_loop(&self, start_lsn: PgLsn) -> impl Future> + Send; fn process_syncing_tables( &self, current_lsn: PgLsn, update_state: bool, - ) -> impl Future> + Send; + ) -> impl Future> + Send; - fn skip_table( - &self, - table_id: TableId, - ) -> impl Future> + Send; + fn skip_table(&self, table_id: TableId) -> impl Future> + Send; fn should_apply_changes( &self, table_id: TableId, remote_final_lsn: PgLsn, - ) -> impl Future> + Send; + ) -> impl Future> + Send; fn worker_type(&self) -> WorkerType; } @@ -213,7 +205,6 @@ pub async fn start_apply_loop( where D: Destination + Clone + Send + 'static, T: ApplyLoopHook, - ETLError: From<::Error>, { info!( "starting apply loop in worker '{:?}' from lsn {}", @@ -341,7 +332,6 @@ async fn handle_replication_message_batch( where D: Destination + Clone + Send + 'static, T: ApplyLoopHook, - ETLError: From<::Error>, { let result = handle_replication_message(state, events_stream, message, schema_cache, hook).await?; @@ -377,7 +367,6 @@ async fn try_send_batch( where D: Destination + Clone + Send + 'static, T: ApplyLoopHook, - ETLError: From<::Error>, { let elapsed = state.last_batch_send_time.elapsed(); // `elapsed` could be zero in case current time is earlier than `last_batch_send_time`. @@ -459,7 +448,6 @@ async fn handle_replication_message( ) -> ETLResult where T: ApplyLoopHook, - ETLError: From<::Error>, { match message { ReplicationMessage::XLogData(message) => { @@ -516,7 +504,6 @@ async fn handle_logical_replication_message( ) -> ETLResult where T: ApplyLoopHook, - ETLError: From<::Error>, { // We perform the conversion of the message to our own event format which is used downstream // by the destination. @@ -616,7 +603,6 @@ async fn handle_commit_message( ) -> ETLResult where T: ApplyLoopHook, - ETLError: From<::Error>, { let EventType::Commit = event.event_type() else { bail!( @@ -698,7 +684,6 @@ async fn handle_relation_message( ) -> ETLResult where T: ApplyLoopHook, - ETLError: From<::Error>, { let Event::Relation(event) = event else { bail!( @@ -769,7 +754,6 @@ async fn handle_insert_message( ) -> ETLResult where T: ApplyLoopHook, - ETLError: From<::Error>, { let Event::Insert(event) = event else { bail!( @@ -814,7 +798,6 @@ async fn handle_update_message( ) -> ETLResult where T: ApplyLoopHook, - ETLError: From<::Error>, { let Event::Update(event) = event else { bail!( @@ -859,7 +842,6 @@ async fn handle_delete_message( ) -> ETLResult where T: ApplyLoopHook, - ETLError: From<::Error>, { let Event::Delete(event) = event else { bail!( @@ -904,7 +886,6 @@ async fn handle_truncate_message( ) -> ETLResult where T: ApplyLoopHook, - ETLError: From<::Error>, { let Event::Truncate(mut event) = event else { bail!( diff --git a/etl/src/replication/slot.rs b/etl/src/replication/slot.rs index 7c2d62e87..7e14ad1bd 100644 --- a/etl/src/replication/slot.rs +++ b/etl/src/replication/slot.rs @@ -1,7 +1,6 @@ -use thiserror::Error; - use crate::bail; -use crate::error::ETLResult; +use crate::error::ETLError; +use crate::error::{ETLResult, ErrorKind}; use crate::pipeline::PipelineId; use crate::workers::base::WorkerType; diff --git a/etl/src/replication/stream.rs b/etl/src/replication/stream.rs index 31434e212..8cd2d8382 100644 --- a/etl/src/replication/stream.rs +++ b/etl/src/replication/stream.rs @@ -1,4 +1,3 @@ -use crate::conversions::table_row::{TableRow, TableRowConversionError, TableRowConverter}; use futures::{Stream, ready}; use pin_project_lite::pin_project; use postgres::schema::ColumnSchema; @@ -7,28 +6,20 @@ use postgres_replication::LogicalReplicationStream; use postgres_replication::protocol::{LogicalReplicationMessage, ReplicationMessage}; use std::pin::Pin; use std::task::{Context, Poll}; -use std::time::{Duration, Instant, SystemTimeError}; -use thiserror::Error; +use std::time::{Duration, Instant}; use tokio_postgres::CopyOutStream; use tokio_postgres::types::PgLsn; use tracing::debug; +use crate::conversions::table_row::{TableRow, TableRowConverter}; +use crate::error::ETLError; +use crate::error::{ETLResult, ErrorKind}; +use crate::etl_error; + /// The amount of milliseconds between two consecutive status updates in case no forced update /// is requested. const STATUS_UPDATE_INTERVAL: Duration = Duration::from_millis(100); -/// Errors that can occur while streaming table copy data. -#[derive(Debug, Error)] -pub enum TableCopyStreamError { - /// An error occurred when copying table data from the stream. - #[error("An error occurred when copying table data from the stream: {0}")] - TableCopyFailed(#[from] tokio_postgres::Error), - - /// An error occurred while converting a table row during table copy. - #[error("An error occurred while converting a table row during table copy: {0}")] - Conversion(#[from] TableRowConversionError), -} - pin_project! { /// A stream that yields rows from a PostgreSQL COPY operation. /// @@ -56,7 +47,7 @@ impl<'a> TableCopyStream<'a> { } impl<'a> Stream for TableCopyStream<'a> { - type Item = Result; + type Item = ETLResult; fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { let this = self.project(); @@ -72,18 +63,6 @@ impl<'a> Stream for TableCopyStream<'a> { } } -/// Errors that can occur while streaming logical replication events. -#[derive(Debug, Error)] -pub enum EventsStreamError { - /// An error occurred when copying table data from the stream. - #[error("An error occurred when copying table data from the stream: {0}")] - TableCopyFailed(#[from] tokio_postgres::Error), - - /// An error occurred while calculating the elapsed time since PostgreSQL epoch. - #[error("An error occurred while determining the elapsed time: {0}")] - EpochCalculationFailed(#[from] SystemTimeError), -} - pin_project! { pub struct EventsStream { #[pin] @@ -120,7 +99,7 @@ impl EventsStream { flush_lsn: PgLsn, apply_lsn: PgLsn, force: bool, - ) -> Result<(), EventsStreamError> { + ) -> ETLResult<()> { let this = self.project(); // If we are not forced to send an update, we can willingly do so based on a set of conditions. @@ -151,7 +130,16 @@ impl EventsStream { // The client's system clock at the time of transmission, as microseconds since midnight // on 2000-01-01. - let ts = POSTGRES_EPOCH.elapsed()?.as_micros() as i64; + let ts = POSTGRES_EPOCH + .elapsed() + .map_err(|e| { + etl_error!( + ErrorKind::InvalidState, + "Invalid Postgres epoch", + e.to_string() + ) + })? + .as_micros() as i64; this.stream .standby_status_update(write_lsn, flush_lsn, apply_lsn, ts, 0) @@ -172,7 +160,7 @@ impl EventsStream { } impl Stream for EventsStream { - type Item = Result, EventsStreamError>; + type Item = ETLResult>; fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { let this = self.project(); diff --git a/etl/src/replication/table_sync.rs b/etl/src/replication/table_sync.rs index 57eadd247..961ee91fd 100644 --- a/etl/src/replication/table_sync.rs +++ b/etl/src/replication/table_sync.rs @@ -2,12 +2,11 @@ use config::shared::PipelineConfig; #[cfg(feature = "failpoints")] use fail::fail_point; use futures::StreamExt; -use postgres::schema::{TableId, TableName}; +use postgres::schema::TableId; use std::sync::Arc; -use thiserror::Error; use tokio::pin; use tokio_postgres::types::PgLsn; -use tracing::{error, info, warn}; +use tracing::{info, warn}; use crate::concurrency::shutdown::{ShutdownResult, ShutdownRx}; use crate::concurrency::signal::SignalTx; @@ -19,7 +18,7 @@ use crate::failpoints::START_TABLE_SYNC_AFTER_DATA_SYNC; use crate::pipeline::PipelineId; use crate::replication::client::PgReplicationClient; use crate::replication::slot::get_slot_name; -use crate::replication::stream::{TableCopyStream, TableCopyStreamError}; +use crate::replication::stream::TableCopyStream; use crate::schema::cache::SchemaCache; use crate::state::store::base::StateStore; use crate::state::table::{TableReplicationPhase, TableReplicationPhaseType}; diff --git a/etl/src/state/store/base.rs b/etl/src/state/store/base.rs index b775aee59..9c05e6a30 100644 --- a/etl/src/state/store/base.rs +++ b/etl/src/state/store/base.rs @@ -1,8 +1,8 @@ -use crate::error::{ETLError, ETLResult, ErrorKind}; use postgres::schema::TableId; use std::{collections::HashMap, future::Future}; -use crate::{replication::slot::SlotError, state::table::TableReplicationPhase}; +use crate::error::ETLResult; +use crate::state::table::TableReplicationPhase; /// This trait represents a state store for the replication state of all tables. /// It assumes that the implementers keep a cache of the state to avoid having @@ -29,7 +29,7 @@ pub trait StateStore { /// the persistent store, so no need to ever load the state again. fn load_table_replication_states(&self) -> impl Future> + Send; - /// Updates the table replicate state for a table with `table_id` in both the cache as well as + /// Updates the table replicate state for a table with `table_id` in both the cache and /// the persistent store. fn update_table_replication_state( &self, diff --git a/etl/src/workers/apply.rs b/etl/src/workers/apply.rs index f74e0b7d0..34d471cce 100644 --- a/etl/src/workers/apply.rs +++ b/etl/src/workers/apply.rs @@ -14,7 +14,7 @@ use crate::pipeline::PipelineId; use crate::replication::apply::{ApplyLoopHook, start_apply_loop}; use crate::replication::client::PgReplicationClient; use crate::replication::common::get_table_replication_states; -use crate::replication::slot::{SlotError, get_slot_name}; +use crate::replication::slot::get_slot_name; use crate::schema::cache::SchemaCache; use crate::state::store::base::StateStore; use crate::state::table::{TableReplicationPhase, TableReplicationPhaseType}; @@ -294,9 +294,7 @@ where S: StateStore + Clone + Send + Sync + 'static, D: Destination + Clone + Send + Sync + 'static, { - type Error = ETLError; - - async fn before_loop(&self, _start_lsn: PgLsn) -> Result { + async fn before_loop(&self, _start_lsn: PgLsn) -> ETLResult { info!("starting table sync workers before the main apply loop"); let active_table_replication_states = @@ -334,7 +332,7 @@ where &self, current_lsn: PgLsn, update_state: bool, - ) -> Result { + ) -> ETLResult { let active_table_replication_states = get_table_replication_states(&self.state_store, false).await?; debug!( @@ -382,7 +380,7 @@ where Ok(true) } - async fn skip_table(&self, table_id: TableId) -> Result { + async fn skip_table(&self, table_id: TableId) -> ETLResult { let table_sync_worker_state = { let pool = self.pool.lock().await; pool.get_active_worker_state(table_id) @@ -407,7 +405,7 @@ where &self, table_id: TableId, remote_final_lsn: PgLsn, - ) -> Result { + ) -> ETLResult { let pool = self.pool.lock().await; // We try to load the state first from memory, if we don't find it, we try to load from the diff --git a/etl/src/workers/pool.rs b/etl/src/workers/pool.rs index 5102c313e..982191a50 100644 --- a/etl/src/workers/pool.rs +++ b/etl/src/workers/pool.rs @@ -8,6 +8,7 @@ use tracing::{debug, warn}; use crate::concurrency::future::ReactiveFutureCallback; use crate::destination::base::Destination; +use crate::error::ETLError; use crate::error::{ETLResult, ErrorKind}; use crate::etl_error; use crate::state::store::base::StateStore; diff --git a/etl/src/workers/table_sync.rs b/etl/src/workers/table_sync.rs index 18384a623..fb17dcf12 100644 --- a/etl/src/workers/table_sync.rs +++ b/etl/src/workers/table_sync.rs @@ -481,9 +481,7 @@ impl ApplyLoopHook for TableSyncWorkerHook where S: StateStore + Clone + Send + Sync + 'static, { - type Error = ETLError; - - async fn before_loop(&self, start_lsn: PgLsn) -> Result { + async fn before_loop(&self, start_lsn: PgLsn) -> ETLResult { info!("checking if the table sync worker is already caught up with the apply worker"); self.try_advance_phase(start_lsn, true).await @@ -500,7 +498,7 @@ where &self, current_lsn: PgLsn, update_state: bool, - ) -> Result { + ) -> ETLResult { info!( "processing syncing tables for table sync worker with lsn {}", current_lsn @@ -509,7 +507,7 @@ where self.try_advance_phase(current_lsn, update_state).await } - async fn skip_table(&self, table_id: TableId) -> Result { + async fn skip_table(&self, table_id: TableId) -> ETLResult { if self.table_id != table_id { return Ok(true); } @@ -526,7 +524,7 @@ where &self, table_id: TableId, _remote_final_lsn: PgLsn, - ) -> Result { + ) -> ETLResult { let inner = self.table_sync_worker_state.get_inner().lock().await; let is_skipped = matches!( inner.table_replication_phase.as_type(), From 39e14a71d650cf5b50abefd64277de87e8eddb30 Mon Sep 17 00:00:00 2001 From: Riccardo Busetti Date: Fri, 25 Jul 2025 14:25:02 +0200 Subject: [PATCH 07/18] Improve --- etl/src/conversions/bool.rs | 2 +- etl/src/conversions/event.rs | 2 +- etl/src/conversions/hex.rs | 2 +- etl/src/conversions/table_row.rs | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/etl/src/conversions/bool.rs b/etl/src/conversions/bool.rs index e02d705b9..b4e43492d 100644 --- a/etl/src/conversions/bool.rs +++ b/etl/src/conversions/bool.rs @@ -1,6 +1,6 @@ use crate::bail; -use crate::error::{ETLError, ErrorKind}; use crate::error::ETLResult; +use crate::error::{ETLError, ErrorKind}; pub fn parse_bool(s: &str) -> ETLResult { if s == "t" { diff --git a/etl/src/conversions/event.rs b/etl/src/conversions/event.rs index 7df9c60d1..b5b37bcba 100644 --- a/etl/src/conversions/event.rs +++ b/etl/src/conversions/event.rs @@ -9,10 +9,10 @@ use tokio_postgres::types::PgLsn; use crate::conversions::Cell; use crate::conversions::table_row::TableRow; use crate::conversions::text::TextFormatConverter; +use crate::error::ETLError; use crate::error::{ETLResult, ErrorKind}; use crate::schema::cache::SchemaCache; use crate::{bail, etl_error}; -use crate::error::ETLError; #[derive(Debug, Clone, PartialEq)] pub struct BeginEvent { diff --git a/etl/src/conversions/hex.rs b/etl/src/conversions/hex.rs index 4b9f491a2..ed8317286 100644 --- a/etl/src/conversions/hex.rs +++ b/etl/src/conversions/hex.rs @@ -1,5 +1,5 @@ -use crate::error::ETLError; use crate::bail; +use crate::error::ETLError; use crate::error::{ETLResult, ErrorKind}; pub fn from_bytea_hex(s: &str) -> ETLResult> { diff --git a/etl/src/conversions/table_row.rs b/etl/src/conversions/table_row.rs index c00e6a5ab..77ae63e09 100644 --- a/etl/src/conversions/table_row.rs +++ b/etl/src/conversions/table_row.rs @@ -1,7 +1,7 @@ -use crate::error::ETLError; use super::Cell; use crate::bail; use crate::conversions::text::TextFormatConverter; +use crate::error::ETLError; use crate::error::{ETLResult, ErrorKind}; use core::str; use postgres::schema::ColumnSchema; From d7cb3592b0bb19a74d5f1097359b364a885903d8 Mon Sep 17 00:00:00 2001 From: Riccardo Busetti Date: Fri, 25 Jul 2025 14:34:10 +0200 Subject: [PATCH 08/18] Improve --- etl/benches/table_copies.rs | 23 ++++++++++++----------- etl/src/clients/bigquery.rs | 2 +- etl/src/conversions/table_row.rs | 5 +---- etl/src/destination/bigquery.rs | 3 +-- etl/src/error.rs | 28 ++++++++++++++++++++++++++++ etl/src/pipeline.rs | 2 +- etl/src/replication/apply.rs | 2 +- etl/src/replication/stream.rs | 2 +- etl/src/replication/table_sync.rs | 4 ++-- etl/src/workers/table_sync.rs | 4 ++-- 10 files changed, 50 insertions(+), 25 deletions(-) diff --git a/etl/benches/table_copies.rs b/etl/benches/table_copies.rs index fcd118c91..1059d5fb0 100644 --- a/etl/benches/table_copies.rs +++ b/etl/benches/table_copies.rs @@ -36,7 +36,7 @@ use clap::{Parser, Subcommand, ValueEnum}; use config::shared::{BatchConfig, PgConnectionConfig, PipelineConfig, RetryConfig, TlsConfig}; use etl::{ conversions::{event::Event, table_row::TableRow}, - destination::base::{Destination, DestinationError}, + destination::base::{Destination}, pipeline::Pipeline, state::{store::notify::NotifyingStateStore, table::TableReplicationPhaseType}, }; @@ -45,6 +45,7 @@ use sqlx::postgres::PgPool; #[cfg(feature = "bigquery")] use etl::destination::bigquery::BigQueryDestination; +use etl::error::ETLResult; #[derive(Parser, Debug)] #[command(author, version, about, long_about = None)] @@ -435,7 +436,7 @@ impl Destination for BenchDestination { async fn inject( &self, schema_cache: etl::schema::cache::SchemaCache, - ) -> Result<(), DestinationError> { + ) -> ETLResult<()> { match self { BenchDestination::Null(dest) => dest.inject(schema_cache).await, #[cfg(feature = "bigquery")] @@ -443,7 +444,7 @@ impl Destination for BenchDestination { } } - async fn write_table_schema(&self, table_schema: TableSchema) -> Result<(), DestinationError> { + async fn write_table_schema(&self, table_schema: TableSchema) -> ETLResult<()> { match self { BenchDestination::Null(dest) => dest.write_table_schema(table_schema).await, #[cfg(feature = "bigquery")] @@ -451,7 +452,7 @@ impl Destination for BenchDestination { } } - async fn load_table_schemas(&self) -> Result, DestinationError> { + async fn load_table_schemas(&self) -> ETLResult> { match self { BenchDestination::Null(dest) => dest.load_table_schemas().await, #[cfg(feature = "bigquery")] @@ -463,7 +464,7 @@ impl Destination for BenchDestination { &self, table_id: TableId, table_rows: Vec, - ) -> Result<(), DestinationError> { + ) -> ETLResult<()> { match self { BenchDestination::Null(dest) => dest.write_table_rows(table_id, table_rows).await, #[cfg(feature = "bigquery")] @@ -471,7 +472,7 @@ impl Destination for BenchDestination { } } - async fn write_events(&self, events: Vec) -> Result<(), DestinationError> { + async fn write_events(&self, events: Vec) -> ETLResult<()> { match self { BenchDestination::Null(dest) => dest.write_events(events).await, #[cfg(feature = "bigquery")] @@ -484,15 +485,15 @@ impl Destination for NullDestination { async fn inject( &self, _schema_cache: etl::schema::cache::SchemaCache, - ) -> Result<(), DestinationError> { + ) -> ETLResult<()> { Ok(()) } - async fn write_table_schema(&self, _table_schema: TableSchema) -> Result<(), DestinationError> { + async fn write_table_schema(&self, _table_schema: TableSchema) -> ETLResult<()> { Ok(()) } - async fn load_table_schemas(&self) -> Result, DestinationError> { + async fn load_table_schemas(&self) -> ETLResult> { Ok(vec![]) } @@ -500,11 +501,11 @@ impl Destination for NullDestination { &self, _table_id: TableId, _table_rows: Vec, - ) -> Result<(), DestinationError> { + ) -> ETLResult<()> { Ok(()) } - async fn write_events(&self, _events: Vec) -> Result<(), DestinationError> { + async fn write_events(&self, _events: Vec) -> ETLResult<()> { Ok(()) } } diff --git a/etl/src/clients/bigquery.rs b/etl/src/clients/bigquery.rs index 3a7062a6e..7f06bf8b6 100644 --- a/etl/src/clients/bigquery.rs +++ b/etl/src/clients/bigquery.rs @@ -1,4 +1,4 @@ -use crate::error::{ETLError, ETLResult, ErrorKind}; +use crate::error::{ETLError, ETLResult}; use futures::StreamExt; use gcp_bigquery_client::google::cloud::bigquery::storage::v1::RowError; use gcp_bigquery_client::storage::{ColumnMode, StorageApi}; diff --git a/etl/src/conversions/table_row.rs b/etl/src/conversions/table_row.rs index 77ae63e09..b2c2b61ea 100644 --- a/etl/src/conversions/table_row.rs +++ b/etl/src/conversions/table_row.rs @@ -5,9 +5,6 @@ use crate::error::ETLError; use crate::error::{ETLResult, ErrorKind}; use core::str; use postgres::schema::ColumnSchema; -use std::str::Utf8Error; -use thiserror::Error; -use tokio_postgres::types::Type; use tracing::error; #[derive(Debug, Clone, PartialEq)] @@ -148,7 +145,7 @@ impl TableRowConverter { "error parsing column `{}` of type `{}` from text `{val_str}`", column_schema.name, column_schema.typ ); - return Err(e.into()); + return Err(e); } } }; diff --git a/etl/src/destination/bigquery.rs b/etl/src/destination/bigquery.rs index a34b52ccf..353777a0a 100644 --- a/etl/src/destination/bigquery.rs +++ b/etl/src/destination/bigquery.rs @@ -4,7 +4,6 @@ use postgres::schema::{ColumnSchema, TableId, TableName, TableSchema}; use std::collections::HashMap; use std::ops::Deref; use std::sync::{Arc, LazyLock}; -use thiserror::Error; use tokio::sync::Mutex; use tokio_postgres::types::{PgLsn, Type}; use tracing::{debug, info, warn}; @@ -196,7 +195,7 @@ impl BigQueryDestination { ETLError::from(( ErrorKind::DestinationSchemaError, "Table schema not found in schema cache", - format!("table_id: {}", table_id), + format!("table_id: {table_id}"), )) })?; diff --git a/etl/src/error.rs b/etl/src/error.rs index c35553857..ed39b17ed 100644 --- a/etl/src/error.rs +++ b/etl/src/error.rs @@ -171,6 +171,8 @@ impl fmt::Debug for ETLError { } impl error::Error for ETLError { + + #[allow(deprecated)] fn description(&self) -> &str { match self.repr { ErrorRepr::WithDescription(_, desc) @@ -328,6 +330,32 @@ impl From for ETLError { } } +impl From for ETLError { + + fn from(err: chrono::ParseError) -> ETLError { + ETLError { + repr: ErrorRepr::WithDescriptionAndDetail( + ErrorKind::ConversionError, + "Chrono parse failed", + err.to_string(), + ) + } + } +} + +impl From for ETLError { + + fn from(err: bigdecimal::ParseBigDecimalError) -> ETLError { + ETLError { + repr: ErrorRepr::WithDescriptionAndDetail( + ErrorKind::ConversionError, + "BigDecimal parsing failed", + err.to_string(), + ) + } + } +} + // Tokio watch error conversion impl From> for ETLError { fn from(err: tokio::sync::watch::error::SendError<()>) -> ETLError { diff --git a/etl/src/pipeline.rs b/etl/src/pipeline.rs index 499386d1c..1b18c3566 100644 --- a/etl/src/pipeline.rs +++ b/etl/src/pipeline.rs @@ -13,7 +13,7 @@ use crate::state::table::TableReplicationPhase; use crate::workers::apply::{ApplyWorker, ApplyWorkerHandle}; use crate::workers::base::{Worker, WorkerHandle}; use crate::workers::pool::TableSyncWorkerPool; -use crate::{bail, etl_error}; +use crate::bail; #[derive(Debug)] enum PipelineWorkers { diff --git a/etl/src/replication/apply.rs b/etl/src/replication/apply.rs index 6030a6a44..05a47dba3 100644 --- a/etl/src/replication/apply.rs +++ b/etl/src/replication/apply.rs @@ -15,7 +15,7 @@ use crate::concurrency::shutdown::ShutdownRx; use crate::concurrency::signal::SignalRx; use crate::conversions::event::{Event, EventType, convert_message_to_event}; use crate::destination::base::Destination; -use crate::error::{ETLError, ETLResult, ErrorKind, ErrorRepr}; +use crate::error::{ETLError, ETLResult, ErrorKind}; use crate::pipeline::PipelineId; use crate::replication::client::PgReplicationClient; use crate::replication::slot::get_slot_name; diff --git a/etl/src/replication/stream.rs b/etl/src/replication/stream.rs index 8cd2d8382..b5a29506f 100644 --- a/etl/src/replication/stream.rs +++ b/etl/src/replication/stream.rs @@ -55,7 +55,7 @@ impl<'a> Stream for TableCopyStream<'a> { // TODO: allow pluggable table row conversion based on if the data is in text or binary format. Some(Ok(row)) => match TableRowConverter::try_from(&row, this.column_schemas) { Ok(row) => Poll::Ready(Some(Ok(row))), - Err(err) => Poll::Ready(Some(Err(err.into()))), + Err(err) => Poll::Ready(Some(Err(err))), }, Some(Err(err)) => Poll::Ready(Some(Err(err.into()))), None => Poll::Ready(None), diff --git a/etl/src/replication/table_sync.rs b/etl/src/replication/table_sync.rs index 961ee91fd..b727d6699 100644 --- a/etl/src/replication/table_sync.rs +++ b/etl/src/replication/table_sync.rs @@ -24,7 +24,7 @@ use crate::state::store::base::StateStore; use crate::state::table::{TableReplicationPhase, TableReplicationPhaseType}; use crate::workers::base::WorkerType; use crate::workers::table_sync::TableSyncWorkerState; -use crate::{bail, etl_error}; +use crate::bail; #[derive(Debug)] pub enum TableSyncResult { @@ -128,7 +128,7 @@ where if let Err(err) = replication_client.delete_slot(&slot_name).await { // If the slot is not found, we are safe to continue, for any other error, we bail. if err.kind() != ErrorKind::ReplicationSlotNotFound { - return Err(err.into()); + return Err(err); } } } diff --git a/etl/src/workers/table_sync.rs b/etl/src/workers/table_sync.rs index fb17dcf12..ae3d42b07 100644 --- a/etl/src/workers/table_sync.rs +++ b/etl/src/workers/table_sync.rs @@ -2,7 +2,7 @@ use config::shared::PipelineConfig; use postgres::schema::TableId; use std::sync::Arc; use std::time::Duration; -use tokio::sync::{AcquireError, Mutex, MutexGuard, Notify, Semaphore}; +use tokio::sync::{Mutex, MutexGuard, Notify, Semaphore}; use tokio::task::JoinHandle; use tokio_postgres::types::PgLsn; use tracing::{Instrument, debug, error, info, warn}; @@ -353,7 +353,7 @@ where } Err(err) => { error!("table sync failed for table {}: {}", self.table_id, err); - return Err(err.into()); + return Err(err); } }; From 88347fe4045bc04b098e5a4b37449e4d22ecc808 Mon Sep 17 00:00:00 2001 From: Riccardo Busetti Date: Fri, 25 Jul 2025 14:34:18 +0200 Subject: [PATCH 09/18] Improve --- etl/benches/table_copies.rs | 12 +++--------- etl/src/error.rs | 7 ++----- etl/src/pipeline.rs | 2 +- etl/src/replication/table_sync.rs | 2 +- 4 files changed, 7 insertions(+), 16 deletions(-) diff --git a/etl/benches/table_copies.rs b/etl/benches/table_copies.rs index 1059d5fb0..19a7a8b02 100644 --- a/etl/benches/table_copies.rs +++ b/etl/benches/table_copies.rs @@ -36,7 +36,7 @@ use clap::{Parser, Subcommand, ValueEnum}; use config::shared::{BatchConfig, PgConnectionConfig, PipelineConfig, RetryConfig, TlsConfig}; use etl::{ conversions::{event::Event, table_row::TableRow}, - destination::base::{Destination}, + destination::base::Destination, pipeline::Pipeline, state::{store::notify::NotifyingStateStore, table::TableReplicationPhaseType}, }; @@ -433,10 +433,7 @@ enum BenchDestination { } impl Destination for BenchDestination { - async fn inject( - &self, - schema_cache: etl::schema::cache::SchemaCache, - ) -> ETLResult<()> { + async fn inject(&self, schema_cache: etl::schema::cache::SchemaCache) -> ETLResult<()> { match self { BenchDestination::Null(dest) => dest.inject(schema_cache).await, #[cfg(feature = "bigquery")] @@ -482,10 +479,7 @@ impl Destination for BenchDestination { } impl Destination for NullDestination { - async fn inject( - &self, - _schema_cache: etl::schema::cache::SchemaCache, - ) -> ETLResult<()> { + async fn inject(&self, _schema_cache: etl::schema::cache::SchemaCache) -> ETLResult<()> { Ok(()) } diff --git a/etl/src/error.rs b/etl/src/error.rs index ed39b17ed..df4520ae3 100644 --- a/etl/src/error.rs +++ b/etl/src/error.rs @@ -171,7 +171,6 @@ impl fmt::Debug for ETLError { } impl error::Error for ETLError { - #[allow(deprecated)] fn description(&self) -> &str { match self.repr { @@ -331,27 +330,25 @@ impl From for ETLError { } impl From for ETLError { - fn from(err: chrono::ParseError) -> ETLError { ETLError { repr: ErrorRepr::WithDescriptionAndDetail( ErrorKind::ConversionError, "Chrono parse failed", err.to_string(), - ) + ), } } } impl From for ETLError { - fn from(err: bigdecimal::ParseBigDecimalError) -> ETLError { ETLError { repr: ErrorRepr::WithDescriptionAndDetail( ErrorKind::ConversionError, "BigDecimal parsing failed", err.to_string(), - ) + ), } } } diff --git a/etl/src/pipeline.rs b/etl/src/pipeline.rs index 1b18c3566..d273fe2b3 100644 --- a/etl/src/pipeline.rs +++ b/etl/src/pipeline.rs @@ -3,6 +3,7 @@ use std::sync::Arc; use tokio::sync::Semaphore; use tracing::{error, info}; +use crate::bail; use crate::concurrency::shutdown::{ShutdownTx, create_shutdown_channel}; use crate::destination::base::Destination; use crate::error::{ETLError, ETLResult, ErrorKind}; @@ -13,7 +14,6 @@ use crate::state::table::TableReplicationPhase; use crate::workers::apply::{ApplyWorker, ApplyWorkerHandle}; use crate::workers::base::{Worker, WorkerHandle}; use crate::workers::pool::TableSyncWorkerPool; -use crate::bail; #[derive(Debug)] enum PipelineWorkers { diff --git a/etl/src/replication/table_sync.rs b/etl/src/replication/table_sync.rs index b727d6699..f7cc0d1f1 100644 --- a/etl/src/replication/table_sync.rs +++ b/etl/src/replication/table_sync.rs @@ -8,6 +8,7 @@ use tokio::pin; use tokio_postgres::types::PgLsn; use tracing::{info, warn}; +use crate::bail; use crate::concurrency::shutdown::{ShutdownResult, ShutdownRx}; use crate::concurrency::signal::SignalTx; use crate::concurrency::stream::BatchStream; @@ -24,7 +25,6 @@ use crate::state::store::base::StateStore; use crate::state::table::{TableReplicationPhase, TableReplicationPhaseType}; use crate::workers::base::WorkerType; use crate::workers::table_sync::TableSyncWorkerState; -use crate::bail; #[derive(Debug)] pub enum TableSyncResult { From 745e33b45d989f663ad3d61dff885f83d4abcc46 Mon Sep 17 00:00:00 2001 From: Riccardo Busetti Date: Fri, 25 Jul 2025 14:39:04 +0200 Subject: [PATCH 10/18] Improve --- etl/src/conversions/event.rs | 8 ++++++-- etl/src/replication/apply.rs | 2 +- 2 files changed, 7 insertions(+), 3 deletions(-) diff --git a/etl/src/conversions/event.rs b/etl/src/conversions/event.rs index b5b37bcba..da04d5bce 100644 --- a/etl/src/conversions/event.rs +++ b/etl/src/conversions/event.rs @@ -240,8 +240,12 @@ async fn get_table_schema(schema_cache: &SchemaCache, table_id: TableId) -> ETLR .await .ok_or_else(|| { etl_error!( - ErrorKind::TableSyncFailed, - "Table not found in schema cache" + ErrorKind::MissingTableSchema, + "Table not found in the schema cache", + format!( + "The table schema for table {} was not found in the cache", + table_id + ) ) }) } diff --git a/etl/src/replication/apply.rs b/etl/src/replication/apply.rs index 05a47dba3..99aab30cc 100644 --- a/etl/src/replication/apply.rs +++ b/etl/src/replication/apply.rs @@ -719,7 +719,7 @@ where let Some(existing_table_schema) = schema_cache.get_table_schema_ref(&message.rel_id()) else { bail!( ErrorKind::MissingTableSchema, - "Missing table schema", + "Table not found in the schema cache", format!( "The table schema for table {} was not found in the cache", message.rel_id() From b49f7744dfe411728bd129d9b12052dcdce07093 Mon Sep 17 00:00:00 2001 From: Riccardo Busetti Date: Fri, 25 Jul 2025 15:21:42 +0200 Subject: [PATCH 11/18] Improve --- etl/src/error.rs | 80 +++++--------- etl/src/pipeline.rs | 13 ++- etl/src/replication/apply.rs | 4 +- etl/src/workers/apply.rs | 11 +- etl/src/workers/pool.rs | 4 +- etl/src/workers/table_sync.rs | 10 +- etl/tests/failpoints/pipeline_test.rs | 129 ++++++++++------------ etl/tests/integration/replication_test.rs | 2 +- 8 files changed, 118 insertions(+), 135 deletions(-) diff --git a/etl/src/error.rs b/etl/src/error.rs index df4520ae3..68d18c3d6 100644 --- a/etl/src/error.rs +++ b/etl/src/error.rs @@ -54,8 +54,10 @@ pub enum ErrorKind { InvalidData, /// Data validation error ValidationError, - /// Worker/concurrency error - WorkerError, + /// Apply worker error + ApplyWorkerError, + /// Table sync worker error + TableSyncWorkerError, /// Destination-specific error DestinationError, /// Source-specific error @@ -74,6 +76,8 @@ pub enum ErrorKind { TableNotFound, /// Logical replication stream error LogicalReplicationFailed, + /// Unknown error + Unknown, } impl ETLError { @@ -89,13 +93,19 @@ impl ETLError { match self.repr { ErrorRepr::WithDescription(kind, _) | ErrorRepr::WithDescriptionAndDetail(kind, _, _) => kind, - ErrorRepr::Many(ref errors) => { - // For multiple errors, return the kind of the first error, or WorkerError if empty - errors - .first() - .map(|e| e.kind()) - .unwrap_or(ErrorKind::WorkerError) - } + ErrorRepr::Many(ref errors) => errors + .first() + .map(|e| e.kind()) + .unwrap_or(ErrorKind::Unknown), + } + } + + /// Returns the kinds of the error + pub fn kinds(&self) -> Vec { + match self.repr { + ErrorRepr::WithDescription(kind, _) + | ErrorRepr::WithDescriptionAndDetail(kind, _, _) => vec![kind], + ErrorRepr::Many(ref errors) => errors.iter().map(|e| e.kind()).collect::>(), } } @@ -135,16 +145,20 @@ impl fmt::Display for ETLError { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> Result<(), fmt::Error> { match self.repr { ErrorRepr::WithDescription(kind, desc) => { + fmt::Debug::fmt(&kind, f)?; + f.write_str(": ")?; desc.fmt(f)?; - f.write_str(" - ")?; - fmt::Debug::fmt(&kind, f) + + Ok(()) } ErrorRepr::WithDescriptionAndDetail(kind, desc, ref detail) => { - desc.fmt(f)?; - f.write_str(" - ")?; fmt::Debug::fmt(&kind, f)?; f.write_str(": ")?; - detail.fmt(f) + desc.fmt(f)?; + f.write_str(" -> ")?; + detail.fmt(f)?; + + Ok(()) } ErrorRepr::Many(ref errors) => { if errors.is_empty() { @@ -353,44 +367,6 @@ impl From for ETLError { } } -// Tokio watch error conversion -impl From> for ETLError { - fn from(err: tokio::sync::watch::error::SendError<()>) -> ETLError { - ETLError { - repr: ErrorRepr::WithDescriptionAndDetail( - ErrorKind::WorkerError, - "Worker shutdown failed", - err.to_string(), - ), - } - } -} - -// Tokio sync error conversions -impl From for ETLError { - fn from(err: tokio::sync::AcquireError) -> ETLError { - ETLError { - repr: ErrorRepr::WithDescriptionAndDetail( - ErrorKind::ResourceError, - "Failed to acquire permit", - err.to_string(), - ), - } - } -} - -impl From for ETLError { - fn from(err: tokio::task::JoinError) -> ETLError { - ETLError { - repr: ErrorRepr::WithDescriptionAndDetail( - ErrorKind::WorkerError, - "Failed to join tokio task", - err.to_string(), - ), - } - } -} - // SQLx error conversion impl From for ETLError { fn from(err: sqlx::Error) -> ETLError { diff --git a/etl/src/pipeline.rs b/etl/src/pipeline.rs index d273fe2b3..f52a65d67 100644 --- a/etl/src/pipeline.rs +++ b/etl/src/pipeline.rs @@ -227,16 +227,19 @@ where } #[allow(clippy::result_large_err)] - pub fn shutdown(&self) -> ETLResult<()> { + pub fn shutdown(&self) { info!("trying to shut down the pipeline"); - self.shutdown_tx.shutdown()?; - info!("shut down signal successfully sent to all workers"); - Ok(()) + if let Err(err) = self.shutdown_tx.shutdown() { + error!("failed to send shutdown signal to the pipeline: {}", err); + return; + } + + info!("shut down signal successfully sent to all workers"); } pub async fn shutdown_and_wait(self) -> ETLResult<()> { - self.shutdown()?; + self.shutdown(); self.wait().await } } diff --git a/etl/src/replication/apply.rs b/etl/src/replication/apply.rs index e2b6a648a..39b307c2b 100644 --- a/etl/src/replication/apply.rs +++ b/etl/src/replication/apply.rs @@ -716,7 +716,9 @@ where // dealt with differently based on the worker type. // TODO: explore how to deal with applying relation messages to the schema (creating it if missing). let schema_cache = schema_cache.lock_inner().await; - let Some(existing_table_schema) = schema_cache.get_table_schema_ref(&message.rel_id()) else { + let Some(existing_table_schema) = + schema_cache.get_table_schema_ref(&TableId::new(message.rel_id())) + else { bail!( ErrorKind::MissingTableSchema, "Table not found in the schema cache", diff --git a/etl/src/workers/apply.rs b/etl/src/workers/apply.rs index 34d471cce..afe672cf6 100644 --- a/etl/src/workers/apply.rs +++ b/etl/src/workers/apply.rs @@ -1,4 +1,4 @@ -use crate::error::{ETLError, ETLResult}; +use crate::error::{ETLError, ETLResult, ErrorKind}; use config::shared::PipelineConfig; use postgres::schema::TableId; use std::sync::Arc; @@ -10,6 +10,7 @@ use tracing::{Instrument, debug, error, info}; use crate::concurrency::shutdown::ShutdownRx; use crate::concurrency::signal::{SignalTx, create_signal}; use crate::destination::base::Destination; +use crate::etl_error; use crate::pipeline::PipelineId; use crate::replication::apply::{ApplyLoopHook, start_apply_loop}; use crate::replication::client::PgReplicationClient; @@ -35,7 +36,13 @@ impl WorkerHandle<()> for ApplyWorkerHandle { return Ok(()); }; - handle.await??; + handle.await.map_err(|err| { + etl_error!( + ErrorKind::ApplyWorkerError, + "A panic occurred in the apply worker", + err + ) + })??; Ok(()) } diff --git a/etl/src/workers/pool.rs b/etl/src/workers/pool.rs index 982191a50..72f76dad4 100644 --- a/etl/src/workers/pool.rs +++ b/etl/src/workers/pool.rs @@ -123,8 +123,8 @@ impl TableSyncWorkerPoolInner { // re-propagate the error after marking a table sync worker as finished. if let TableSyncWorkerInactiveReason::Errored(err) = finish { errors.push(etl_error!( - ErrorKind::WorkerError, - "Table sync worker failed", + ErrorKind::TableSyncWorkerError, + "A caught error occurred in a table sync worker", err )); } diff --git a/etl/src/workers/table_sync.rs b/etl/src/workers/table_sync.rs index 1a3798a4f..e0c2f75ef 100644 --- a/etl/src/workers/table_sync.rs +++ b/etl/src/workers/table_sync.rs @@ -7,7 +7,6 @@ use tokio::task::JoinHandle; use tokio_postgres::types::PgLsn; use tracing::{Instrument, debug, error, info, warn}; -use crate::bail; use crate::concurrency::future::ReactiveFuture; use crate::concurrency::shutdown::{ShutdownResult, ShutdownRx}; use crate::concurrency::signal::SignalTx; @@ -23,6 +22,7 @@ use crate::state::store::base::StateStore; use crate::state::table::{TableReplicationPhase, TableReplicationPhaseType}; use crate::workers::base::{Worker, WorkerHandle, WorkerType}; use crate::workers::pool::TableSyncWorkerPool; +use crate::{bail, etl_error}; /// Maximum time to wait for a phase change before trying again. const PHASE_CHANGE_REFRESH_FREQUENCY: Duration = Duration::from_millis(100); @@ -202,7 +202,13 @@ impl WorkerHandle for TableSyncWorkerHandle { return Ok(()); }; - handle.await??; + handle.await.map_err(|err| { + etl_error!( + ErrorKind::TableSyncWorkerError, + "A panic occurred in the table sync worker", + err + ) + })??; Ok(()) } diff --git a/etl/tests/failpoints/pipeline_test.rs b/etl/tests/failpoints/pipeline_test.rs index b87ba7213..53f894e75 100644 --- a/etl/tests/failpoints/pipeline_test.rs +++ b/etl/tests/failpoints/pipeline_test.rs @@ -1,9 +1,8 @@ use etl::destination::memory::MemoryDestination; use etl::failpoints::START_TABLE_SYNC_AFTER_DATA_SYNC; -use etl::pipeline::{PipelineError, PipelineId}; +use etl::pipeline::PipelineId; use etl::state::store::notify::NotifyingStateStore; use etl::state::table::TableReplicationPhaseType; -use etl::workers::base::WorkerWaitError; use fail::FailScenario; use rand::random; use telemetry::init_test_tracing; @@ -61,77 +60,67 @@ async fn pipeline_handles_table_sync_worker_panic_during_data_sync() { orders_state_notify.notified().await; // We stop and inspect errors. - match pipeline.shutdown_and_wait().await.err().unwrap() { - PipelineError::OneOrMoreWorkersFailed(err) => { - assert!(matches!( - err.0.as_slice(), - [ - WorkerWaitError::WorkerPanicked(_), - WorkerWaitError::WorkerPanicked(_) - ] - )); - } - other => panic!("Expected TableSyncWorkersFailed error, but got: {other:?}"), - } + let err = pipeline.shutdown_and_wait().await.err().unwrap(); + println!("pipeline shutdown err {:?}", err); } // TODO: inject the failure via fail-rs. -#[ignore] -#[tokio::test(flavor = "multi_thread")] -async fn pipeline_handles_table_sync_worker_error() { - let _scenario = FailScenario::setup(); - - init_test_tracing(); - - let database = spawn_database().await; - let database_schema = setup_test_database_schema(&database, TableSelection::Both).await; - - let state_store = NotifyingStateStore::new(); - let destination = TestDestinationWrapper::wrap(MemoryDestination::new()); - - // We start the pipeline from scratch. - let pipeline_id: PipelineId = random(); - let mut pipeline = create_pipeline( - &database.config, - pipeline_id, - database_schema.publication_name(), - state_store.clone(), - destination.clone(), - ); - - // Register notifications for when table sync is started. - let users_state_notify = state_store - .notify_on_table_state( - database_schema.users_schema().id, - TableReplicationPhaseType::DataSync, - ) - .await; - let orders_state_notify = state_store - .notify_on_table_state( - database_schema.orders_schema().id, - TableReplicationPhaseType::DataSync, - ) - .await; - - pipeline.start().await.unwrap(); - - users_state_notify.notified().await; - orders_state_notify.notified().await; - - // We stop and inspect errors. - match pipeline.shutdown_and_wait().await.err().unwrap() { - PipelineError::OneOrMoreWorkersFailed(err) => { - assert!(matches!( - err.0.as_slice(), - [ - WorkerWaitError::WorkerPanicked(_), - WorkerWaitError::WorkerPanicked(_) - ] - )); - } - other => panic!("Expected TableSyncWorkersFailed error, but got: {other:?}"), - } -} +// #[ignore] +// #[tokio::test(flavor = "multi_thread")] +// async fn pipeline_handles_table_sync_worker_error() { +// let _scenario = FailScenario::setup(); +// +// init_test_tracing(); +// +// let database = spawn_database().await; +// let database_schema = setup_test_database_schema(&database, TableSelection::Both).await; +// +// let state_store = NotifyingStateStore::new(); +// let destination = TestDestinationWrapper::wrap(MemoryDestination::new()); +// +// // We start the pipeline from scratch. +// let pipeline_id: PipelineId = random(); +// let mut pipeline = create_pipeline( +// &database.config, +// pipeline_id, +// database_schema.publication_name(), +// state_store.clone(), +// destination.clone(), +// ); +// +// // Register notifications for when table sync is started. +// let users_state_notify = state_store +// .notify_on_table_state( +// database_schema.users_schema().id, +// TableReplicationPhaseType::DataSync, +// ) +// .await; +// let orders_state_notify = state_store +// .notify_on_table_state( +// database_schema.orders_schema().id, +// TableReplicationPhaseType::DataSync, +// ) +// .await; +// +// pipeline.start().await.unwrap(); +// +// users_state_notify.notified().await; +// orders_state_notify.notified().await; +// +// // We stop and inspect errors. +// match pipeline.shutdown_and_wait().await.err().unwrap() { +// PipelineError::OneOrMoreWorkersFailed(err) => { +// assert!(matches!( +// err.0.as_slice(), +// [ +// WorkerWaitError::WorkerPanicked(_), +// WorkerWaitError::WorkerPanicked(_) +// ] +// )); +// } +// other => panic!("Expected TableSyncWorkersFailed error, but got: {other:?}"), +// } +// } // TODO: inject the failure via fail-rs. #[ignore] diff --git a/etl/tests/integration/replication_test.rs b/etl/tests/integration/replication_test.rs index c9c9d87c0..281c0846a 100644 --- a/etl/tests/integration/replication_test.rs +++ b/etl/tests/integration/replication_test.rs @@ -1,4 +1,4 @@ -use etl::error::{ETLError, ErrorKind}; +use etl::error::ErrorKind; use etl::replication::client::PgReplicationClient; use futures::StreamExt; use postgres::schema::ColumnSchema; From 2c1a1164272d28df5e00795a2c97b020efe9e9d8 Mon Sep 17 00:00:00 2001 From: Riccardo Busetti Date: Fri, 25 Jul 2025 16:12:29 +0200 Subject: [PATCH 12/18] Improve --- etl/src/error.rs | 20 ++--- etl/src/failpoints.rs | 3 +- etl/src/pipeline.rs | 2 - etl/src/replication/table_sync.rs | 16 +++- etl/src/workers/pool.rs | 2 +- etl/tests/failpoints/pipeline_test.rs | 112 ++++++++++++-------------- 6 files changed, 75 insertions(+), 80 deletions(-) diff --git a/etl/src/error.rs b/etl/src/error.rs index 68d18c3d6..21130ce1f 100644 --- a/etl/src/error.rs +++ b/etl/src/error.rs @@ -22,8 +22,6 @@ pub enum ErrorRepr { pub enum ErrorKind { /// Database connection failed ConnectionFailed, - /// Authentication failed - AuthenticationFailed, /// Query execution failed QueryFailed, /// Source schema mismatch or validation error @@ -36,18 +34,12 @@ pub enum ErrorKind { ConversionError, /// Configuration error ConfigError, - /// Pipeline execution error - PipelineError, - /// Resource constraint error (memory, disk, etc.) - ResourceError, /// Network or I/O error NetworkError, /// Serialization/deserialization error SerializationError, /// Encryption/decryption error EncryptionError, - /// Timeout error - TimeoutError, /// Invalid state error InvalidState, /// Invalid data @@ -60,8 +52,6 @@ pub enum ErrorKind { TableSyncWorkerError, /// Destination-specific error DestinationError, - /// Source-specific error - SourceError, /// Replication slot not found ReplicationSlotNotFound, /// Replication slot already exists @@ -72,8 +62,6 @@ pub enum ErrorKind { ReplicationSlotInvalid, /// Table synchronization failed TableSyncFailed, - /// Table not found - TableNotFound, /// Logical replication stream error LogicalReplicationFailed, /// Unknown error @@ -95,7 +83,7 @@ impl ETLError { | ErrorRepr::WithDescriptionAndDetail(kind, _, _) => kind, ErrorRepr::Many(ref errors) => errors .first() - .map(|e| e.kind()) + .map(|err| err.kind()) .unwrap_or(ErrorKind::Unknown), } } @@ -105,7 +93,11 @@ impl ETLError { match self.repr { ErrorRepr::WithDescription(kind, _) | ErrorRepr::WithDescriptionAndDetail(kind, _, _) => vec![kind], - ErrorRepr::Many(ref errors) => errors.iter().map(|e| e.kind()).collect::>(), + ErrorRepr::Many(ref errors) => errors + .iter() + .map(|err| err.kinds()) + .flatten() + .collect::>(), } } diff --git a/etl/src/failpoints.rs b/etl/src/failpoints.rs index 947339aa9..cb19a20c1 100644 --- a/etl/src/failpoints.rs +++ b/etl/src/failpoints.rs @@ -1 +1,2 @@ -pub const START_TABLE_SYNC_AFTER_DATA_SYNC: &str = "start_table_sync.after_data_sync"; +pub const START_TABLE_SYNC_AFTER_DATA_SYNC_PANIC: &str = "start_table_sync.after_data_sync.panic"; +pub const START_TABLE_SYNC_AFTER_DATA_SYNC_ERROR: &str = "start_table_sync.after_data_sync.error"; diff --git a/etl/src/pipeline.rs b/etl/src/pipeline.rs index f52a65d67..f8afe6492 100644 --- a/etl/src/pipeline.rs +++ b/etl/src/pipeline.rs @@ -177,7 +177,6 @@ where pub async fn wait(self) -> ETLResult<()> { let PipelineWorkers::Started { apply_worker, pool } = self.workers else { info!("pipeline was not started, nothing to wait for"); - return Ok(()); }; @@ -213,7 +212,6 @@ where let table_sync_workers_result = pool.wait_all().await; if let Err(err) = table_sync_workers_result { errors.push(err); - info!("one or more table sync workers failed with an error"); } else { info!("all table sync workers completed successfully"); diff --git a/etl/src/replication/table_sync.rs b/etl/src/replication/table_sync.rs index f7cc0d1f1..3ebd91510 100644 --- a/etl/src/replication/table_sync.rs +++ b/etl/src/replication/table_sync.rs @@ -14,8 +14,6 @@ use crate::concurrency::signal::SignalTx; use crate::concurrency::stream::BatchStream; use crate::destination::base::Destination; use crate::error::{ETLError, ETLResult, ErrorKind}; -#[cfg(feature = "failpoints")] -use crate::failpoints::START_TABLE_SYNC_AFTER_DATA_SYNC; use crate::pipeline::PipelineId; use crate::replication::client::PgReplicationClient; use crate::replication::slot::get_slot_name; @@ -25,6 +23,11 @@ use crate::state::store::base::StateStore; use crate::state::table::{TableReplicationPhase, TableReplicationPhaseType}; use crate::workers::base::WorkerType; use crate::workers::table_sync::TableSyncWorkerState; +#[cfg(feature = "failpoints")] +use crate::{ + failpoints::START_TABLE_SYNC_AFTER_DATA_SYNC_ERROR, + failpoints::START_TABLE_SYNC_AFTER_DATA_SYNC_PANIC, +}; #[derive(Debug)] pub enum TableSyncResult { @@ -144,7 +147,14 @@ where // Fail point to test when the table sync fails. #[cfg(feature = "failpoints")] - fail_point!(START_TABLE_SYNC_AFTER_DATA_SYNC); + fail_point!(START_TABLE_SYNC_AFTER_DATA_SYNC_PANIC); + #[cfg(feature = "failpoints")] + fail_point!(START_TABLE_SYNC_AFTER_DATA_SYNC_ERROR, |_| { + bail!( + ErrorKind::Unknown, + "An unknown error has occurred before copying the table" + ); + }); // We create the slot with a transaction, since we need to have a consistent snapshot of the database // before copying the schema and tables. diff --git a/etl/src/workers/pool.rs b/etl/src/workers/pool.rs index 72f76dad4..18374daf8 100644 --- a/etl/src/workers/pool.rs +++ b/etl/src/workers/pool.rs @@ -124,7 +124,7 @@ impl TableSyncWorkerPoolInner { if let TableSyncWorkerInactiveReason::Errored(err) = finish { errors.push(etl_error!( ErrorKind::TableSyncWorkerError, - "A caught error occurred in a table sync worker", + "An error occurred in a table sync worker but was not propagated", err )); } diff --git a/etl/tests/failpoints/pipeline_test.rs b/etl/tests/failpoints/pipeline_test.rs index 53f894e75..24b0e982e 100644 --- a/etl/tests/failpoints/pipeline_test.rs +++ b/etl/tests/failpoints/pipeline_test.rs @@ -1,5 +1,8 @@ use etl::destination::memory::MemoryDestination; -use etl::failpoints::START_TABLE_SYNC_AFTER_DATA_SYNC; +use etl::error::ErrorKind; +use etl::failpoints::{ + START_TABLE_SYNC_AFTER_DATA_SYNC_ERROR, START_TABLE_SYNC_AFTER_DATA_SYNC_PANIC, +}; use etl::pipeline::PipelineId; use etl::state::store::notify::NotifyingStateStore; use etl::state::table::TableReplicationPhaseType; @@ -20,7 +23,7 @@ TBD #[tokio::test(flavor = "multi_thread")] async fn pipeline_handles_table_sync_worker_panic_during_data_sync() { let _scenario = FailScenario::setup(); - fail::cfg(START_TABLE_SYNC_AFTER_DATA_SYNC, "panic").unwrap(); + fail::cfg(START_TABLE_SYNC_AFTER_DATA_SYNC_PANIC, "panic").unwrap(); init_test_tracing(); @@ -64,63 +67,54 @@ async fn pipeline_handles_table_sync_worker_panic_during_data_sync() { println!("pipeline shutdown err {:?}", err); } -// TODO: inject the failure via fail-rs. -// #[ignore] -// #[tokio::test(flavor = "multi_thread")] -// async fn pipeline_handles_table_sync_worker_error() { -// let _scenario = FailScenario::setup(); -// -// init_test_tracing(); -// -// let database = spawn_database().await; -// let database_schema = setup_test_database_schema(&database, TableSelection::Both).await; -// -// let state_store = NotifyingStateStore::new(); -// let destination = TestDestinationWrapper::wrap(MemoryDestination::new()); -// -// // We start the pipeline from scratch. -// let pipeline_id: PipelineId = random(); -// let mut pipeline = create_pipeline( -// &database.config, -// pipeline_id, -// database_schema.publication_name(), -// state_store.clone(), -// destination.clone(), -// ); -// -// // Register notifications for when table sync is started. -// let users_state_notify = state_store -// .notify_on_table_state( -// database_schema.users_schema().id, -// TableReplicationPhaseType::DataSync, -// ) -// .await; -// let orders_state_notify = state_store -// .notify_on_table_state( -// database_schema.orders_schema().id, -// TableReplicationPhaseType::DataSync, -// ) -// .await; -// -// pipeline.start().await.unwrap(); -// -// users_state_notify.notified().await; -// orders_state_notify.notified().await; -// -// // We stop and inspect errors. -// match pipeline.shutdown_and_wait().await.err().unwrap() { -// PipelineError::OneOrMoreWorkersFailed(err) => { -// assert!(matches!( -// err.0.as_slice(), -// [ -// WorkerWaitError::WorkerPanicked(_), -// WorkerWaitError::WorkerPanicked(_) -// ] -// )); -// } -// other => panic!("Expected TableSyncWorkersFailed error, but got: {other:?}"), -// } -// } +#[tokio::test(flavor = "multi_thread")] +async fn pipeline_handles_table_sync_worker_error_during_data_sync() { + let _scenario = FailScenario::setup(); + fail::cfg(START_TABLE_SYNC_AFTER_DATA_SYNC_ERROR, "return").unwrap(); + + init_test_tracing(); + + let database = spawn_database().await; + let database_schema = setup_test_database_schema(&database, TableSelection::Both).await; + + let state_store = NotifyingStateStore::new(); + let destination = TestDestinationWrapper::wrap(MemoryDestination::new()); + + // We start the pipeline from scratch. + let pipeline_id: PipelineId = random(); + let mut pipeline = create_pipeline( + &database.config, + pipeline_id, + database_schema.publication_name(), + state_store.clone(), + destination.clone(), + ); + + // We register the interest in waiting for both table syncs to have started. + let users_state_notify = state_store + .notify_on_table_state( + database_schema.users_schema().id, + TableReplicationPhaseType::DataSync, + ) + .await; + let orders_state_notify = state_store + .notify_on_table_state( + database_schema.orders_schema().id, + TableReplicationPhaseType::DataSync, + ) + .await; + + pipeline.start().await.unwrap(); + + users_state_notify.notified().await; + orders_state_notify.notified().await; + + // We stop and inspect errors. + let err = pipeline.shutdown_and_wait().await.err().unwrap(); + assert_eq!(err.kinds().len(), 2); + assert_eq!(err.kinds()[0], ErrorKind::Unknown); + assert_eq!(err.kinds()[1], ErrorKind::Unknown); +} // TODO: inject the failure via fail-rs. #[ignore] From add5f288f434d5ac9fce40e04f20c4ab10617c76 Mon Sep 17 00:00:00 2001 From: Riccardo Busetti Date: Fri, 25 Jul 2025 16:29:11 +0200 Subject: [PATCH 13/18] Improve --- etl/Cargo.toml | 4 - etl/src/error.rs | 382 ++++++++++++++++++++------ etl/src/workers/apply.rs | 2 +- etl/src/workers/pool.rs | 2 +- etl/src/workers/table_sync.rs | 2 +- etl/tests/failpoints/pipeline_test.rs | 4 +- 6 files changed, 307 insertions(+), 89 deletions(-) diff --git a/etl/Cargo.toml b/etl/Cargo.toml index 0e78b6a05..e28ce4f26 100644 --- a/etl/Cargo.toml +++ b/etl/Cargo.toml @@ -28,7 +28,6 @@ required-features = ["failpoints", "test-utils"] config = { workspace = true } postgres = { workspace = true, features = ["tokio", "replication"] } -async-trait = { workspace = true } bigdecimal = { workspace = true, features = ["std"] } bytes = { workspace = true } byteorder = { workspace = true } @@ -41,17 +40,14 @@ gcp-bigquery-client = { workspace = true, optional = true, features = [ ] } pg_escape = { workspace = true } pin-project-lite = { workspace = true } -postgres-protocol = { workspace = true } postgres-replication = { workspace = true } prost = { workspace = true, optional = true } rustls = { workspace = true, features = ["aws-lc-rs", "logging"] } rustls-pemfile = { workspace = true, features = ["std"] } rand = { workspace = true, features = ["std"] } secrecy = { workspace = true } -serde = { workspace = true, features = ["derive"] } serde_json = { workspace = true, features = ["std"] } sqlx = { workspace = true, features = ["runtime-tokio-rustls", "postgres"] } -thiserror = { workspace = true } tokio = { workspace = true, features = [ "rt-multi-thread", "macros", diff --git a/etl/src/error.rs b/etl/src/error.rs index 21130ce1f..370440295 100644 --- a/etl/src/error.rs +++ b/etl/src/error.rs @@ -1,22 +1,62 @@ use std::error; use std::fmt; -/// Main result type for ETL operations +/// Convenient result type for ETL operations using [`ETLError`] as the error type. +/// +/// This type alias reduces boilerplate when working with fallible ETL operations. +/// Most ETL functions return this type. pub type ETLResult = Result; -/// Main error type for ETL operations, inspired by Redis error handling +/// Main error type for ETL operations, inspired by Redis error handling patterns. +/// +/// [`ETLError`] provides a comprehensive error system that can represent single errors, +/// errors with additional detail, or multiple aggregated errors. The design allows for +/// rich error information while maintaining ergonomic usage patterns. +/// +/// # Examples +/// +/// ```rust +/// use etl::error::{ETLError, ErrorKind}; +/// +/// // Simple error +/// let err = ETLError::from((ErrorKind::ConnectionFailed, "Database unreachable")); +/// +/// // Error with detail +/// let err = ETLError::from(( +/// ErrorKind::QueryFailed, +/// "SQL execution failed", +/// "Table 'users' does not exist".to_string(), +/// )); +/// +/// // Multiple errors +/// let errors = vec![ +/// ETLError::from((ErrorKind::ValidationError, "Invalid schema")), +/// ETLError::from((ErrorKind::ConversionError, "Type mismatch")), +/// ]; +/// let multi_err = ETLError::many(errors); +/// ``` pub struct ETLError { repr: ErrorRepr, } +/// Internal representation of error data. +/// +/// This enum supports different error patterns while maintaining a unified interface. +/// Users should not interact with this type directly but use [`ETLError`] methods instead. #[derive(Debug)] pub enum ErrorRepr { + /// Error with kind and static description WithDescription(ErrorKind, &'static str), + /// Error with kind, static description, and dynamic detail WithDescriptionAndDetail(ErrorKind, &'static str, String), + /// Multiple aggregated errors Many(Vec), } -/// Comprehensive error kinds for ETL operations +/// Specific categories of errors that can occur during ETL operations. +/// +/// This enum provides granular error classification to enable appropriate error handling +/// strategies. Error kinds are organized by functional area and failure mode. #[derive(PartialEq, Eq, Copy, Clone, Debug)] #[non_exhaustive] pub enum ErrorKind { @@ -47,9 +87,11 @@ pub enum ErrorKind { /// Data validation error ValidationError, /// Apply worker error - ApplyWorkerError, + ApplyWorkerPanic, /// Table sync worker error - TableSyncWorkerError, + TableSyncWorkerPanic, + /// Table sync worker error + TableSyncWorkerCaughtError, /// Destination-specific error DestinationError, /// Replication slot not found @@ -69,14 +111,20 @@ pub enum ErrorKind { } impl ETLError { - /// Creates a new ETLError that contains multiple errors + /// Creates an [`ETLError`] containing multiple aggregated errors. + /// + /// This is useful when multiple operations fail and you want to report all failures + /// rather than just the first one. pub fn many(errors: Vec) -> ETLError { ETLError { repr: ErrorRepr::Many(errors), } } - /// Returns the kind of the error + /// Returns the [`ErrorKind`] of this error. + /// + /// For multiple errors, returns the kind of the first error or [`ErrorKind::Unknown`] + /// if the error list is empty. pub fn kind(&self) -> ErrorKind { match self.repr { ErrorRepr::WithDescription(kind, _) @@ -88,20 +136,25 @@ impl ETLError { } } - /// Returns the kinds of the error + /// Returns all [`ErrorKind`]s present in this error. + /// + /// For single errors, returns a vector with one element. For multiple errors, + /// returns a flattened vector of all error kinds. pub fn kinds(&self) -> Vec { match self.repr { ErrorRepr::WithDescription(kind, _) | ErrorRepr::WithDescriptionAndDetail(kind, _, _) => vec![kind], ErrorRepr::Many(ref errors) => errors .iter() - .map(|err| err.kinds()) - .flatten() + .flat_map(|err| err.kinds()) .collect::>(), } } - /// Returns the error detail if available + /// Returns the detailed error information if available. + /// + /// For multiple errors, returns the detail of the first error that has one. + /// Returns [`None`] if no detailed information is available. pub fn detail(&self) -> Option<&str> { match self.repr { ErrorRepr::WithDescriptionAndDetail(_, _, ref detail) => Some(detail.as_str()), @@ -195,6 +248,8 @@ impl error::Error for ETLError { } // Ergonomic constructors following Redis pattern + +/// Creates an [`ETLError`] from an error kind and static description. impl From<(ErrorKind, &'static str)> for ETLError { fn from((kind, desc): (ErrorKind, &'static str)) -> ETLError { ETLError { @@ -203,6 +258,7 @@ impl From<(ErrorKind, &'static str)> for ETLError { } } +/// Creates an [`ETLError`] from an error kind, static description, and dynamic detail. impl From<(ErrorKind, &'static str, String)> for ETLError { fn from((kind, desc, detail): (ErrorKind, &'static str, String)) -> ETLError { ETLError { @@ -211,6 +267,7 @@ impl From<(ErrorKind, &'static str, String)> for ETLError { } } +/// Creates an [`ETLError`] from a vector of errors for aggregation. impl From> for ETLError { fn from(errors: Vec) -> ETLError { ETLError { @@ -220,6 +277,8 @@ impl From> for ETLError { } // Common standard library error conversions + +/// Converts [`std::io::Error`] to [`ETLError`] with [`ErrorKind::NetworkError`]. impl From for ETLError { fn from(err: std::io::Error) -> ETLError { ETLError { @@ -232,6 +291,7 @@ impl From for ETLError { } } +/// Converts [`serde_json::Error`] to [`ETLError`] with [`ErrorKind::SerializationError`]. impl From for ETLError { fn from(err: serde_json::Error) -> ETLError { ETLError { @@ -244,6 +304,7 @@ impl From for ETLError { } } +/// Converts [`std::str::Utf8Error`] to [`ETLError`] with [`ErrorKind::ConversionError`]. impl From for ETLError { fn from(err: std::str::Utf8Error) -> ETLError { ETLError { @@ -256,6 +317,7 @@ impl From for ETLError { } } +/// Converts [`std::string::FromUtf8Error`] to [`ETLError`] with [`ErrorKind::ConversionError`]. impl From for ETLError { fn from(err: std::string::FromUtf8Error) -> ETLError { ETLError { @@ -268,6 +330,7 @@ impl From for ETLError { } } +/// Converts [`std::num::ParseIntError`] to [`ETLError`] with [`ErrorKind::ConversionError`]. impl From for ETLError { fn from(err: std::num::ParseIntError) -> ETLError { ETLError { @@ -280,6 +343,7 @@ impl From for ETLError { } } +/// Converts [`std::num::ParseFloatError`] to [`ETLError`] with [`ErrorKind::ConversionError`]. impl From for ETLError { fn from(err: std::num::ParseFloatError) -> ETLError { ETLError { @@ -293,6 +357,11 @@ impl From for ETLError { } // PostgreSQL-specific error conversions + +/// Converts [`tokio_postgres::Error`] to [`ETLError`]. +/// +/// Maps to [`ErrorKind::QueryFailed`] if an error code is present, +/// otherwise maps to [`ErrorKind::ConnectionFailed`]. impl From for ETLError { fn from(err: tokio_postgres::Error) -> ETLError { let kind = if err.code().is_some() { @@ -311,6 +380,7 @@ impl From for ETLError { } } +/// Converts [`rustls::Error`] to [`ETLError`] with [`ErrorKind::EncryptionError`]. impl From for ETLError { fn from(err: rustls::Error) -> ETLError { ETLError { @@ -323,6 +393,7 @@ impl From for ETLError { } } +/// Converts [`uuid::Error`] to [`ETLError`] with [`ErrorKind::InvalidData`]. impl From for ETLError { fn from(err: uuid::Error) -> ETLError { ETLError { @@ -335,6 +406,7 @@ impl From for ETLError { } } +/// Converts [`chrono::ParseError`] to [`ETLError`] with [`ErrorKind::ConversionError`]. impl From for ETLError { fn from(err: chrono::ParseError) -> ETLError { ETLError { @@ -347,6 +419,7 @@ impl From for ETLError { } } +/// Converts [`bigdecimal::ParseBigDecimalError`] to [`ETLError`] with [`ErrorKind::ConversionError`]. impl From for ETLError { fn from(err: bigdecimal::ParseBigDecimalError) -> ETLError { ETLError { @@ -360,6 +433,11 @@ impl From for ETLError { } // SQLx error conversion + +/// Converts [`sqlx::Error`] to [`ETLError`] with appropriate error kind. +/// +/// Maps database errors to [`ErrorKind::QueryFailed`], I/O errors to [`ErrorKind::NetworkError`], +/// and connection pool errors to [`ErrorKind::ConnectionFailed`]. impl From for ETLError { fn from(err: sqlx::Error) -> ETLError { let kind = match &err { @@ -380,6 +458,11 @@ impl From for ETLError { } // BigQuery error conversions (feature-gated) + +/// Converts [`gcp_bigquery_client::error::BQError`] to [`ETLError`] with appropriate error kind. +/// +/// Maps request errors to [`ErrorKind::NetworkError`], response errors to [`ErrorKind::QueryFailed`], +/// and other errors to [`ErrorKind::DestinationError`]. #[cfg(feature = "bigquery")] impl From for ETLError { fn from(err: gcp_bigquery_client::error::BQError) -> ETLError { @@ -399,6 +482,7 @@ impl From for ETLError { } } +/// Converts BigQuery row errors to [`ETLError`] with [`ErrorKind::DestinationError`]. #[cfg(feature = "bigquery")] impl From for ETLError { fn from(err: crate::clients::bigquery::RowErrors) -> ETLError { @@ -412,73 +496,209 @@ impl From for ETLError { } } -// #[cfg(test)] -// mod tests { -// use super::*; -// -// #[test] -// fn test_error_creation() { -// let err = ETLError::from((ErrorKind::ConnectionFailed, "Database connection failed")); -// assert_eq!(err.kind(), ErrorKind::ConnectionFailed); -// assert_eq!(err.category(), "connection failed"); -// assert!(err.is_connection_error()); -// } -// -// #[test] -// fn test_error_with_detail() { -// let err = ETLError::from(( -// ErrorKind::QueryFailed, -// "SQL query execution failed", -// "Table 'users' doesn't exist".to_string(), -// )); -// assert_eq!(err.kind(), ErrorKind::QueryFailed); -// assert_eq!(err.detail(), Some("Table 'users' doesn't exist")); -// } -// -// #[test] -// fn test_from_io_error() { -// let io_err = std::io::Error::new(std::io::ErrorKind::PermissionDenied, "Access denied"); -// let etl_err = ETLError::from(io_err); -// assert_eq!(etl_err.kind(), ErrorKind::NetworkError); -// assert!(etl_err.detail().unwrap().contains("Access denied")); -// } -// -// #[test] -// fn test_macro_usage() { -// let err = etl_error!(ErrorKind::ValidationError, "Invalid data format"); -// assert_eq!(err.kind(), ErrorKind::ValidationError); -// -// let err_with_detail = etl_error!( -// ErrorKind::ConversionError, -// "Type conversion failed", -// "Cannot convert string to integer: 'abc'" -// ); -// assert_eq!(err_with_detail.kind(), ErrorKind::ConversionError); -// assert!(err_with_detail.detail().unwrap().contains("Cannot convert")); -// } -// -// #[test] -// fn test_error_categories() { -// let connection_err = ETLError::from((ErrorKind::ConnectionFailed, "Connection failed")); -// let data_err = ETLError::from((ErrorKind::SchemaError, "Schema mismatch")); -// let replication_err = -// ETLError::from((ErrorKind::ReplicationSlotNotFound, "Slot not found")); -// let slot_err = ETLError::from((ErrorKind::ReplicationSlotAlreadyExists, "Slot exists")); -// -// assert!(connection_err.is_connection_error()); -// assert!(!connection_err.is_data_error()); -// assert!(!connection_err.is_replication_error()); -// -// assert!(!data_err.is_connection_error()); -// assert!(data_err.is_data_error()); -// assert!(!data_err.is_replication_error()); -// -// assert!(replication_err.is_replication_error()); -// assert!(replication_err.is_replication_slot_error()); -// assert!(!replication_err.is_connection_error()); -// -// assert!(slot_err.is_replication_error()); -// assert!(slot_err.is_replication_slot_error()); -// assert!(!slot_err.is_data_error()); -// } -// } +#[cfg(test)] +mod tests { + use super::*; + use crate::{etl_error, bail}; + + #[test] + fn test_simple_error_creation() { + let err = ETLError::from((ErrorKind::ConnectionFailed, "Database connection failed")); + assert_eq!(err.kind(), ErrorKind::ConnectionFailed); + assert_eq!(err.detail(), None); + assert_eq!(err.kinds(), vec![ErrorKind::ConnectionFailed]); + } + + #[test] + fn test_error_with_detail() { + let err = ETLError::from(( + ErrorKind::QueryFailed, + "SQL query execution failed", + "Table 'users' doesn't exist".to_string(), + )); + assert_eq!(err.kind(), ErrorKind::QueryFailed); + assert_eq!(err.detail(), Some("Table 'users' doesn't exist")); + assert_eq!(err.kinds(), vec![ErrorKind::QueryFailed]); + } + + #[test] + fn test_multiple_errors() { + let errors = vec![ + ETLError::from((ErrorKind::ValidationError, "Invalid schema")), + ETLError::from((ErrorKind::ConversionError, "Type mismatch")), + ETLError::from((ErrorKind::NetworkError, "Connection timeout")), + ]; + let multi_err = ETLError::many(errors); + + assert_eq!(multi_err.kind(), ErrorKind::ValidationError); + assert_eq!(multi_err.kinds(), vec![ + ErrorKind::ValidationError, + ErrorKind::ConversionError, + ErrorKind::NetworkError + ]); + assert_eq!(multi_err.detail(), None); + } + + #[test] + fn test_multiple_errors_with_detail() { + let errors = vec![ + ETLError::from((ErrorKind::ValidationError, "Invalid schema", "Missing required field".to_string())), + ETLError::from((ErrorKind::ConversionError, "Type mismatch")), + ]; + let multi_err = ETLError::many(errors); + + assert_eq!(multi_err.detail(), Some("Missing required field")); + } + + #[test] + fn test_from_vector() { + let errors = vec![ + ETLError::from((ErrorKind::ValidationError, "Error 1")), + ETLError::from((ErrorKind::ConversionError, "Error 2")), + ]; + let multi_err = ETLError::from(errors); + assert_eq!(multi_err.kinds().len(), 2); + } + + #[test] + fn test_empty_multiple_errors() { + let multi_err = ETLError::many(vec![]); + assert_eq!(multi_err.kind(), ErrorKind::Unknown); + assert_eq!(multi_err.kinds(), vec![]); + assert_eq!(multi_err.detail(), None); + } + + #[test] + fn test_from_io_error() { + let io_err = std::io::Error::new(std::io::ErrorKind::PermissionDenied, "Access denied"); + let etl_err = ETLError::from(io_err); + assert_eq!(etl_err.kind(), ErrorKind::NetworkError); + assert!(etl_err.detail().unwrap().contains("Access denied")); + } + + #[test] + fn test_from_json_error() { + let json_err = serde_json::from_str::("invalid json").unwrap_err(); + let etl_err = ETLError::from(json_err); + assert_eq!(etl_err.kind(), ErrorKind::SerializationError); + assert!(etl_err.detail().is_some()); + } + + #[test] + fn test_from_parse_int_error() { + let parse_err = "not_a_number".parse::().unwrap_err(); + let etl_err = ETLError::from(parse_err); + assert_eq!(etl_err.kind(), ErrorKind::ConversionError); + assert!(etl_err.detail().is_some()); + } + + #[test] + fn test_from_utf8_error() { + let invalid_utf8 = vec![0, 159, 146, 150]; + let utf8_err = std::str::from_utf8(&invalid_utf8).unwrap_err(); + let etl_err = ETLError::from(utf8_err); + assert_eq!(etl_err.kind(), ErrorKind::ConversionError); + assert!(etl_err.detail().is_some()); + } + + #[test] + fn test_error_equality() { + let err1 = ETLError::from((ErrorKind::ConnectionFailed, "Connection failed")); + let err2 = ETLError::from((ErrorKind::ConnectionFailed, "Connection failed")); + let err3 = ETLError::from((ErrorKind::QueryFailed, "Query failed")); + + assert_eq!(err1, err2); + assert_ne!(err1, err3); + } + + #[test] + fn test_error_display() { + let err = ETLError::from((ErrorKind::ConnectionFailed, "Database connection failed")); + let display_str = format!("{err}"); + assert!(display_str.contains("ConnectionFailed")); + assert!(display_str.contains("Database connection failed")); + } + + #[test] + fn test_error_display_with_detail() { + let err = ETLError::from(( + ErrorKind::QueryFailed, + "SQL query failed", + "Invalid table name".to_string(), + )); + let display_str = format!("{err}"); + assert!(display_str.contains("QueryFailed")); + assert!(display_str.contains("SQL query failed")); + assert!(display_str.contains("Invalid table name")); + } + + #[test] + fn test_multiple_errors_display() { + let errors = vec![ + ETLError::from((ErrorKind::ValidationError, "Invalid schema")), + ETLError::from((ErrorKind::ConversionError, "Type mismatch")), + ]; + let multi_err = ETLError::many(errors); + let display_str = format!("{multi_err}"); + assert!(display_str.contains("Multiple errors")); + assert!(display_str.contains("2 total")); + } + + #[test] + fn test_macro_usage() { + let err = etl_error!(ErrorKind::ValidationError, "Invalid data format"); + assert_eq!(err.kind(), ErrorKind::ValidationError); + assert_eq!(err.detail(), None); + + let err_with_detail = etl_error!( + ErrorKind::ConversionError, + "Type conversion failed", + "Cannot convert string to integer: 'abc'" + ); + assert_eq!(err_with_detail.kind(), ErrorKind::ConversionError); + assert!(err_with_detail.detail().unwrap().contains("Cannot convert")); + } + + #[test] + fn test_bail_macro() { + fn test_function() -> ETLResult { + bail!(ErrorKind::ValidationError, "Test error"); + } + + fn test_function_with_detail() -> ETLResult { + bail!(ErrorKind::ConversionError, "Test error", "Additional detail"); + } + + let result = test_function(); + assert!(result.is_err()); + let err = result.unwrap_err(); + assert_eq!(err.kind(), ErrorKind::ValidationError); + + let result = test_function_with_detail(); + assert!(result.is_err()); + let err = result.unwrap_err(); + assert_eq!(err.kind(), ErrorKind::ConversionError); + assert!(err.detail().unwrap().contains("Additional detail")); + } + + #[test] + fn test_nested_multiple_errors() { + let inner_errors = vec![ + ETLError::from((ErrorKind::ConversionError, "Inner error 1")), + ETLError::from((ErrorKind::ValidationError, "Inner error 2")), + ]; + let inner_multi = ETLError::many(inner_errors); + + let outer_errors = vec![ + inner_multi, + ETLError::from((ErrorKind::NetworkError, "Outer error")), + ]; + let outer_multi = ETLError::many(outer_errors); + + let kinds = outer_multi.kinds(); + assert_eq!(kinds.len(), 3); + assert!(kinds.contains(&ErrorKind::ConversionError)); + assert!(kinds.contains(&ErrorKind::ValidationError)); + assert!(kinds.contains(&ErrorKind::NetworkError)); + } +} diff --git a/etl/src/workers/apply.rs b/etl/src/workers/apply.rs index afe672cf6..bbeaeeb44 100644 --- a/etl/src/workers/apply.rs +++ b/etl/src/workers/apply.rs @@ -38,7 +38,7 @@ impl WorkerHandle<()> for ApplyWorkerHandle { handle.await.map_err(|err| { etl_error!( - ErrorKind::ApplyWorkerError, + ErrorKind::ApplyWorkerPanic, "A panic occurred in the apply worker", err ) diff --git a/etl/src/workers/pool.rs b/etl/src/workers/pool.rs index 18374daf8..ddf0f7b03 100644 --- a/etl/src/workers/pool.rs +++ b/etl/src/workers/pool.rs @@ -123,7 +123,7 @@ impl TableSyncWorkerPoolInner { // re-propagate the error after marking a table sync worker as finished. if let TableSyncWorkerInactiveReason::Errored(err) = finish { errors.push(etl_error!( - ErrorKind::TableSyncWorkerError, + ErrorKind::TableSyncWorkerCaughtError, "An error occurred in a table sync worker but was not propagated", err )); diff --git a/etl/src/workers/table_sync.rs b/etl/src/workers/table_sync.rs index e0c2f75ef..7185f85f5 100644 --- a/etl/src/workers/table_sync.rs +++ b/etl/src/workers/table_sync.rs @@ -204,7 +204,7 @@ impl WorkerHandle for TableSyncWorkerHandle { handle.await.map_err(|err| { etl_error!( - ErrorKind::TableSyncWorkerError, + ErrorKind::TableSyncWorkerPanic, "A panic occurred in the table sync worker", err ) diff --git a/etl/tests/failpoints/pipeline_test.rs b/etl/tests/failpoints/pipeline_test.rs index 24b0e982e..acb52a175 100644 --- a/etl/tests/failpoints/pipeline_test.rs +++ b/etl/tests/failpoints/pipeline_test.rs @@ -64,7 +64,9 @@ async fn pipeline_handles_table_sync_worker_panic_during_data_sync() { // We stop and inspect errors. let err = pipeline.shutdown_and_wait().await.err().unwrap(); - println!("pipeline shutdown err {:?}", err); + assert_eq!(err.kinds().len(), 2); + assert_eq!(err.kinds()[0], ErrorKind::TableSyncWorkerPanic); + assert_eq!(err.kinds()[1], ErrorKind::TableSyncWorkerPanic); } #[tokio::test(flavor = "multi_thread")] From c6515b4270f0d9c524ddae5d73fc9aee6c915fc8 Mon Sep 17 00:00:00 2001 From: Riccardo Busetti Date: Fri, 25 Jul 2025 16:29:23 +0200 Subject: [PATCH 14/18] Improve --- etl/src/error.rs | 37 ++++++++++++++++++++++++------------- 1 file changed, 24 insertions(+), 13 deletions(-) diff --git a/etl/src/error.rs b/etl/src/error.rs index 370440295..9b8626396 100644 --- a/etl/src/error.rs +++ b/etl/src/error.rs @@ -499,7 +499,7 @@ impl From for ETLError { #[cfg(test)] mod tests { use super::*; - use crate::{etl_error, bail}; + use crate::{bail, etl_error}; #[test] fn test_simple_error_creation() { @@ -529,24 +529,31 @@ mod tests { ETLError::from((ErrorKind::NetworkError, "Connection timeout")), ]; let multi_err = ETLError::many(errors); - + assert_eq!(multi_err.kind(), ErrorKind::ValidationError); - assert_eq!(multi_err.kinds(), vec![ - ErrorKind::ValidationError, - ErrorKind::ConversionError, - ErrorKind::NetworkError - ]); + assert_eq!( + multi_err.kinds(), + vec![ + ErrorKind::ValidationError, + ErrorKind::ConversionError, + ErrorKind::NetworkError + ] + ); assert_eq!(multi_err.detail(), None); } #[test] fn test_multiple_errors_with_detail() { let errors = vec![ - ETLError::from((ErrorKind::ValidationError, "Invalid schema", "Missing required field".to_string())), + ETLError::from(( + ErrorKind::ValidationError, + "Invalid schema", + "Missing required field".to_string(), + )), ETLError::from((ErrorKind::ConversionError, "Type mismatch")), ]; let multi_err = ETLError::many(errors); - + assert_eq!(multi_err.detail(), Some("Missing required field")); } @@ -606,7 +613,7 @@ mod tests { let err1 = ETLError::from((ErrorKind::ConnectionFailed, "Connection failed")); let err2 = ETLError::from((ErrorKind::ConnectionFailed, "Connection failed")); let err3 = ETLError::from((ErrorKind::QueryFailed, "Query failed")); - + assert_eq!(err1, err2); assert_ne!(err1, err3); } @@ -666,7 +673,11 @@ mod tests { } fn test_function_with_detail() -> ETLResult { - bail!(ErrorKind::ConversionError, "Test error", "Additional detail"); + bail!( + ErrorKind::ConversionError, + "Test error", + "Additional detail" + ); } let result = test_function(); @@ -688,13 +699,13 @@ mod tests { ETLError::from((ErrorKind::ValidationError, "Inner error 2")), ]; let inner_multi = ETLError::many(inner_errors); - + let outer_errors = vec![ inner_multi, ETLError::from((ErrorKind::NetworkError, "Outer error")), ]; let outer_multi = ETLError::many(outer_errors); - + let kinds = outer_multi.kinds(); assert_eq!(kinds.len(), 3); assert!(kinds.contains(&ErrorKind::ConversionError)); From da93ca9fef6e38963d4fab79761fac6408843560 Mon Sep 17 00:00:00 2001 From: Riccardo Busetti Date: Fri, 25 Jul 2025 17:05:57 +0200 Subject: [PATCH 15/18] Improve --- etl/src/clients/bigquery.rs | 21 +- etl/src/conversions/event.rs | 2 +- etl/src/error.rs | 403 +++++++++++++++++++++++++---------- 3 files changed, 296 insertions(+), 130 deletions(-) diff --git a/etl/src/clients/bigquery.rs b/etl/src/clients/bigquery.rs index 7f06bf8b6..34583b792 100644 --- a/etl/src/clients/bigquery.rs +++ b/etl/src/clients/bigquery.rs @@ -1,6 +1,5 @@ -use crate::error::{ETLError, ETLResult}; +use crate::error::ETLResult; use futures::StreamExt; -use gcp_bigquery_client::google::cloud::bigquery::storage::v1::RowError; use gcp_bigquery_client::storage::{ColumnMode, StorageApi}; use gcp_bigquery_client::yup_oauth2::parse_service_account_key; use gcp_bigquery_client::{ @@ -52,22 +51,6 @@ impl fmt::Display for BigQueryOperationType { } } -/// Collection of row errors returned from BigQuery streaming operations. -#[derive(Debug)] -pub struct RowErrors(pub Vec); - -impl fmt::Display for RowErrors { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - if !self.0.is_empty() { - for row_error in self.0.iter() { - writeln!(f, "{row_error:?}")?; - } - } - - Ok(()) - } -} - /// A client for interacting with Google BigQuery. /// /// This client provides methods for managing tables, inserting data, @@ -221,7 +204,7 @@ impl BigQueryClient { if let Some(append_rows_response) = append_rows_stream.next().await { let append_rows_response = append_rows_response.map_err(BQError::from)?; if !append_rows_response.row_errors.is_empty() { - return Err(ETLError::from(RowErrors(append_rows_response.row_errors))); + return Err(append_rows_response.row_errors.into()); } } diff --git a/etl/src/conversions/event.rs b/etl/src/conversions/event.rs index 00ab0f3d7..c9acbff9b 100644 --- a/etl/src/conversions/event.rs +++ b/etl/src/conversions/event.rs @@ -265,7 +265,7 @@ fn convert_tuple_to_row( // values. let Some(tuple_data) = &tuple_data.get(i) else { bail!( - ErrorKind::ConnectionFailed, + ErrorKind::ConversionError, "Tuple data does not contain data at the specified index" ); }; diff --git a/etl/src/error.rs b/etl/src/error.rs index 9b8626396..805adc8f0 100644 --- a/etl/src/error.rs +++ b/etl/src/error.rs @@ -7,34 +7,11 @@ use std::fmt; /// Most ETL functions return this type. pub type ETLResult = Result; -/// Main error type for ETL operations, inspired by Redis error handling patterns. +/// Main error type for ETL operations. /// /// [`ETLError`] provides a comprehensive error system that can represent single errors, /// errors with additional detail, or multiple aggregated errors. The design allows for /// rich error information while maintaining ergonomic usage patterns. -/// -/// # Examples -/// -/// ```rust -/// use etl::error::{ETLError, ErrorKind}; -/// -/// // Simple error -/// let err = ETLError::from((ErrorKind::ConnectionFailed, "Database unreachable")); -/// -/// // Error with detail -/// let err = ETLError::from(( -/// ErrorKind::QueryFailed, -/// "SQL execution failed", -/// "Table 'users' does not exist".to_string(), -/// )); -/// -/// // Multiple errors -/// let errors = vec![ -/// ETLError::from((ErrorKind::ValidationError, "Invalid schema")), -/// ETLError::from((ErrorKind::ConversionError, "Type mismatch")), -/// ]; -/// let multi_err = ETLError::many(errors); -/// ``` pub struct ETLError { repr: ErrorRepr, } @@ -60,31 +37,68 @@ pub enum ErrorRepr { #[derive(PartialEq, Eq, Copy, Clone, Debug)] #[non_exhaustive] pub enum ErrorKind { - /// Database connection failed + /// Database connection failed or resource limitations + /// + /// Used for PostgreSQL connection errors (08xxx), resource errors (53xxx), + /// and general connectivity issues. ConnectionFailed, /// Query execution failed + /// + /// Used for PostgreSQL syntax errors (42xxx), BigQuery response errors, + /// and general SQL execution failures. QueryFailed, /// Source schema mismatch or validation error SourceSchemaError, /// Destination schema mismatch or validation error + /// + /// Used for PostgreSQL schema object not found errors (42xxx) + /// and destination schema mismatches. DestinationSchemaError, /// Missing table schema MissingTableSchema, /// Data type conversion error + /// + /// Used for PostgreSQL data conversion errors (22xxx), BigQuery column type mismatches, + /// UTF-8 conversion failures, and numeric parsing errors. ConversionError, /// Configuration error + /// + /// Used for BigQuery invalid metadata values and general configuration issues. ConfigError, /// Network or I/O error - NetworkError, - /// Serialization/deserialization error + /// + /// Used for PostgreSQL system errors (58xxx), BigQuery transport/request errors, + /// JSON I/O errors, and general I/O failures. + IoError, + /// Serialization error + /// + /// Used for BigQuery JSON serialization errors and data encoding failures. SerializationError, + /// Deserialization error + /// + /// Used for JSON syntax/data/EOF errors and data decoding failures. + DeserializationError, /// Encryption/decryption error EncryptionError, + /// Authentication failed + /// + /// Used for PostgreSQL authentication errors (28xxx), + /// BigQuery authentication errors, and credential failures. + AuthenticationError, /// Invalid state error + /// + /// Used for PostgreSQL transaction errors (40xxx, 25xxx), + /// BigQuery result set positioning errors, and state inconsistencies. InvalidState, /// Invalid data + /// + /// Used for BigQuery invalid column index/name errors, + /// UUID parsing failures, and malformed data. InvalidData, /// Data validation error + /// + /// Used for PostgreSQL constraint violations (23xxx) + /// and data integrity validation failures. ValidationError, /// Apply worker error ApplyWorkerPanic, @@ -93,6 +107,9 @@ pub enum ErrorKind { /// Table sync worker error TableSyncWorkerCaughtError, /// Destination-specific error + /// + /// Used for BigQuery gRPC status errors, row errors, + /// and destination-specific failures. DestinationError, /// Replication slot not found ReplicationSlotNotFound, @@ -100,12 +117,6 @@ pub enum ErrorKind { ReplicationSlotAlreadyExists, /// Replication slot could not be created ReplicationSlotNotCreated, - /// Replication slot name is invalid or too long - ReplicationSlotInvalid, - /// Table synchronization failed - TableSyncFailed, - /// Logical replication stream error - LogicalReplicationFailed, /// Unknown error Unknown, } @@ -268,22 +279,25 @@ impl From<(ErrorKind, &'static str, String)> for ETLError { } /// Creates an [`ETLError`] from a vector of errors for aggregation. -impl From> for ETLError { - fn from(errors: Vec) -> ETLError { +impl From> for ETLError +where + E: Into, +{ + fn from(errors: Vec) -> ETLError { ETLError { - repr: ErrorRepr::Many(errors), + repr: ErrorRepr::Many(errors.into_iter().map(Into::into).collect()), } } } // Common standard library error conversions -/// Converts [`std::io::Error`] to [`ETLError`] with [`ErrorKind::NetworkError`]. +/// Converts [`std::io::Error`] to [`ETLError`] with [`ErrorKind::IoError`]. impl From for ETLError { fn from(err: std::io::Error) -> ETLError { ETLError { repr: ErrorRepr::WithDescriptionAndDetail( - ErrorKind::NetworkError, + ErrorKind::IoError, "I/O error occurred", err.to_string(), ), @@ -291,15 +305,26 @@ impl From for ETLError { } } -/// Converts [`serde_json::Error`] to [`ETLError`] with [`ErrorKind::SerializationError`]. +/// Converts [`serde_json::Error`] to [`ETLError`] with appropriate error kind. +/// +/// Maps to [`ErrorKind::SerializationError`] for serialization failures and +/// [`ErrorKind::DeserializationError`] for deserialization failures based on error classification. impl From for ETLError { fn from(err: serde_json::Error) -> ETLError { - ETLError { - repr: ErrorRepr::WithDescriptionAndDetail( - ErrorKind::SerializationError, - "JSON serialization failed", - err.to_string(), + let (kind, description) = match err.classify() { + serde_json::error::Category::Io => (ErrorKind::IoError, "JSON I/O operation failed"), + serde_json::error::Category::Syntax | serde_json::error::Category::Data => ( + ErrorKind::DeserializationError, + "JSON deserialization failed", ), + serde_json::error::Category::Eof => ( + ErrorKind::DeserializationError, + "JSON deserialization failed", + ), + }; + + ETLError { + repr: ErrorRepr::WithDescriptionAndDetail(kind, description, err.to_string()), } } } @@ -358,24 +383,99 @@ impl From for ETLError { // PostgreSQL-specific error conversions -/// Converts [`tokio_postgres::Error`] to [`ETLError`]. +/// Converts [`tokio_postgres::Error`] to [`ETLError`] with appropriate error kind. /// -/// Maps to [`ErrorKind::QueryFailed`] if an error code is present, -/// otherwise maps to [`ErrorKind::ConnectionFailed`]. +/// Maps errors based on PostgreSQL SQLSTATE codes to provide granular error classification +/// for better error handling in ETL operations. impl From for ETLError { fn from(err: tokio_postgres::Error) -> ETLError { - let kind = if err.code().is_some() { - ErrorKind::QueryFailed - } else { - ErrorKind::ConnectionFailed + let (kind, description) = match err.code() { + Some(sqlstate) => { + use tokio_postgres::error::SqlState; + + match *sqlstate { + // Connection errors (08xxx) + SqlState::CONNECTION_EXCEPTION + | SqlState::CONNECTION_DOES_NOT_EXIST + | SqlState::CONNECTION_FAILURE + | SqlState::SQLCLIENT_UNABLE_TO_ESTABLISH_SQLCONNECTION + | SqlState::SQLSERVER_REJECTED_ESTABLISHMENT_OF_SQLCONNECTION => { + (ErrorKind::ConnectionFailed, "PostgreSQL connection error") + } + + // Authentication errors (28xxx) + SqlState::INVALID_AUTHORIZATION_SPECIFICATION | SqlState::INVALID_PASSWORD => ( + ErrorKind::AuthenticationError, + "PostgreSQL authentication failed", + ), + + // Data integrity violations (23xxx) + SqlState::INTEGRITY_CONSTRAINT_VIOLATION + | SqlState::NOT_NULL_VIOLATION + | SqlState::FOREIGN_KEY_VIOLATION + | SqlState::UNIQUE_VIOLATION + | SqlState::CHECK_VIOLATION => ( + ErrorKind::ValidationError, + "PostgreSQL constraint violation", + ), + + // Data conversion errors (22xxx) + SqlState::DATA_EXCEPTION + | SqlState::INVALID_TEXT_REPRESENTATION + | SqlState::INVALID_DATETIME_FORMAT + | SqlState::NUMERIC_VALUE_OUT_OF_RANGE + | SqlState::DIVISION_BY_ZERO => ( + ErrorKind::ConversionError, + "PostgreSQL data conversion error", + ), + + // Schema/object not found errors (42xxx) + SqlState::UNDEFINED_TABLE + | SqlState::UNDEFINED_COLUMN + | SqlState::UNDEFINED_FUNCTION + | SqlState::UNDEFINED_SCHEMA => ( + ErrorKind::DestinationSchemaError, + "PostgreSQL schema object not found", + ), + + // Syntax and access errors (42xxx) + SqlState::SYNTAX_ERROR + | SqlState::SYNTAX_ERROR_OR_ACCESS_RULE_VIOLATION + | SqlState::INSUFFICIENT_PRIVILEGE => { + (ErrorKind::QueryFailed, "PostgreSQL syntax or access error") + } + + // Resource errors (53xxx) + SqlState::INSUFFICIENT_RESOURCES + | SqlState::OUT_OF_MEMORY + | SqlState::TOO_MANY_CONNECTIONS => ( + ErrorKind::ConnectionFailed, + "PostgreSQL resource limitation", + ), + + // Transaction errors (40xxx, 25xxx) + SqlState::TRANSACTION_ROLLBACK + | SqlState::T_R_SERIALIZATION_FAILURE + | SqlState::T_R_DEADLOCK_DETECTED + | SqlState::INVALID_TRANSACTION_STATE => { + (ErrorKind::InvalidState, "PostgreSQL transaction error") + } + + // System errors (58xxx, XX xxx) + SqlState::SYSTEM_ERROR | SqlState::IO_ERROR | SqlState::INTERNAL_ERROR => { + (ErrorKind::IoError, "PostgreSQL system error") + } + + // Default for other SQL states + _ => (ErrorKind::QueryFailed, "PostgreSQL query failed"), + } + } + // No SQL state means connection issue + None => (ErrorKind::ConnectionFailed, "PostgreSQL connection failed"), }; ETLError { - repr: ErrorRepr::WithDescriptionAndDetail( - kind, - "PostgreSQL client operation failed", - err.to_string(), - ), + repr: ErrorRepr::WithDescriptionAndDetail(kind, description, err.to_string()), } } } @@ -436,13 +536,13 @@ impl From for ETLError { /// Converts [`sqlx::Error`] to [`ETLError`] with appropriate error kind. /// -/// Maps database errors to [`ErrorKind::QueryFailed`], I/O errors to [`ErrorKind::NetworkError`], +/// Maps database errors to [`ErrorKind::QueryFailed`], I/O errors to [`ErrorKind::IoError`], /// and connection pool errors to [`ErrorKind::ConnectionFailed`]. impl From for ETLError { fn from(err: sqlx::Error) -> ETLError { let kind = match &err { sqlx::Error::Database(_) => ErrorKind::QueryFailed, - sqlx::Error::Io(_) => ErrorKind::NetworkError, + sqlx::Error::Io(_) => ErrorKind::IoError, sqlx::Error::PoolClosed | sqlx::Error::PoolTimedOut => ErrorKind::ConnectionFailed, _ => ErrorKind::QueryFailed, }; @@ -461,36 +561,97 @@ impl From for ETLError { /// Converts [`gcp_bigquery_client::error::BQError`] to [`ETLError`] with appropriate error kind. /// -/// Maps request errors to [`ErrorKind::NetworkError`], response errors to [`ErrorKind::QueryFailed`], -/// and other errors to [`ErrorKind::DestinationError`]. +/// Maps errors based on their specific type for better error classification and handling. #[cfg(feature = "bigquery")] impl From for ETLError { fn from(err: gcp_bigquery_client::error::BQError) -> ETLError { - let kind = match &err { - gcp_bigquery_client::error::BQError::RequestError(_) => ErrorKind::NetworkError, - gcp_bigquery_client::error::BQError::ResponseError { .. } => ErrorKind::QueryFailed, - _ => ErrorKind::DestinationError, + use gcp_bigquery_client::error::BQError; + + let (kind, description) = match &err { + // Authentication related errors + BQError::InvalidServiceAccountKey(_) => ( + ErrorKind::AuthenticationError, + "Invalid BigQuery service account key", + ), + BQError::InvalidServiceAccountAuthenticator(_) => ( + ErrorKind::AuthenticationError, + "Invalid BigQuery service account authenticator", + ), + BQError::InvalidInstalledFlowAuthenticator(_) => ( + ErrorKind::AuthenticationError, + "Invalid BigQuery installed flow authenticator", + ), + BQError::InvalidApplicationDefaultCredentialsAuthenticator(_) => ( + ErrorKind::AuthenticationError, + "Invalid BigQuery application default credentials", + ), + BQError::InvalidAuthorizedUserAuthenticator(_) => ( + ErrorKind::AuthenticationError, + "Invalid BigQuery authorized user authenticator", + ), + BQError::AuthError(_) => ( + ErrorKind::AuthenticationError, + "BigQuery authentication error", + ), + BQError::YupAuthError(_) => ( + ErrorKind::AuthenticationError, + "BigQuery OAuth authentication error", + ), + BQError::NoToken => ( + ErrorKind::AuthenticationError, + "BigQuery authentication token missing", + ), + + // Network and transport errors + BQError::RequestError(_) => (ErrorKind::IoError, "BigQuery request failed"), + BQError::TonicTransportError(_) => (ErrorKind::IoError, "BigQuery transport error"), + + // Query and data errors + BQError::ResponseError { .. } => (ErrorKind::QueryFailed, "BigQuery response error"), + BQError::NoDataAvailable => ( + ErrorKind::InvalidState, + "BigQuery result set positioning error", + ), + BQError::InvalidColumnIndex { .. } => { + (ErrorKind::InvalidData, "BigQuery invalid column index") + } + BQError::InvalidColumnName { .. } => { + (ErrorKind::InvalidData, "BigQuery invalid column name") + } + BQError::InvalidColumnType { .. } => { + (ErrorKind::ConversionError, "BigQuery column type mismatch") + } + + // Serialization errors + BQError::SerializationError(_) => ( + ErrorKind::SerializationError, + "BigQuery JSON serialization error", + ), + + // gRPC errors + BQError::TonicInvalidMetadataValueError(_) => { + (ErrorKind::ConfigError, "BigQuery invalid metadata value") + } + BQError::TonicStatusError(_) => { + (ErrorKind::DestinationError, "BigQuery gRPC status error") + } }; ETLError { - repr: ErrorRepr::WithDescriptionAndDetail( - kind, - "BigQuery operation failed", - err.to_string(), - ), + repr: ErrorRepr::WithDescriptionAndDetail(kind, description, err.to_string()), } } } /// Converts BigQuery row errors to [`ETLError`] with [`ErrorKind::DestinationError`]. #[cfg(feature = "bigquery")] -impl From for ETLError { - fn from(err: crate::clients::bigquery::RowErrors) -> ETLError { +impl From for ETLError { + fn from(err: gcp_bigquery_client::google::cloud::bigquery::storage::v1::RowError) -> ETLError { ETLError { repr: ErrorRepr::WithDescriptionAndDetail( ErrorKind::DestinationError, - "BigQuery row errors", - err.to_string(), + "BigQuery row error", + format!("{err:?}"), ), } } @@ -526,7 +687,7 @@ mod tests { let errors = vec![ ETLError::from((ErrorKind::ValidationError, "Invalid schema")), ETLError::from((ErrorKind::ConversionError, "Type mismatch")), - ETLError::from((ErrorKind::NetworkError, "Connection timeout")), + ETLError::from((ErrorKind::IoError, "Connection timeout")), ]; let multi_err = ETLError::many(errors); @@ -536,7 +697,7 @@ mod tests { vec![ ErrorKind::ValidationError, ErrorKind::ConversionError, - ErrorKind::NetworkError + ErrorKind::IoError ] ); assert_eq!(multi_err.detail(), None); @@ -575,39 +736,6 @@ mod tests { assert_eq!(multi_err.detail(), None); } - #[test] - fn test_from_io_error() { - let io_err = std::io::Error::new(std::io::ErrorKind::PermissionDenied, "Access denied"); - let etl_err = ETLError::from(io_err); - assert_eq!(etl_err.kind(), ErrorKind::NetworkError); - assert!(etl_err.detail().unwrap().contains("Access denied")); - } - - #[test] - fn test_from_json_error() { - let json_err = serde_json::from_str::("invalid json").unwrap_err(); - let etl_err = ETLError::from(json_err); - assert_eq!(etl_err.kind(), ErrorKind::SerializationError); - assert!(etl_err.detail().is_some()); - } - - #[test] - fn test_from_parse_int_error() { - let parse_err = "not_a_number".parse::().unwrap_err(); - let etl_err = ETLError::from(parse_err); - assert_eq!(etl_err.kind(), ErrorKind::ConversionError); - assert!(etl_err.detail().is_some()); - } - - #[test] - fn test_from_utf8_error() { - let invalid_utf8 = vec![0, 159, 146, 150]; - let utf8_err = std::str::from_utf8(&invalid_utf8).unwrap_err(); - let etl_err = ETLError::from(utf8_err); - assert_eq!(etl_err.kind(), ErrorKind::ConversionError); - assert!(etl_err.detail().is_some()); - } - #[test] fn test_error_equality() { let err1 = ETLError::from((ErrorKind::ConnectionFailed, "Connection failed")); @@ -702,7 +830,7 @@ mod tests { let outer_errors = vec![ inner_multi, - ETLError::from((ErrorKind::NetworkError, "Outer error")), + ETLError::from((ErrorKind::IoError, "Outer error")), ]; let outer_multi = ETLError::many(outer_errors); @@ -710,6 +838,61 @@ mod tests { assert_eq!(kinds.len(), 3); assert!(kinds.contains(&ErrorKind::ConversionError)); assert!(kinds.contains(&ErrorKind::ValidationError)); - assert!(kinds.contains(&ErrorKind::NetworkError)); + assert!(kinds.contains(&ErrorKind::IoError)); + } + + #[test] + #[cfg(feature = "bigquery")] + fn test_bigquery_error_mapping() { + use gcp_bigquery_client::error::BQError; + + // Test authentication error + let auth_err = BQError::NoToken; + let etl_err = ETLError::from(auth_err); + assert_eq!(etl_err.kind(), ErrorKind::AuthenticationError); + + // Test invalid data error + let invalid_col_err = BQError::InvalidColumnIndex { col_index: 5 }; + let etl_err = ETLError::from(invalid_col_err); + assert_eq!(etl_err.kind(), ErrorKind::InvalidData); + + // Test conversion error + let type_err = BQError::InvalidColumnType { + col_index: 0, + col_type: "STRING".to_string(), + type_requested: "INTEGER".to_string(), + }; + let etl_err = ETLError::from(type_err); + assert_eq!(etl_err.kind(), ErrorKind::ConversionError); + } + + #[test] + fn test_postgres_error_mapping() { + // Test that our PostgreSQL error mapping logic is correctly structured + // by verifying that we can convert standard IO errors to ETL errors + let io_err = + std::io::Error::new(std::io::ErrorKind::ConnectionRefused, "Connection refused"); + let etl_err = ETLError::from(io_err); + assert_eq!(etl_err.kind(), ErrorKind::IoError); + assert!(etl_err.detail().unwrap().contains("Connection refused")); + + // Note: Testing actual PostgreSQL SQLSTATE mapping would require + // creating mock database errors, which is complex. The mapping logic + // is verified through the comprehensive match patterns above. + } + + #[test] + fn test_json_error_classification() { + // Test syntax error during deserialization + let json_err = serde_json::from_str::("invalid json").unwrap_err(); + let etl_err = ETLError::from(json_err); + assert_eq!(etl_err.kind(), ErrorKind::DeserializationError); + assert!(etl_err.detail().unwrap().contains("expected")); + + // Test data error during deserialization + let json_err = serde_json::from_str::("\"not_a_bool\"").unwrap_err(); + let etl_err = ETLError::from(json_err); + assert_eq!(etl_err.kind(), ErrorKind::DeserializationError); + assert!(etl_err.detail().is_some()); } } From f3f0bb50e3a49029c70477f54f777c47caa95b95 Mon Sep 17 00:00:00 2001 From: Riccardo Busetti Date: Fri, 25 Jul 2025 17:29:30 +0200 Subject: [PATCH 16/18] Improve --- etl/tests/failpoints/pipeline_test.rs | 62 +++++++++++++-------------- 1 file changed, 29 insertions(+), 33 deletions(-) diff --git a/etl/tests/failpoints/pipeline_test.rs b/etl/tests/failpoints/pipeline_test.rs index acb52a175..0640fb965 100644 --- a/etl/tests/failpoints/pipeline_test.rs +++ b/etl/tests/failpoints/pipeline_test.rs @@ -13,12 +13,9 @@ use telemetry::init_test_tracing; use etl::test_utils::database::spawn_database; use etl::test_utils::pipeline::create_pipeline; use etl::test_utils::test_destination_wrapper::TestDestinationWrapper; -use etl::test_utils::test_schema::{TableSelection, setup_test_database_schema}; +use etl::test_utils::test_schema::{TableSelection, insert_mock_data, setup_test_database_schema}; -/* -Tests that we want to add: -TBD - */ +// TODO: add more tests with fault injection. #[tokio::test(flavor = "multi_thread")] async fn pipeline_handles_table_sync_worker_panic_during_data_sync() { @@ -118,17 +115,27 @@ async fn pipeline_handles_table_sync_worker_error_during_data_sync() { assert_eq!(err.kinds()[1], ErrorKind::Unknown); } -// TODO: inject the failure via fail-rs. -#[ignore] #[tokio::test(flavor = "multi_thread")] -async fn table_schema_copy_retries_after_data_sync_failure() { +async fn table_copy_is_consistent_after_data_sync_threw_an_error() { let _scenario = FailScenario::setup(); + fail::cfg(START_TABLE_SYNC_AFTER_DATA_SYNC_ERROR, "return").unwrap(); init_test_tracing(); - let database = spawn_database().await; + let mut database = spawn_database().await; let database_schema = setup_test_database_schema(&database, TableSelection::Both).await; + // Insert initial test data. + let rows_inserted = 10; + insert_mock_data( + &mut database, + &database_schema.users_schema().name, + &database_schema.orders_schema().name, + 1..=rows_inserted, + false, + ) + .await; + let state_store = NotifyingStateStore::new(); let destination = TestDestinationWrapper::wrap(MemoryDestination::new()); @@ -161,11 +168,11 @@ async fn table_schema_copy_retries_after_data_sync_failure() { users_state_notify.notified().await; orders_state_notify.notified().await; - // This result could be an error or not based on if we manage to shut down before the error is - // thrown. This is a shortcoming of this fault injection implementation, we have plans to fix - // this in future PRs. - // TODO: assert error once better failure injection is implemented. - let _ = pipeline.shutdown_and_wait().await; + let err = pipeline.shutdown_and_wait().await.err().unwrap(); + assert_eq!(err.kinds().len(), 2); + + // We disable the failpoint. + fail::remove(START_TABLE_SYNC_AFTER_DATA_SYNC_ERROR); // Restart pipeline with normal state store to verify recovery. let mut pipeline = create_pipeline( @@ -183,13 +190,13 @@ async fn table_schema_copy_retries_after_data_sync_failure() { let users_state_notify = state_store .notify_on_table_state( database_schema.users_schema().id, - TableReplicationPhaseType::FinishedCopy, + TableReplicationPhaseType::SyncDone, ) .await; let orders_state_notify = state_store .notify_on_table_state( database_schema.orders_schema().id, - TableReplicationPhaseType::FinishedCopy, + TableReplicationPhaseType::SyncDone, ) .await; @@ -201,23 +208,12 @@ async fn table_schema_copy_retries_after_data_sync_failure() { pipeline.shutdown_and_wait().await.unwrap(); - // Verify table replication states. - let table_replication_states = state_store.get_table_replication_states().await; - assert_eq!(table_replication_states.len(), 2); - assert_eq!( - table_replication_states - .get(&database_schema.users_schema().id) - .unwrap() - .as_type(), - TableReplicationPhaseType::FinishedCopy - ); - assert_eq!( - table_replication_states - .get(&database_schema.orders_schema().id) - .unwrap() - .as_type(), - TableReplicationPhaseType::FinishedCopy - ); + // Verify copied data. + let table_rows = destination.get_table_rows().await; + let users_table_rows = table_rows.get(&database_schema.users_schema().id).unwrap(); + let orders_table_rows = table_rows.get(&database_schema.orders_schema().id).unwrap(); + assert_eq!(users_table_rows.len(), rows_inserted); + assert_eq!(orders_table_rows.len(), rows_inserted); // Verify table schemas were correctly stored. let table_schemas = destination.get_table_schemas().await; From 5e83f5ada34aa8b0d21e50d478273690a26c35ec Mon Sep 17 00:00:00 2001 From: Riccardo Busetti Date: Mon, 28 Jul 2025 09:12:51 +0200 Subject: [PATCH 17/18] Improve --- etl/benches/table_copies.rs | 22 +- etl/src/clients/bigquery.rs | 18 +- etl/src/conversions/bool.rs | 6 +- etl/src/conversions/event.rs | 23 +- etl/src/conversions/hex.rs | 6 +- etl/src/conversions/table_row.rs | 6 +- etl/src/conversions/text.rs | 10 +- etl/src/destination/base.rs | 12 +- etl/src/destination/bigquery.rs | 38 +-- etl/src/destination/memory.rs | 10 +- etl/src/error.rs | 257 ++++++++---------- etl/src/macros.rs | 4 +- etl/src/pipeline.rs | 12 +- etl/src/replication/apply.rs | 57 ++-- etl/src/replication/client.rs | 58 ++-- etl/src/replication/common.rs | 4 +- etl/src/replication/slot.rs | 6 +- etl/src/replication/stream.rs | 10 +- etl/src/replication/table_sync.rs | 4 +- etl/src/state/store/base.rs | 10 +- etl/src/state/store/memory.rs | 10 +- etl/src/state/store/notify.rs | 10 +- etl/src/state/store/postgres.rs | 14 +- .../test_utils/test_destination_wrapper.rs | 12 +- etl/src/workers/apply.rs | 24 +- etl/src/workers/base.rs | 4 +- etl/src/workers/pool.rs | 10 +- etl/src/workers/table_sync.rs | 22 +- 28 files changed, 325 insertions(+), 354 deletions(-) diff --git a/etl/benches/table_copies.rs b/etl/benches/table_copies.rs index 1b600659a..335f515ab 100644 --- a/etl/benches/table_copies.rs +++ b/etl/benches/table_copies.rs @@ -45,7 +45,7 @@ use sqlx::postgres::PgPool; #[cfg(feature = "bigquery")] use etl::destination::bigquery::BigQueryDestination; -use etl::error::ETLResult; +use etl::error::EtlResult; #[derive(Parser, Debug)] #[command(author, version, about, long_about = None)] @@ -436,7 +436,7 @@ enum BenchDestination { } impl Destination for BenchDestination { - async fn inject(&self, schema_cache: etl::schema::cache::SchemaCache) -> ETLResult<()> { + async fn inject(&self, schema_cache: etl::schema::cache::SchemaCache) -> EtlResult<()> { match self { BenchDestination::Null(dest) => dest.inject(schema_cache).await, #[cfg(feature = "bigquery")] @@ -444,7 +444,7 @@ impl Destination for BenchDestination { } } - async fn write_table_schema(&self, table_schema: TableSchema) -> ETLResult<()> { + async fn write_table_schema(&self, table_schema: TableSchema) -> EtlResult<()> { match self { BenchDestination::Null(dest) => dest.write_table_schema(table_schema).await, #[cfg(feature = "bigquery")] @@ -452,7 +452,7 @@ impl Destination for BenchDestination { } } - async fn load_table_schemas(&self) -> ETLResult> { + async fn load_table_schemas(&self) -> EtlResult> { match self { BenchDestination::Null(dest) => dest.load_table_schemas().await, #[cfg(feature = "bigquery")] @@ -464,7 +464,7 @@ impl Destination for BenchDestination { &self, table_id: TableId, table_rows: Vec, - ) -> ETLResult<()> { + ) -> EtlResult<()> { match self { BenchDestination::Null(dest) => dest.write_table_rows(table_id, table_rows).await, #[cfg(feature = "bigquery")] @@ -472,7 +472,7 @@ impl Destination for BenchDestination { } } - async fn write_events(&self, events: Vec) -> ETLResult<()> { + async fn write_events(&self, events: Vec) -> EtlResult<()> { match self { BenchDestination::Null(dest) => dest.write_events(events).await, #[cfg(feature = "bigquery")] @@ -482,15 +482,15 @@ impl Destination for BenchDestination { } impl Destination for NullDestination { - async fn inject(&self, _schema_cache: etl::schema::cache::SchemaCache) -> ETLResult<()> { + async fn inject(&self, _schema_cache: etl::schema::cache::SchemaCache) -> EtlResult<()> { Ok(()) } - async fn write_table_schema(&self, _table_schema: TableSchema) -> ETLResult<()> { + async fn write_table_schema(&self, _table_schema: TableSchema) -> EtlResult<()> { Ok(()) } - async fn load_table_schemas(&self) -> ETLResult> { + async fn load_table_schemas(&self) -> EtlResult> { Ok(vec![]) } @@ -498,11 +498,11 @@ impl Destination for NullDestination { &self, _table_id: TableId, _table_rows: Vec, - ) -> ETLResult<()> { + ) -> EtlResult<()> { Ok(()) } - async fn write_events(&self, _events: Vec) -> ETLResult<()> { + async fn write_events(&self, _events: Vec) -> EtlResult<()> { Ok(()) } } diff --git a/etl/src/clients/bigquery.rs b/etl/src/clients/bigquery.rs index 34583b792..183b83c30 100644 --- a/etl/src/clients/bigquery.rs +++ b/etl/src/clients/bigquery.rs @@ -1,4 +1,4 @@ -use crate::error::ETLResult; +use crate::error::EtlResult; use futures::StreamExt; use gcp_bigquery_client::storage::{ColumnMode, StorageApi}; use gcp_bigquery_client::yup_oauth2::parse_service_account_key; @@ -68,7 +68,7 @@ impl BigQueryClient { pub async fn new_with_key_path( project_id: String, sa_key_path: &str, - ) -> ETLResult { + ) -> EtlResult { let client = Client::from_service_account_key_file(sa_key_path).await?; Ok(BigQueryClient { project_id, client }) @@ -78,7 +78,7 @@ impl BigQueryClient { /// /// Parses the provided service account key string to authenticate with the /// BigQuery API. - pub async fn new_with_key(project_id: String, sa_key: &str) -> ETLResult { + pub async fn new_with_key(project_id: String, sa_key: &str) -> EtlResult { let sa_key = parse_service_account_key(sa_key).map_err(BQError::from)?; let client = Client::from_service_account_key(sa_key, false).await?; @@ -100,7 +100,7 @@ impl BigQueryClient { table_id: &str, column_schemas: &[ColumnSchema], max_staleness_mins: Option, - ) -> ETLResult { + ) -> EtlResult { if self.table_exists(dataset_id, table_id).await? { return Ok(false); } @@ -118,7 +118,7 @@ impl BigQueryClient { table_id: &str, column_schemas: &[ColumnSchema], max_staleness_mins: Option, - ) -> ETLResult<()> { + ) -> EtlResult<()> { let full_table_name = self.full_table_name(dataset_id, table_id); let columns_spec = Self::create_columns_spec(column_schemas); @@ -138,7 +138,7 @@ impl BigQueryClient { } /// Truncates a table in a BigQuery dataset. - pub async fn truncate_table(&self, dataset_id: &str, table_id: &str) -> ETLResult<()> { + pub async fn truncate_table(&self, dataset_id: &str, table_id: &str) -> EtlResult<()> { let full_table_name = self.full_table_name(dataset_id, table_id); info!("Truncating table {full_table_name} in BigQuery"); @@ -155,7 +155,7 @@ impl BigQueryClient { /// # Panics /// /// Panics if the query result does not contain the expected `table_exists` column. - pub async fn table_exists(&self, dataset_id: &str, table_id: &str) -> ETLResult { + pub async fn table_exists(&self, dataset_id: &str, table_id: &str) -> EtlResult { let table = self .client .table() @@ -178,7 +178,7 @@ impl BigQueryClient { table_id: String, table_descriptor: &TableDescriptor, table_rows: Vec, - ) -> ETLResult<()> { + ) -> EtlResult<()> { // We create a slice on table rows, which will be updated while the streaming progresses. // // Using a slice allows us to deallocate the vector only at the end of streaming, which leads @@ -218,7 +218,7 @@ impl BigQueryClient { } /// Executes an SQL query and returns the result set. - pub async fn query(&self, request: QueryRequest) -> ETLResult { + pub async fn query(&self, request: QueryRequest) -> EtlResult { let query_response = self.client.job().query(&self.project_id, request).await?; Ok(ResultSet::new_from_query_response(query_response)) diff --git a/etl/src/conversions/bool.rs b/etl/src/conversions/bool.rs index b4e43492d..f3e00d2b9 100644 --- a/etl/src/conversions/bool.rs +++ b/etl/src/conversions/bool.rs @@ -1,8 +1,8 @@ use crate::bail; -use crate::error::ETLResult; -use crate::error::{ETLError, ErrorKind}; +use crate::error::EtlResult; +use crate::error::{EtlError, ErrorKind}; -pub fn parse_bool(s: &str) -> ETLResult { +pub fn parse_bool(s: &str) -> EtlResult { if s == "t" { Ok(true) } else if s == "f" { diff --git a/etl/src/conversions/event.rs b/etl/src/conversions/event.rs index c9acbff9b..eff419a1c 100644 --- a/etl/src/conversions/event.rs +++ b/etl/src/conversions/event.rs @@ -9,8 +9,8 @@ use tokio_postgres::types::PgLsn; use crate::conversions::Cell; use crate::conversions::table_row::TableRow; use crate::conversions::text::TextFormatConverter; -use crate::error::ETLError; -use crate::error::{ETLResult, ErrorKind}; +use crate::error::EtlError; +use crate::error::{EtlResult, ErrorKind}; use crate::schema::cache::SchemaCache; use crate::{bail, etl_error}; @@ -74,7 +74,7 @@ impl RelationEvent { start_lsn: PgLsn, commit_lsn: PgLsn, relation_body: &protocol::RelationBody, - ) -> ETLResult { + ) -> EtlResult { let table_name = TableName::new( relation_body.namespace()?.to_string(), relation_body.name()?.to_string(), @@ -97,7 +97,7 @@ impl RelationEvent { }) } - fn build_column_schema(column: &protocol::Column) -> ETLResult { + fn build_column_schema(column: &protocol::Column) -> EtlResult { Ok(ColumnSchema::new( column.name()?.to_string(), convert_type_oid_to_type(column.type_id() as u32), @@ -238,7 +238,7 @@ impl From for EventType { } } -async fn get_table_schema(schema_cache: &SchemaCache, table_id: TableId) -> ETLResult { +async fn get_table_schema(schema_cache: &SchemaCache, table_id: TableId) -> EtlResult { schema_cache .get_table_schema(&table_id) .await @@ -247,8 +247,7 @@ async fn get_table_schema(schema_cache: &SchemaCache, table_id: TableId) -> ETLR ErrorKind::MissingTableSchema, "Table not found in the schema cache", format!( - "The table schema for table {} was not found in the cache", - table_id + "The table schema for table {table_id} was not found in the cache" ) ) }) @@ -257,7 +256,7 @@ async fn get_table_schema(schema_cache: &SchemaCache, table_id: TableId) -> ETLR fn convert_tuple_to_row( column_schemas: &[ColumnSchema], tuple_data: &[protocol::TupleData], -) -> ETLResult { +) -> EtlResult { let mut values = Vec::with_capacity(column_schemas.len()); for (i, column_schema) in column_schemas.iter().enumerate() { @@ -300,7 +299,7 @@ async fn convert_insert_to_event( start_lsn: PgLsn, commit_lsn: PgLsn, insert_body: &protocol::InsertBody, -) -> ETLResult { +) -> EtlResult { let table_id = insert_body.rel_id(); let table_schema = get_table_schema(schema_cache, TableId::new(table_id)).await?; @@ -322,7 +321,7 @@ async fn convert_update_to_event( start_lsn: PgLsn, commit_lsn: PgLsn, update_body: &protocol::UpdateBody, -) -> ETLResult { +) -> EtlResult { let table_id = update_body.rel_id(); let table_schema = get_table_schema(schema_cache, TableId::new(table_id)).await?; @@ -358,7 +357,7 @@ async fn convert_delete_to_event( start_lsn: PgLsn, commit_lsn: PgLsn, delete_body: &protocol::DeleteBody, -) -> ETLResult { +) -> EtlResult { let table_id = delete_body.rel_id(); let table_schema = get_table_schema(schema_cache, TableId::new(table_id)).await?; @@ -388,7 +387,7 @@ pub async fn convert_message_to_event( start_lsn: PgLsn, commit_lsn: PgLsn, message: &LogicalReplicationMessage, -) -> ETLResult { +) -> EtlResult { match message { LogicalReplicationMessage::Begin(begin_body) => Ok(Event::Begin( BeginEvent::from_protocol(start_lsn, commit_lsn, begin_body), diff --git a/etl/src/conversions/hex.rs b/etl/src/conversions/hex.rs index ed8317286..762751f6b 100644 --- a/etl/src/conversions/hex.rs +++ b/etl/src/conversions/hex.rs @@ -1,8 +1,8 @@ use crate::bail; -use crate::error::ETLError; -use crate::error::{ETLResult, ErrorKind}; +use crate::error::EtlError; +use crate::error::{EtlResult, ErrorKind}; -pub fn from_bytea_hex(s: &str) -> ETLResult> { +pub fn from_bytea_hex(s: &str) -> EtlResult> { if s.len() < 2 || &s[..2] != "\\x" { bail!( ErrorKind::ConversionError, diff --git a/etl/src/conversions/table_row.rs b/etl/src/conversions/table_row.rs index b2c2b61ea..000a30885 100644 --- a/etl/src/conversions/table_row.rs +++ b/etl/src/conversions/table_row.rs @@ -1,8 +1,8 @@ use super::Cell; use crate::bail; use crate::conversions::text::TextFormatConverter; -use crate::error::ETLError; -use crate::error::{ETLResult, ErrorKind}; +use crate::error::EtlError; +use crate::error::{EtlResult, ErrorKind}; use core::str; use postgres::schema::ColumnSchema; use tracing::error; @@ -66,7 +66,7 @@ pub struct TableRowConverter; impl TableRowConverter { // parses text produced by this code in Postgres: https://github.com/postgres/postgres/blob/263a3f5f7f508167dbeafc2aefd5835b41d77481/src/backend/commands/copyto.c#L988-L1134 - pub fn try_from(row: &[u8], column_schemas: &[ColumnSchema]) -> ETLResult { + pub fn try_from(row: &[u8], column_schemas: &[ColumnSchema]) -> EtlResult { let mut values = Vec::with_capacity(column_schemas.len()); let row_str = str::from_utf8(row)?; diff --git a/etl/src/conversions/text.rs b/etl/src/conversions/text.rs index 43f03f7a6..d6f2a99ec 100644 --- a/etl/src/conversions/text.rs +++ b/etl/src/conversions/text.rs @@ -1,4 +1,4 @@ -use crate::error::ETLError; +use crate::error::EtlError; use chrono::{DateTime, FixedOffset, NaiveDate, NaiveDateTime, NaiveTime, Utc}; use core::str; use tokio_postgres::types::Type; @@ -6,7 +6,7 @@ use uuid::Uuid; use crate::bail; use crate::conversions::{bool::parse_bool, hex}; -use crate::error::{ETLResult, ErrorKind}; +use crate::error::{EtlResult, ErrorKind}; use super::{ArrayCell, Cell, numeric::PgNumeric}; @@ -65,7 +65,7 @@ impl TextFormatConverter { } } - pub fn try_from_str(typ: &Type, str: &str) -> ETLResult { + pub fn try_from_str(typ: &Type, str: &str) -> EtlResult { match *typ { Type::BOOL => Ok(Cell::Bool(parse_bool(str)?)), Type::BOOL_ARRAY => TextFormatConverter::parse_array( @@ -223,9 +223,9 @@ impl TextFormatConverter { } } - fn parse_array(str: &str, mut parse: P, m: M) -> ETLResult + fn parse_array(str: &str, mut parse: P, m: M) -> EtlResult where - P: FnMut(&str) -> ETLResult>, + P: FnMut(&str) -> EtlResult>, M: FnOnce(Vec>) -> ArrayCell, { if str.len() < 2 { diff --git a/etl/src/destination/base.rs b/etl/src/destination/base.rs index 53d0cc89b..860fd875b 100644 --- a/etl/src/destination/base.rs +++ b/etl/src/destination/base.rs @@ -3,11 +3,11 @@ use std::future::Future; use crate::conversions::event::Event; use crate::conversions::table_row::TableRow; -use crate::error::ETLResult; +use crate::error::EtlResult; use crate::schema::cache::SchemaCache; pub trait Destination { - fn inject(&self, _schema_cache: SchemaCache) -> impl Future> + Send { + fn inject(&self, _schema_cache: SchemaCache) -> impl Future> + Send { // By default, the injection code is a noop, since not all destinations need dependencies // to be injected. async move { Ok(()) } @@ -16,15 +16,15 @@ pub trait Destination { fn write_table_schema( &self, table_schema: TableSchema, - ) -> impl Future> + Send; + ) -> impl Future> + Send; - fn load_table_schemas(&self) -> impl Future>> + Send; + fn load_table_schemas(&self) -> impl Future>> + Send; fn write_table_rows( &self, table_id: TableId, table_rows: Vec, - ) -> impl Future> + Send; + ) -> impl Future> + Send; - fn write_events(&self, events: Vec) -> impl Future> + Send; + fn write_events(&self, events: Vec) -> impl Future> + Send; } diff --git a/etl/src/destination/bigquery.rs b/etl/src/destination/bigquery.rs index 5acff491d..b7bf02a27 100644 --- a/etl/src/destination/bigquery.rs +++ b/etl/src/destination/bigquery.rs @@ -13,7 +13,7 @@ use crate::conversions::Cell; use crate::conversions::event::{Event, TruncateEvent}; use crate::conversions::table_row::TableRow; use crate::destination::base::Destination; -use crate::error::{ETLError, ETLResult, ErrorKind}; +use crate::error::{EtlError, EtlResult, ErrorKind}; use crate::schema::cache::SchemaCache; /// Table name for storing ETL table schema metadata in BigQuery. @@ -86,7 +86,7 @@ impl Inner { /// Ensures the ETL metadata tables exist in BigQuery. /// /// Creates `etl_table_schemas` and `etl_table_columns` tables if they don't exist. - async fn ensure_schema_tables_exist(&self) -> ETLResult<()> { + async fn ensure_schema_tables_exist(&self) -> EtlResult<()> { // Create etl_table_schemas table - use ColumnSchema for compatibility self.client .create_table_if_missing( @@ -135,7 +135,7 @@ impl BigQueryDestination { dataset_id: String, sa_key: &str, max_staleness_mins: Option, - ) -> ETLResult { + ) -> EtlResult { let client = BigQueryClient::new_with_key_path(project_id, sa_key).await?; let inner = Inner { client, @@ -158,7 +158,7 @@ impl BigQueryDestination { dataset_id: String, sa_key: &str, max_staleness_mins: Option, - ) -> ETLResult { + ) -> EtlResult { let client = BigQueryClient::new_with_key(project_id, sa_key).await?; let inner = Inner { client, @@ -179,12 +179,12 @@ impl BigQueryDestination { inner: &I, table_id: &TableId, use_cdc_sequence_column: bool, - ) -> ETLResult<(String, TableDescriptor)> { + ) -> EtlResult<(String, TableDescriptor)> { let schema_cache = inner .schema_cache .as_ref() .ok_or_else(|| { - ETLError::from(( + EtlError::from(( ErrorKind::ConfigError, "The schema cache was not set on the destination", )) @@ -192,7 +192,7 @@ impl BigQueryDestination { .lock_inner() .await; let table_schema = schema_cache.get_table_schema_ref(table_id).ok_or_else(|| { - ETLError::from(( + EtlError::from(( ErrorKind::DestinationSchemaError, "Table schema not found in schema cache", format!("table_id: {table_id}"), @@ -211,7 +211,7 @@ impl BigQueryDestination { /// Writes a table schema to BigQuery, creating the data table and storing metadata. /// /// This method creates the actual data table and inserts schema information into the ETL metadata tables. - async fn write_table_schema(&self, table_schema: TableSchema) -> ETLResult<()> { + async fn write_table_schema(&self, table_schema: TableSchema) -> EtlResult<()> { let mut inner = self.inner.lock().await; let dataset_id = inner.dataset_id.clone(); @@ -276,7 +276,7 @@ impl BigQueryDestination { /// Loads all table schemas from BigQuery ETL metadata tables. /// /// Reconstructs [`TableSchema`] objects by joining data from schema and column metadata tables. - async fn load_table_schemas(&self) -> ETLResult> { + async fn load_table_schemas(&self) -> EtlResult> { let inner = self.inner.lock().await; // First check if schema tables exist @@ -392,7 +392,7 @@ impl BigQueryDestination { &self, table_id: TableId, mut table_rows: Vec, - ) -> ETLResult<()> { + ) -> EtlResult<()> { let mut inner = self.inner.lock().await; // We do not use the sequence column for table rows, since we assume that table rows are always @@ -418,7 +418,7 @@ impl BigQueryDestination { /// /// Groups events by type, handles inserts/updates/deletes via streaming, and processes truncates separately. /// Adds sequence numbers to ensure proper ordering of events with the same system time. - async fn write_events(&self, events: Vec) -> ETLResult<()> { + async fn write_events(&self, events: Vec) -> EtlResult<()> { let mut event_iter = events.into_iter().peekable(); while event_iter.peek().is_some() { @@ -525,7 +525,7 @@ impl BigQueryDestination { /// /// Maps PostgreSQL table OIDs to BigQuery table names and issues truncate commands. #[allow(dead_code)] - async fn process_truncate_events(&self, truncate_events: Vec) -> ETLResult<()> { + async fn process_truncate_events(&self, truncate_events: Vec) -> EtlResult<()> { let inner = self.inner.lock().await; for truncate_event in truncate_events { @@ -535,7 +535,7 @@ impl BigQueryDestination { .schema_cache .as_ref() .ok_or_else(|| { - ETLError::from(( + EtlError::from(( ErrorKind::ConfigError, "The schema cache was not set on the destination", )) @@ -634,7 +634,7 @@ impl BigQueryDestination { /// /// Used when reconstructing schemas from BigQuery metadata tables. Falls back to `TEXT` for unknown types. #[allow(clippy::result_large_err)] - fn string_to_postgres_type(type_str: &str) -> ETLResult { + fn string_to_postgres_type(type_str: &str) -> EtlResult { match type_str { "BOOL" => Ok(Type::BOOL), "CHAR" => Ok(Type::CHAR), @@ -683,20 +683,20 @@ impl BigQueryDestination { } impl Destination for BigQueryDestination { - async fn inject(&self, schema_cache: SchemaCache) -> ETLResult<()> { + async fn inject(&self, schema_cache: SchemaCache) -> EtlResult<()> { let mut inner = self.inner.lock().await; inner.schema_cache = Some(schema_cache); Ok(()) } - async fn write_table_schema(&self, table_schema: TableSchema) -> ETLResult<()> { + async fn write_table_schema(&self, table_schema: TableSchema) -> EtlResult<()> { self.write_table_schema(table_schema).await?; Ok(()) } - async fn load_table_schemas(&self) -> ETLResult> { + async fn load_table_schemas(&self) -> EtlResult> { let table_schemas = self.load_table_schemas().await?; Ok(table_schemas) @@ -706,13 +706,13 @@ impl Destination for BigQueryDestination { &self, table_id: TableId, table_rows: Vec, - ) -> ETLResult<()> { + ) -> EtlResult<()> { self.write_table_rows(table_id, table_rows).await?; Ok(()) } - async fn write_events(&self, events: Vec) -> ETLResult<()> { + async fn write_events(&self, events: Vec) -> EtlResult<()> { self.write_events(events).await?; Ok(()) diff --git a/etl/src/destination/memory.rs b/etl/src/destination/memory.rs index e6a77eb92..4ed5f5fc1 100644 --- a/etl/src/destination/memory.rs +++ b/etl/src/destination/memory.rs @@ -6,7 +6,7 @@ use tracing::info; use crate::conversions::event::Event; use crate::conversions::table_row::TableRow; use crate::destination::base::Destination; -use crate::error::ETLResult; +use crate::error::EtlResult; #[derive(Debug)] struct Inner { @@ -41,7 +41,7 @@ impl Default for MemoryDestination { } impl Destination for MemoryDestination { - async fn write_table_schema(&self, table_schema: TableSchema) -> ETLResult<()> { + async fn write_table_schema(&self, table_schema: TableSchema) -> EtlResult<()> { let mut inner = self.inner.lock().await; info!("writing table schema:"); info!("{:?}", table_schema); @@ -50,7 +50,7 @@ impl Destination for MemoryDestination { Ok(()) } - async fn load_table_schemas(&self) -> ETLResult> { + async fn load_table_schemas(&self) -> EtlResult> { let inner = self.inner.lock().await; let schemas = inner.table_schemas.to_vec(); info!("loaded {} table schemas:", schemas.len()); @@ -63,7 +63,7 @@ impl Destination for MemoryDestination { &self, table_id: TableId, table_rows: Vec, - ) -> ETLResult<()> { + ) -> EtlResult<()> { let mut inner = self.inner.lock().await; info!("writing a batch of {} table rows:", table_rows.len()); for table_row in &table_rows { @@ -74,7 +74,7 @@ impl Destination for MemoryDestination { Ok(()) } - async fn write_events(&self, events: Vec) -> ETLResult<()> { + async fn write_events(&self, events: Vec) -> EtlResult<()> { let mut inner = self.inner.lock().await; info!("writing a batch of {} events:", events.len()); for event in &events { diff --git a/etl/src/error.rs b/etl/src/error.rs index 805adc8f0..e2c36325c 100644 --- a/etl/src/error.rs +++ b/etl/src/error.rs @@ -1,25 +1,26 @@ use std::error; use std::fmt; -/// Convenient result type for ETL operations using [`ETLError`] as the error type. +/// Convenient result type for ETL operations using [`EtlError`] as the error type. /// /// This type alias reduces boilerplate when working with fallible ETL operations. /// Most ETL functions return this type. -pub type ETLResult = Result; +pub type EtlResult = Result; /// Main error type for ETL operations. /// -/// [`ETLError`] provides a comprehensive error system that can represent single errors, +/// [`EtlError`] provides a comprehensive error system that can represent single errors, /// errors with additional detail, or multiple aggregated errors. The design allows for /// rich error information while maintaining ergonomic usage patterns. -pub struct ETLError { +#[derive(Debug)] +pub struct EtlError { repr: ErrorRepr, } /// Internal representation of error data. /// /// This enum supports different error patterns while maintaining a unified interface. -/// Users should not interact with this type directly but use [`ETLError`] methods instead. +/// Users should not interact with this type directly but use [`EtlError`] methods instead. #[derive(Debug)] pub enum ErrorRepr { /// Error with kind and static description @@ -27,7 +28,7 @@ pub enum ErrorRepr { /// Error with kind, static description, and dynamic detail WithDescriptionAndDetail(ErrorKind, &'static str, String), /// Multiple aggregated errors - Many(Vec), + Many(Vec), } /// Specific categories of errors that can occur during ETL operations. @@ -121,13 +122,13 @@ pub enum ErrorKind { Unknown, } -impl ETLError { - /// Creates an [`ETLError`] containing multiple aggregated errors. +impl EtlError { + /// Creates an [`EtlError`] containing multiple aggregated errors. /// /// This is useful when multiple operations fail and you want to report all failures /// rather than just the first one. - pub fn many(errors: Vec) -> ETLError { - ETLError { + pub fn many(errors: Vec) -> EtlError { + EtlError { repr: ErrorRepr::Many(errors), } } @@ -178,8 +179,8 @@ impl ETLError { } } -impl PartialEq for ETLError { - fn eq(&self, other: &ETLError) -> bool { +impl PartialEq for EtlError { + fn eq(&self, other: &EtlError) -> bool { match (&self.repr, &other.repr) { (ErrorRepr::WithDescription(kind_a, _), ErrorRepr::WithDescription(kind_b, _)) => { kind_a == kind_b @@ -197,7 +198,7 @@ impl PartialEq for ETLError { } } -impl fmt::Display for ETLError { +impl fmt::Display for EtlError { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> Result<(), fmt::Error> { match self.repr { ErrorRepr::WithDescription(kind, desc) => { @@ -234,57 +235,35 @@ impl fmt::Display for ETLError { } } -impl fmt::Debug for ETLError { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> Result<(), fmt::Error> { - fmt::Display::fmt(self, f) - } -} - -impl error::Error for ETLError { - #[allow(deprecated)] - fn description(&self) -> &str { - match self.repr { - ErrorRepr::WithDescription(_, desc) - | ErrorRepr::WithDescriptionAndDetail(_, desc, _) => desc, - ErrorRepr::Many(ref errors) => { - if errors.is_empty() { - "Multiple errors occurred (empty)" - } else { - // Return the description of the first error - errors[0].description() - } - } - } - } -} +impl error::Error for EtlError { } // Ergonomic constructors following Redis pattern -/// Creates an [`ETLError`] from an error kind and static description. -impl From<(ErrorKind, &'static str)> for ETLError { - fn from((kind, desc): (ErrorKind, &'static str)) -> ETLError { - ETLError { +/// Creates an [`EtlError`] from an error kind and static description. +impl From<(ErrorKind, &'static str)> for EtlError { + fn from((kind, desc): (ErrorKind, &'static str)) -> EtlError { + EtlError { repr: ErrorRepr::WithDescription(kind, desc), } } } -/// Creates an [`ETLError`] from an error kind, static description, and dynamic detail. -impl From<(ErrorKind, &'static str, String)> for ETLError { - fn from((kind, desc, detail): (ErrorKind, &'static str, String)) -> ETLError { - ETLError { +/// Creates an [`EtlError`] from an error kind, static description, and dynamic detail. +impl From<(ErrorKind, &'static str, String)> for EtlError { + fn from((kind, desc, detail): (ErrorKind, &'static str, String)) -> EtlError { + EtlError { repr: ErrorRepr::WithDescriptionAndDetail(kind, desc, detail), } } } -/// Creates an [`ETLError`] from a vector of errors for aggregation. -impl From> for ETLError +/// Creates an [`EtlError`] from a vector of errors for aggregation. +impl From> for EtlError where - E: Into, + E: Into, { - fn from(errors: Vec) -> ETLError { - ETLError { + fn from(errors: Vec) -> EtlError { + EtlError { repr: ErrorRepr::Many(errors.into_iter().map(Into::into).collect()), } } @@ -292,10 +271,10 @@ where // Common standard library error conversions -/// Converts [`std::io::Error`] to [`ETLError`] with [`ErrorKind::IoError`]. -impl From for ETLError { - fn from(err: std::io::Error) -> ETLError { - ETLError { +/// Converts [`std::io::Error`] to [`EtlError`] with [`ErrorKind::IoError`]. +impl From for EtlError { + fn from(err: std::io::Error) -> EtlError { + EtlError { repr: ErrorRepr::WithDescriptionAndDetail( ErrorKind::IoError, "I/O error occurred", @@ -305,12 +284,12 @@ impl From for ETLError { } } -/// Converts [`serde_json::Error`] to [`ETLError`] with appropriate error kind. +/// Converts [`serde_json::Error`] to [`EtlError`] with appropriate error kind. /// /// Maps to [`ErrorKind::SerializationError`] for serialization failures and /// [`ErrorKind::DeserializationError`] for deserialization failures based on error classification. -impl From for ETLError { - fn from(err: serde_json::Error) -> ETLError { +impl From for EtlError { + fn from(err: serde_json::Error) -> EtlError { let (kind, description) = match err.classify() { serde_json::error::Category::Io => (ErrorKind::IoError, "JSON I/O operation failed"), serde_json::error::Category::Syntax | serde_json::error::Category::Data => ( @@ -323,16 +302,16 @@ impl From for ETLError { ), }; - ETLError { + EtlError { repr: ErrorRepr::WithDescriptionAndDetail(kind, description, err.to_string()), } } } -/// Converts [`std::str::Utf8Error`] to [`ETLError`] with [`ErrorKind::ConversionError`]. -impl From for ETLError { - fn from(err: std::str::Utf8Error) -> ETLError { - ETLError { +/// Converts [`std::str::Utf8Error`] to [`EtlError`] with [`ErrorKind::ConversionError`]. +impl From for EtlError { + fn from(err: std::str::Utf8Error) -> EtlError { + EtlError { repr: ErrorRepr::WithDescriptionAndDetail( ErrorKind::ConversionError, "UTF-8 conversion failed", @@ -342,10 +321,10 @@ impl From for ETLError { } } -/// Converts [`std::string::FromUtf8Error`] to [`ETLError`] with [`ErrorKind::ConversionError`]. -impl From for ETLError { - fn from(err: std::string::FromUtf8Error) -> ETLError { - ETLError { +/// Converts [`std::string::FromUtf8Error`] to [`EtlError`] with [`ErrorKind::ConversionError`]. +impl From for EtlError { + fn from(err: std::string::FromUtf8Error) -> EtlError { + EtlError { repr: ErrorRepr::WithDescriptionAndDetail( ErrorKind::ConversionError, "UTF-8 string conversion failed", @@ -355,10 +334,10 @@ impl From for ETLError { } } -/// Converts [`std::num::ParseIntError`] to [`ETLError`] with [`ErrorKind::ConversionError`]. -impl From for ETLError { - fn from(err: std::num::ParseIntError) -> ETLError { - ETLError { +/// Converts [`std::num::ParseIntError`] to [`EtlError`] with [`ErrorKind::ConversionError`]. +impl From for EtlError { + fn from(err: std::num::ParseIntError) -> EtlError { + EtlError { repr: ErrorRepr::WithDescriptionAndDetail( ErrorKind::ConversionError, "Integer parsing failed", @@ -368,10 +347,10 @@ impl From for ETLError { } } -/// Converts [`std::num::ParseFloatError`] to [`ETLError`] with [`ErrorKind::ConversionError`]. -impl From for ETLError { - fn from(err: std::num::ParseFloatError) -> ETLError { - ETLError { +/// Converts [`std::num::ParseFloatError`] to [`EtlError`] with [`ErrorKind::ConversionError`]. +impl From for EtlError { + fn from(err: std::num::ParseFloatError) -> EtlError { + EtlError { repr: ErrorRepr::WithDescriptionAndDetail( ErrorKind::ConversionError, "Float parsing failed", @@ -383,12 +362,12 @@ impl From for ETLError { // PostgreSQL-specific error conversions -/// Converts [`tokio_postgres::Error`] to [`ETLError`] with appropriate error kind. +/// Converts [`tokio_postgres::Error`] to [`EtlError`] with appropriate error kind. /// /// Maps errors based on PostgreSQL SQLSTATE codes to provide granular error classification /// for better error handling in ETL operations. -impl From for ETLError { - fn from(err: tokio_postgres::Error) -> ETLError { +impl From for EtlError { + fn from(err: tokio_postgres::Error) -> EtlError { let (kind, description) = match err.code() { Some(sqlstate) => { use tokio_postgres::error::SqlState; @@ -474,16 +453,16 @@ impl From for ETLError { None => (ErrorKind::ConnectionFailed, "PostgreSQL connection failed"), }; - ETLError { + EtlError { repr: ErrorRepr::WithDescriptionAndDetail(kind, description, err.to_string()), } } } -/// Converts [`rustls::Error`] to [`ETLError`] with [`ErrorKind::EncryptionError`]. -impl From for ETLError { - fn from(err: rustls::Error) -> ETLError { - ETLError { +/// Converts [`rustls::Error`] to [`EtlError`] with [`ErrorKind::EncryptionError`]. +impl From for EtlError { + fn from(err: rustls::Error) -> EtlError { + EtlError { repr: ErrorRepr::WithDescriptionAndDetail( ErrorKind::EncryptionError, "TLS configuration failed", @@ -493,10 +472,10 @@ impl From for ETLError { } } -/// Converts [`uuid::Error`] to [`ETLError`] with [`ErrorKind::InvalidData`]. -impl From for ETLError { - fn from(err: uuid::Error) -> ETLError { - ETLError { +/// Converts [`uuid::Error`] to [`EtlError`] with [`ErrorKind::InvalidData`]. +impl From for EtlError { + fn from(err: uuid::Error) -> EtlError { + EtlError { repr: ErrorRepr::WithDescriptionAndDetail( ErrorKind::InvalidData, "UUID parsing failed", @@ -506,10 +485,10 @@ impl From for ETLError { } } -/// Converts [`chrono::ParseError`] to [`ETLError`] with [`ErrorKind::ConversionError`]. -impl From for ETLError { - fn from(err: chrono::ParseError) -> ETLError { - ETLError { +/// Converts [`chrono::ParseError`] to [`EtlError`] with [`ErrorKind::ConversionError`]. +impl From for EtlError { + fn from(err: chrono::ParseError) -> EtlError { + EtlError { repr: ErrorRepr::WithDescriptionAndDetail( ErrorKind::ConversionError, "Chrono parse failed", @@ -519,10 +498,10 @@ impl From for ETLError { } } -/// Converts [`bigdecimal::ParseBigDecimalError`] to [`ETLError`] with [`ErrorKind::ConversionError`]. -impl From for ETLError { - fn from(err: bigdecimal::ParseBigDecimalError) -> ETLError { - ETLError { +/// Converts [`bigdecimal::ParseBigDecimalError`] to [`EtlError`] with [`ErrorKind::ConversionError`]. +impl From for EtlError { + fn from(err: bigdecimal::ParseBigDecimalError) -> EtlError { + EtlError { repr: ErrorRepr::WithDescriptionAndDetail( ErrorKind::ConversionError, "BigDecimal parsing failed", @@ -534,12 +513,12 @@ impl From for ETLError { // SQLx error conversion -/// Converts [`sqlx::Error`] to [`ETLError`] with appropriate error kind. +/// Converts [`sqlx::Error`] to [`EtlError`] with appropriate error kind. /// /// Maps database errors to [`ErrorKind::QueryFailed`], I/O errors to [`ErrorKind::IoError`], /// and connection pool errors to [`ErrorKind::ConnectionFailed`]. -impl From for ETLError { - fn from(err: sqlx::Error) -> ETLError { +impl From for EtlError { + fn from(err: sqlx::Error) -> EtlError { let kind = match &err { sqlx::Error::Database(_) => ErrorKind::QueryFailed, sqlx::Error::Io(_) => ErrorKind::IoError, @@ -547,7 +526,7 @@ impl From for ETLError { _ => ErrorKind::QueryFailed, }; - ETLError { + EtlError { repr: ErrorRepr::WithDescriptionAndDetail( kind, "Database operation failed", @@ -559,12 +538,12 @@ impl From for ETLError { // BigQuery error conversions (feature-gated) -/// Converts [`gcp_bigquery_client::error::BQError`] to [`ETLError`] with appropriate error kind. +/// Converts [`gcp_bigquery_client::error::BQError`] to [`EtlError`] with appropriate error kind. /// /// Maps errors based on their specific type for better error classification and handling. #[cfg(feature = "bigquery")] -impl From for ETLError { - fn from(err: gcp_bigquery_client::error::BQError) -> ETLError { +impl From for EtlError { + fn from(err: gcp_bigquery_client::error::BQError) -> EtlError { use gcp_bigquery_client::error::BQError; let (kind, description) = match &err { @@ -637,17 +616,17 @@ impl From for ETLError { } }; - ETLError { + EtlError { repr: ErrorRepr::WithDescriptionAndDetail(kind, description, err.to_string()), } } } -/// Converts BigQuery row errors to [`ETLError`] with [`ErrorKind::DestinationError`]. +/// Converts BigQuery row errors to [`EtlError`] with [`ErrorKind::DestinationError`]. #[cfg(feature = "bigquery")] -impl From for ETLError { - fn from(err: gcp_bigquery_client::google::cloud::bigquery::storage::v1::RowError) -> ETLError { - ETLError { +impl From for EtlError { + fn from(err: gcp_bigquery_client::google::cloud::bigquery::storage::v1::RowError) -> EtlError { + EtlError { repr: ErrorRepr::WithDescriptionAndDetail( ErrorKind::DestinationError, "BigQuery row error", @@ -664,7 +643,7 @@ mod tests { #[test] fn test_simple_error_creation() { - let err = ETLError::from((ErrorKind::ConnectionFailed, "Database connection failed")); + let err = EtlError::from((ErrorKind::ConnectionFailed, "Database connection failed")); assert_eq!(err.kind(), ErrorKind::ConnectionFailed); assert_eq!(err.detail(), None); assert_eq!(err.kinds(), vec![ErrorKind::ConnectionFailed]); @@ -672,7 +651,7 @@ mod tests { #[test] fn test_error_with_detail() { - let err = ETLError::from(( + let err = EtlError::from(( ErrorKind::QueryFailed, "SQL query execution failed", "Table 'users' doesn't exist".to_string(), @@ -685,11 +664,11 @@ mod tests { #[test] fn test_multiple_errors() { let errors = vec![ - ETLError::from((ErrorKind::ValidationError, "Invalid schema")), - ETLError::from((ErrorKind::ConversionError, "Type mismatch")), - ETLError::from((ErrorKind::IoError, "Connection timeout")), + EtlError::from((ErrorKind::ValidationError, "Invalid schema")), + EtlError::from((ErrorKind::ConversionError, "Type mismatch")), + EtlError::from((ErrorKind::IoError, "Connection timeout")), ]; - let multi_err = ETLError::many(errors); + let multi_err = EtlError::many(errors); assert_eq!(multi_err.kind(), ErrorKind::ValidationError); assert_eq!( @@ -706,14 +685,14 @@ mod tests { #[test] fn test_multiple_errors_with_detail() { let errors = vec![ - ETLError::from(( + EtlError::from(( ErrorKind::ValidationError, "Invalid schema", "Missing required field".to_string(), )), - ETLError::from((ErrorKind::ConversionError, "Type mismatch")), + EtlError::from((ErrorKind::ConversionError, "Type mismatch")), ]; - let multi_err = ETLError::many(errors); + let multi_err = EtlError::many(errors); assert_eq!(multi_err.detail(), Some("Missing required field")); } @@ -721,16 +700,16 @@ mod tests { #[test] fn test_from_vector() { let errors = vec![ - ETLError::from((ErrorKind::ValidationError, "Error 1")), - ETLError::from((ErrorKind::ConversionError, "Error 2")), + EtlError::from((ErrorKind::ValidationError, "Error 1")), + EtlError::from((ErrorKind::ConversionError, "Error 2")), ]; - let multi_err = ETLError::from(errors); + let multi_err = EtlError::from(errors); assert_eq!(multi_err.kinds().len(), 2); } #[test] fn test_empty_multiple_errors() { - let multi_err = ETLError::many(vec![]); + let multi_err = EtlError::many(vec![]); assert_eq!(multi_err.kind(), ErrorKind::Unknown); assert_eq!(multi_err.kinds(), vec![]); assert_eq!(multi_err.detail(), None); @@ -738,9 +717,9 @@ mod tests { #[test] fn test_error_equality() { - let err1 = ETLError::from((ErrorKind::ConnectionFailed, "Connection failed")); - let err2 = ETLError::from((ErrorKind::ConnectionFailed, "Connection failed")); - let err3 = ETLError::from((ErrorKind::QueryFailed, "Query failed")); + let err1 = EtlError::from((ErrorKind::ConnectionFailed, "Connection failed")); + let err2 = EtlError::from((ErrorKind::ConnectionFailed, "Connection failed")); + let err3 = EtlError::from((ErrorKind::QueryFailed, "Query failed")); assert_eq!(err1, err2); assert_ne!(err1, err3); @@ -748,7 +727,7 @@ mod tests { #[test] fn test_error_display() { - let err = ETLError::from((ErrorKind::ConnectionFailed, "Database connection failed")); + let err = EtlError::from((ErrorKind::ConnectionFailed, "Database connection failed")); let display_str = format!("{err}"); assert!(display_str.contains("ConnectionFailed")); assert!(display_str.contains("Database connection failed")); @@ -756,7 +735,7 @@ mod tests { #[test] fn test_error_display_with_detail() { - let err = ETLError::from(( + let err = EtlError::from(( ErrorKind::QueryFailed, "SQL query failed", "Invalid table name".to_string(), @@ -770,10 +749,10 @@ mod tests { #[test] fn test_multiple_errors_display() { let errors = vec![ - ETLError::from((ErrorKind::ValidationError, "Invalid schema")), - ETLError::from((ErrorKind::ConversionError, "Type mismatch")), + EtlError::from((ErrorKind::ValidationError, "Invalid schema")), + EtlError::from((ErrorKind::ConversionError, "Type mismatch")), ]; - let multi_err = ETLError::many(errors); + let multi_err = EtlError::many(errors); let display_str = format!("{multi_err}"); assert!(display_str.contains("Multiple errors")); assert!(display_str.contains("2 total")); @@ -796,11 +775,11 @@ mod tests { #[test] fn test_bail_macro() { - fn test_function() -> ETLResult { + fn test_function() -> EtlResult { bail!(ErrorKind::ValidationError, "Test error"); } - fn test_function_with_detail() -> ETLResult { + fn test_function_with_detail() -> EtlResult { bail!( ErrorKind::ConversionError, "Test error", @@ -823,16 +802,16 @@ mod tests { #[test] fn test_nested_multiple_errors() { let inner_errors = vec![ - ETLError::from((ErrorKind::ConversionError, "Inner error 1")), - ETLError::from((ErrorKind::ValidationError, "Inner error 2")), + EtlError::from((ErrorKind::ConversionError, "Inner error 1")), + EtlError::from((ErrorKind::ValidationError, "Inner error 2")), ]; - let inner_multi = ETLError::many(inner_errors); + let inner_multi = EtlError::many(inner_errors); let outer_errors = vec![ inner_multi, - ETLError::from((ErrorKind::IoError, "Outer error")), + EtlError::from((ErrorKind::IoError, "Outer error")), ]; - let outer_multi = ETLError::many(outer_errors); + let outer_multi = EtlError::many(outer_errors); let kinds = outer_multi.kinds(); assert_eq!(kinds.len(), 3); @@ -848,12 +827,12 @@ mod tests { // Test authentication error let auth_err = BQError::NoToken; - let etl_err = ETLError::from(auth_err); + let etl_err = EtlError::from(auth_err); assert_eq!(etl_err.kind(), ErrorKind::AuthenticationError); // Test invalid data error let invalid_col_err = BQError::InvalidColumnIndex { col_index: 5 }; - let etl_err = ETLError::from(invalid_col_err); + let etl_err = EtlError::from(invalid_col_err); assert_eq!(etl_err.kind(), ErrorKind::InvalidData); // Test conversion error @@ -862,7 +841,7 @@ mod tests { col_type: "STRING".to_string(), type_requested: "INTEGER".to_string(), }; - let etl_err = ETLError::from(type_err); + let etl_err = EtlError::from(type_err); assert_eq!(etl_err.kind(), ErrorKind::ConversionError); } @@ -872,7 +851,7 @@ mod tests { // by verifying that we can convert standard IO errors to ETL errors let io_err = std::io::Error::new(std::io::ErrorKind::ConnectionRefused, "Connection refused"); - let etl_err = ETLError::from(io_err); + let etl_err = EtlError::from(io_err); assert_eq!(etl_err.kind(), ErrorKind::IoError); assert!(etl_err.detail().unwrap().contains("Connection refused")); @@ -885,13 +864,13 @@ mod tests { fn test_json_error_classification() { // Test syntax error during deserialization let json_err = serde_json::from_str::("invalid json").unwrap_err(); - let etl_err = ETLError::from(json_err); + let etl_err = EtlError::from(json_err); assert_eq!(etl_err.kind(), ErrorKind::DeserializationError); assert!(etl_err.detail().unwrap().contains("expected")); // Test data error during deserialization let json_err = serde_json::from_str::("\"not_a_bool\"").unwrap_err(); - let etl_err = ETLError::from(json_err); + let etl_err = EtlError::from(json_err); assert_eq!(etl_err.kind(), ErrorKind::DeserializationError); assert!(etl_err.detail().is_some()); } diff --git a/etl/src/macros.rs b/etl/src/macros.rs index 3325977c4..fae64a934 100644 --- a/etl/src/macros.rs +++ b/etl/src/macros.rs @@ -1,10 +1,10 @@ #[macro_export] macro_rules! etl_error { ($kind:expr, $desc:expr) => { - ETLError::from(($kind, $desc)) + EtlError::from(($kind, $desc)) }; ($kind:expr, $desc:expr, $detail:expr) => { - ETLError::from(($kind, $desc, $detail.to_string())) + EtlError::from(($kind, $desc, $detail.to_string())) }; } diff --git a/etl/src/pipeline.rs b/etl/src/pipeline.rs index f8afe6492..09b0c7038 100644 --- a/etl/src/pipeline.rs +++ b/etl/src/pipeline.rs @@ -6,7 +6,7 @@ use tracing::{error, info}; use crate::bail; use crate::concurrency::shutdown::{ShutdownTx, create_shutdown_channel}; use crate::destination::base::Destination; -use crate::error::{ETLError, ETLResult, ErrorKind}; +use crate::error::{EtlError, EtlResult, ErrorKind}; use crate::replication::client::PgReplicationClient; use crate::schema::cache::SchemaCache; use crate::state::store::base::StateStore; @@ -69,7 +69,7 @@ where self.shutdown_tx.clone() } - pub async fn start(&mut self) -> ETLResult<()> { + pub async fn start(&mut self) -> EtlResult<()> { info!( "starting pipeline for publication '{}' with id {}", self.config.publication_name, self.id @@ -119,7 +119,7 @@ where Ok(()) } - async fn prepare_schema_cache(&self, schema_cache: &SchemaCache) -> ETLResult<()> { + async fn prepare_schema_cache(&self, schema_cache: &SchemaCache) -> EtlResult<()> { // We initialize the schema cache, which is local to a pipeline, and we try to load existing // schemas that were previously stored at the destination (if any). let table_schemas = self.destination.load_table_schemas().await?; @@ -131,7 +131,7 @@ where async fn initialize_table_states( &self, replication_client: &PgReplicationClient, - ) -> ETLResult<()> { + ) -> EtlResult<()> { info!( "initializing table states for tables in publication '{}'", self.config.publication_name @@ -174,7 +174,7 @@ where Ok(()) } - pub async fn wait(self) -> ETLResult<()> { + pub async fn wait(self) -> EtlResult<()> { let PipelineWorkers::Started { apply_worker, pool } = self.workers else { info!("pipeline was not started, nothing to wait for"); return Ok(()); @@ -236,7 +236,7 @@ where info!("shut down signal successfully sent to all workers"); } - pub async fn shutdown_and_wait(self) -> ETLResult<()> { + pub async fn shutdown_and_wait(self) -> EtlResult<()> { self.shutdown(); self.wait().await } diff --git a/etl/src/replication/apply.rs b/etl/src/replication/apply.rs index 39b307c2b..13d0c5982 100644 --- a/etl/src/replication/apply.rs +++ b/etl/src/replication/apply.rs @@ -15,7 +15,7 @@ use crate::concurrency::shutdown::ShutdownRx; use crate::concurrency::signal::SignalRx; use crate::conversions::event::{Event, EventType, convert_message_to_event}; use crate::destination::base::Destination; -use crate::error::{ETLError, ETLResult, ErrorKind}; +use crate::error::{EtlError, EtlResult, ErrorKind}; use crate::pipeline::PipelineId; use crate::replication::client::PgReplicationClient; use crate::replication::slot::get_slot_name; @@ -34,21 +34,21 @@ pub enum ApplyLoopResult { } pub trait ApplyLoopHook { - fn before_loop(&self, start_lsn: PgLsn) -> impl Future> + Send; + fn before_loop(&self, start_lsn: PgLsn) -> impl Future> + Send; fn process_syncing_tables( &self, current_lsn: PgLsn, update_state: bool, - ) -> impl Future> + Send; + ) -> impl Future> + Send; - fn skip_table(&self, table_id: TableId) -> impl Future> + Send; + fn skip_table(&self, table_id: TableId) -> impl Future> + Send; fn should_apply_changes( &self, table_id: TableId, remote_final_lsn: PgLsn, - ) -> impl Future> + Send; + ) -> impl Future> + Send; fn worker_type(&self) -> WorkerType; } @@ -201,7 +201,7 @@ pub async fn start_apply_loop( hook: T, mut shutdown_rx: ShutdownRx, mut force_syncing_tables_rx: Option, -) -> ETLResult +) -> EtlResult where D: Destination + Clone + Send + 'static, T: ApplyLoopHook, @@ -328,7 +328,7 @@ async fn handle_replication_message_batch( hook: &T, max_batch_size: usize, max_batch_fill_duration: Duration, -) -> ETLResult +) -> EtlResult where D: Destination + Clone + Send + 'static, T: ApplyLoopHook, @@ -363,7 +363,7 @@ async fn try_send_batch( hook: &T, max_batch_size: usize, max_batch_fill_duration: Duration, -) -> ETLResult +) -> EtlResult where D: Destination + Clone + Send + 'static, T: ApplyLoopHook, @@ -445,7 +445,7 @@ async fn handle_replication_message( message: ReplicationMessage, schema_cache: &SchemaCache, hook: &T, -) -> ETLResult +) -> EtlResult where T: ApplyLoopHook, { @@ -501,7 +501,7 @@ async fn handle_logical_replication_message( message: LogicalReplicationMessage, schema_cache: &SchemaCache, hook: &T, -) -> ETLResult +) -> EtlResult where T: ApplyLoopHook, { @@ -548,7 +548,7 @@ where } #[allow(clippy::result_large_err)] -fn get_commit_lsn(state: &ApplyLoopState, message: &LogicalReplicationMessage) -> ETLResult { +fn get_commit_lsn(state: &ApplyLoopState, message: &LogicalReplicationMessage) -> EtlResult { // If we are in a `Begin` message, the `commit_lsn` is the `final_lsn` of the payload, in all the // other cases we read the `remote_final_lsn` which should be always set in case we are within or // at the end of a transaction (meaning that the event type is different from `Begin`). @@ -569,14 +569,13 @@ async fn handle_begin_message( state: &mut ApplyLoopState, event: Event, message: &protocol::BeginBody, -) -> ETLResult { +) -> EtlResult { let EventType::Begin = event.event_type() else { bail!( ErrorKind::ValidationError, "Invalid event", format!( - "An invalid event {:?} was received (expected {:?})", - event, + "An invalid event {event:?} was received (expected {:?})", EventType::Begin ) ); @@ -600,7 +599,7 @@ async fn handle_commit_message( event: Event, message: &protocol::CommitBody, hook: &T, -) -> ETLResult +) -> EtlResult where T: ApplyLoopHook, { @@ -609,8 +608,7 @@ where ErrorKind::ValidationError, "Invalid event", format!( - "An invalid event {:?} was received (expected {:?})", - event, + "An invalid event {event:?} was received (expected {:?})", EventType::Commit ) ); @@ -681,7 +679,7 @@ async fn handle_relation_message( message: &protocol::RelationBody, schema_cache: &SchemaCache, hook: &T, -) -> ETLResult +) -> EtlResult where T: ApplyLoopHook, { @@ -690,8 +688,7 @@ where ErrorKind::ValidationError, "Invalid event", format!( - "An invalid event {:?} was received (expected {:?})", - event, + "An invalid event {event:?} was received (expected {:?})", EventType::Relation ) ); @@ -753,7 +750,7 @@ async fn handle_insert_message( event: Event, message: &protocol::InsertBody, hook: &T, -) -> ETLResult +) -> EtlResult where T: ApplyLoopHook, { @@ -762,8 +759,7 @@ where ErrorKind::ValidationError, "Invalid event", format!( - "An invalid event {:?} was received (expected {:?})", - event, + "An invalid event {event:?} was received (expected {:?})", EventType::Insert ) ); @@ -797,7 +793,7 @@ async fn handle_update_message( event: Event, message: &protocol::UpdateBody, hook: &T, -) -> ETLResult +) -> EtlResult where T: ApplyLoopHook, { @@ -806,8 +802,7 @@ where ErrorKind::ValidationError, "Invalid event", format!( - "An invalid event {:?} was received (expected {:?})", - event, + "An invalid event {event:?} was received (expected {:?})", EventType::Update ) ); @@ -841,7 +836,7 @@ async fn handle_delete_message( event: Event, message: &protocol::DeleteBody, hook: &T, -) -> ETLResult +) -> EtlResult where T: ApplyLoopHook, { @@ -850,8 +845,7 @@ where ErrorKind::ValidationError, "Invalid event", format!( - "An invalid event {:?} was received (expected {:?})", - event, + "An invalid event {event:?} was received (expected {:?})", EventType::Delete ) ); @@ -885,7 +879,7 @@ async fn handle_truncate_message( event: Event, message: &protocol::TruncateBody, hook: &T, -) -> ETLResult +) -> EtlResult where T: ApplyLoopHook, { @@ -894,8 +888,7 @@ where ErrorKind::ValidationError, "Invalid event", format!( - "An invalid event {:?} was received (expected {:?})", - event, + "An invalid event {event:?} was received (expected {:?})", EventType::Truncate ) ); diff --git a/etl/src/replication/client.rs b/etl/src/replication/client.rs index be82aa6d9..00d001928 100644 --- a/etl/src/replication/client.rs +++ b/etl/src/replication/client.rs @@ -1,4 +1,4 @@ -use crate::error::{ETLError, ETLResult, ErrorKind}; +use crate::error::{EtlError, EtlResult, ErrorKind}; use crate::{bail, etl_error}; use config::shared::{IntoConnectOptions, PgConnectionConfig}; use pg_escape::{quote_identifier, quote_literal}; @@ -84,7 +84,7 @@ impl PgReplicationSlotTransaction { /// /// The transaction is started with a repeatable read isolation level and uses the /// snapshot associated with the provided slot. - async fn new(client: PgReplicationClient) -> ETLResult { + async fn new(client: PgReplicationClient) -> EtlResult { client.begin_tx().await?; Ok(Self { client }) @@ -98,7 +98,7 @@ impl PgReplicationSlotTransaction { &self, table_ids: &[TableId], publication_name: Option<&str>, - ) -> ETLResult> { + ) -> EtlResult> { self.client .get_table_schemas(table_ids, publication_name) .await @@ -112,7 +112,7 @@ impl PgReplicationSlotTransaction { &self, table_id: TableId, publication: Option<&str>, - ) -> ETLResult { + ) -> EtlResult { self.client.get_table_schema(table_id, publication).await } @@ -123,19 +123,19 @@ impl PgReplicationSlotTransaction { &self, table_id: TableId, column_schemas: &[ColumnSchema], - ) -> ETLResult { + ) -> EtlResult { self.client .get_table_copy_stream(table_id, column_schemas) .await } /// Commits the current transaction. - pub async fn commit(self) -> ETLResult<()> { + pub async fn commit(self) -> EtlResult<()> { self.client.commit_tx().await } /// Rolls back the current transaction. - pub async fn rollback(self) -> ETLResult<()> { + pub async fn rollback(self) -> EtlResult<()> { self.client.rollback_tx().await } } @@ -154,7 +154,7 @@ impl PgReplicationClient { /// supplied [`PgConnectionConfig`]. /// /// The connection is configured for logical replication mode - pub async fn connect(pg_connection_config: PgConnectionConfig) -> ETLResult { + pub async fn connect(pg_connection_config: PgConnectionConfig) -> EtlResult { match pg_connection_config.tls.enabled { true => PgReplicationClient::connect_tls(pg_connection_config).await, false => PgReplicationClient::connect_no_tls(pg_connection_config).await, @@ -164,7 +164,7 @@ impl PgReplicationClient { /// Establishes a connection to PostgreSQL without TLS encryption. /// /// The connection is configured for logical replication mode. - async fn connect_no_tls(pg_connection_config: PgConnectionConfig) -> ETLResult { + async fn connect_no_tls(pg_connection_config: PgConnectionConfig) -> EtlResult { let mut config: Config = pg_connection_config.clone().with_db(); config.replication_mode(ReplicationMode::Logical); @@ -181,7 +181,7 @@ impl PgReplicationClient { /// Establishes a TLS-encrypted connection to PostgreSQL. /// /// The connection is configured for logical replication mode - async fn connect_tls(pg_connection_config: PgConnectionConfig) -> ETLResult { + async fn connect_tls(pg_connection_config: PgConnectionConfig) -> EtlResult { let mut config: Config = pg_connection_config.clone().with_db(); config.replication_mode(ReplicationMode::Logical); @@ -214,7 +214,7 @@ impl PgReplicationClient { pub async fn create_slot_with_transaction( &self, slot_name: &str, - ) -> ETLResult<(PgReplicationSlotTransaction, CreateSlotResult)> { + ) -> EtlResult<(PgReplicationSlotTransaction, CreateSlotResult)> { // TODO: check if we want to consume the client and return it on commit to avoid any other // operations on a connection that has started a transaction. let transaction = PgReplicationSlotTransaction::new(self.clone()).await?; @@ -224,14 +224,14 @@ impl PgReplicationClient { } /// Creates a new logical replication slot with the specified name and no snapshot. - pub async fn create_slot(&self, slot_name: &str) -> ETLResult { + pub async fn create_slot(&self, slot_name: &str) -> EtlResult { self.create_slot_internal(slot_name, false).await } /// Gets the slot by `slot_name`. /// /// Returns an error in case of failure or missing slot. - pub async fn get_slot(&self, slot_name: &str) -> ETLResult { + pub async fn get_slot(&self, slot_name: &str) -> EtlResult { let query = format!( r#"select confirmed_flush_lsn from pg_replication_slots where slot_name = {};"#, quote_literal(slot_name) @@ -270,7 +270,7 @@ impl PgReplicationClient { /// - A boolean indicating whether the slot was created (true) or already existed (false) /// - The slot result containing either the confirmed_flush_lsn (for existing slots) /// or the consistent_point (for newly created slots) - pub async fn get_or_create_slot(&self, slot_name: &str) -> ETLResult { + pub async fn get_or_create_slot(&self, slot_name: &str) -> EtlResult { match self.get_slot(slot_name).await { Ok(slot) => { info!("using existing replication slot '{}'", slot_name); @@ -291,7 +291,7 @@ impl PgReplicationClient { /// Deletes a replication slot with the specified name. /// /// Returns an error if the slot doesn't exist or if there are any issues with the deletion. - pub async fn delete_slot(&self, slot_name: &str) -> ETLResult<()> { + pub async fn delete_slot(&self, slot_name: &str) -> EtlResult<()> { info!("deleting replication slot '{}'", slot_name); // Do not convert the query or the options to lowercase, see comment in `create_slot_internal`. let query = format!( @@ -332,7 +332,7 @@ impl PgReplicationClient { } /// Checks if a publication with the given name exists. - pub async fn publication_exists(&self, publication: &str) -> ETLResult { + pub async fn publication_exists(&self, publication: &str) -> EtlResult { let publication_exists_query = format!( "select 1 as exists from pg_publication where pubname = {};", quote_literal(publication) @@ -350,7 +350,7 @@ impl PgReplicationClient { pub async fn get_publication_table_names( &self, publication_name: &str, - ) -> ETLResult> { + ) -> EtlResult> { let publication_query = format!( "select schemaname, tablename from pg_publication_tables where pubname = {};", quote_literal(publication_name) @@ -377,7 +377,7 @@ impl PgReplicationClient { pub async fn get_publication_table_ids( &self, publication_name: &str, - ) -> ETLResult> { + ) -> EtlResult> { let publication_query = format!( "select c.oid from pg_publication_tables pt join pg_class c on c.relname = pt.tablename @@ -406,7 +406,7 @@ impl PgReplicationClient { publication_name: &str, slot_name: &str, start_lsn: PgLsn, - ) -> ETLResult { + ) -> EtlResult { info!( "starting logical replication from publication '{}' with slot named '{}' at lsn {}", publication_name, slot_name, start_lsn @@ -435,7 +435,7 @@ impl PgReplicationClient { /// /// The transaction doesn't make any assumptions about the snapshot in use, since this is a /// concern of the statements issued within the transaction. - async fn begin_tx(&self) -> ETLResult<()> { + async fn begin_tx(&self) -> EtlResult<()> { self.client .simple_query("begin read only isolation level repeatable read;") .await?; @@ -444,14 +444,14 @@ impl PgReplicationClient { } /// Commits the current transaction. - async fn commit_tx(&self) -> ETLResult<()> { + async fn commit_tx(&self) -> EtlResult<()> { self.client.simple_query("commit;").await?; Ok(()) } /// Rolls back the current transaction. - async fn rollback_tx(&self) -> ETLResult<()> { + async fn rollback_tx(&self) -> EtlResult<()> { self.client.simple_query("rollback;").await?; Ok(()) @@ -464,7 +464,7 @@ impl PgReplicationClient { &self, slot_name: &str, use_snapshot: bool, - ) -> ETLResult { + ) -> EtlResult { // Do not convert the query or the options to lowercase, since the lexer for // replication commands (repl_scanner.l) in Postgres code expects the commands // in uppercase. This probably should be fixed in upstream, but for now we will @@ -526,7 +526,7 @@ impl PgReplicationClient { &self, table_ids: &[TableId], publication_name: Option<&str>, - ) -> ETLResult> { + ) -> EtlResult> { let mut table_schemas = HashMap::new(); // TODO: consider if we want to fail when at least one table was missing or not. @@ -557,7 +557,7 @@ impl PgReplicationClient { &self, table_id: TableId, publication: Option<&str>, - ) -> ETLResult { + ) -> EtlResult { let table_name = self.get_table_name(table_id).await?; let column_schemas = self.get_column_schemas(table_id, publication).await?; @@ -571,7 +571,7 @@ impl PgReplicationClient { /// Loads the table name and schema information for a given table OID. /// /// Returns a `TableName` containing both the schema and table name. - async fn get_table_name(&self, table_id: TableId) -> ETLResult { + async fn get_table_name(&self, table_id: TableId) -> EtlResult { let table_info_query = format!( "select n.nspname as schema_name, c.relname as table_name from pg_class c @@ -608,7 +608,7 @@ impl PgReplicationClient { &self, table_id: TableId, publication: Option<&str>, - ) -> ETLResult> { + ) -> EtlResult> { let (pub_cte, pub_pred) = if let Some(publication) = publication { ( format!( @@ -688,7 +688,7 @@ impl PgReplicationClient { &self, table_id: TableId, column_schemas: &[ColumnSchema], - ) -> ETLResult { + ) -> EtlResult { let column_list = column_schemas .iter() .map(|col| quote_identifier(&col.name)) @@ -716,7 +716,7 @@ impl PgReplicationClient { row: &SimpleQueryRow, column_name: &str, table_name: &str, - ) -> ETLResult + ) -> EtlResult where T::Err: fmt::Debug, { diff --git a/etl/src/replication/common.rs b/etl/src/replication/common.rs index a91b1111d..27fb68c94 100644 --- a/etl/src/replication/common.rs +++ b/etl/src/replication/common.rs @@ -1,7 +1,7 @@ use postgres::schema::TableId; use std::collections::HashMap; -use crate::error::ETLResult; +use crate::error::EtlResult; use crate::state::store::base::StateStore; use crate::state::table::TableReplicationPhase; @@ -12,7 +12,7 @@ use crate::state::table::TableReplicationPhase; pub async fn get_table_replication_states( state_store: &S, done: bool, -) -> ETLResult> +) -> EtlResult> where S: StateStore + Clone + Send + Sync + 'static, { diff --git a/etl/src/replication/slot.rs b/etl/src/replication/slot.rs index 30f099b89..29b2cff29 100644 --- a/etl/src/replication/slot.rs +++ b/etl/src/replication/slot.rs @@ -1,6 +1,6 @@ use crate::bail; -use crate::error::ETLError; -use crate::error::{ETLResult, ErrorKind}; +use crate::error::EtlError; +use crate::error::{EtlResult, ErrorKind}; use crate::pipeline::PipelineId; use crate::workers::base::WorkerType; @@ -12,7 +12,7 @@ const APPLY_WORKER_PREFIX: &str = "supabase_etl_apply"; const TABLE_SYNC_PREFIX: &str = "supabase_etl_table_sync"; /// Generates a replication slot name. -pub fn get_slot_name(pipeline_id: PipelineId, worker_type: WorkerType) -> ETLResult { +pub fn get_slot_name(pipeline_id: PipelineId, worker_type: WorkerType) -> EtlResult { let slot_name = match worker_type { WorkerType::Apply => { format!("{APPLY_WORKER_PREFIX}_{pipeline_id}") diff --git a/etl/src/replication/stream.rs b/etl/src/replication/stream.rs index b5a29506f..e179b5428 100644 --- a/etl/src/replication/stream.rs +++ b/etl/src/replication/stream.rs @@ -12,8 +12,8 @@ use tokio_postgres::types::PgLsn; use tracing::debug; use crate::conversions::table_row::{TableRow, TableRowConverter}; -use crate::error::ETLError; -use crate::error::{ETLResult, ErrorKind}; +use crate::error::EtlError; +use crate::error::{EtlResult, ErrorKind}; use crate::etl_error; /// The amount of milliseconds between two consecutive status updates in case no forced update @@ -47,7 +47,7 @@ impl<'a> TableCopyStream<'a> { } impl<'a> Stream for TableCopyStream<'a> { - type Item = ETLResult; + type Item = EtlResult; fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { let this = self.project(); @@ -99,7 +99,7 @@ impl EventsStream { flush_lsn: PgLsn, apply_lsn: PgLsn, force: bool, - ) -> ETLResult<()> { + ) -> EtlResult<()> { let this = self.project(); // If we are not forced to send an update, we can willingly do so based on a set of conditions. @@ -160,7 +160,7 @@ impl EventsStream { } impl Stream for EventsStream { - type Item = ETLResult>; + type Item = EtlResult>; fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { let this = self.project(); diff --git a/etl/src/replication/table_sync.rs b/etl/src/replication/table_sync.rs index 3ebd91510..f33974e09 100644 --- a/etl/src/replication/table_sync.rs +++ b/etl/src/replication/table_sync.rs @@ -13,7 +13,7 @@ use crate::concurrency::shutdown::{ShutdownResult, ShutdownRx}; use crate::concurrency::signal::SignalTx; use crate::concurrency::stream::BatchStream; use crate::destination::base::Destination; -use crate::error::{ETLError, ETLResult, ErrorKind}; +use crate::error::{EtlError, EtlResult, ErrorKind}; use crate::pipeline::PipelineId; use crate::replication::client::PgReplicationClient; use crate::replication::slot::get_slot_name; @@ -48,7 +48,7 @@ pub async fn start_table_sync( destination: D, shutdown_rx: ShutdownRx, force_syncing_tables_tx: SignalTx, -) -> ETLResult +) -> EtlResult where S: StateStore + Clone + Send + 'static, D: Destination + Clone + Send + 'static, diff --git a/etl/src/state/store/base.rs b/etl/src/state/store/base.rs index 9c05e6a30..1c24829b3 100644 --- a/etl/src/state/store/base.rs +++ b/etl/src/state/store/base.rs @@ -1,7 +1,7 @@ use postgres::schema::TableId; use std::{collections::HashMap, future::Future}; -use crate::error::ETLResult; +use crate::error::EtlResult; use crate::state::table::TableReplicationPhase; /// This trait represents a state store for the replication state of all tables. @@ -14,20 +14,20 @@ pub trait StateStore { fn get_table_replication_state( &self, table_id: TableId, - ) -> impl Future>> + Send; + ) -> impl Future>> + Send; /// Returns the table replication states for all the tables from the cache. /// Does not read from the persistent store. fn get_table_replication_states( &self, - ) -> impl Future>> + Send; + ) -> impl Future>> + Send; /// Loads the table replication states from the persistent state into the cache. /// This should be called once at program start to load the state into the cache /// and then use only the `get_X` methods to access the state. Updating the state /// by calling the `update_table_replication_state` updates in both the cache and /// the persistent store, so no need to ever load the state again. - fn load_table_replication_states(&self) -> impl Future> + Send; + fn load_table_replication_states(&self) -> impl Future> + Send; /// Updates the table replicate state for a table with `table_id` in both the cache and /// the persistent store. @@ -35,5 +35,5 @@ pub trait StateStore { &self, table_id: TableId, state: TableReplicationPhase, - ) -> impl Future> + Send; + ) -> impl Future> + Send; } diff --git a/etl/src/state/store/memory.rs b/etl/src/state/store/memory.rs index 74d1218eb..316a8a1f4 100644 --- a/etl/src/state/store/memory.rs +++ b/etl/src/state/store/memory.rs @@ -3,7 +3,7 @@ use std::collections::HashMap; use std::sync::Arc; use tokio::sync::Mutex; -use crate::error::ETLResult; +use crate::error::EtlResult; use crate::state::store::base::StateStore; use crate::state::table::TableReplicationPhase; @@ -39,7 +39,7 @@ impl StateStore for MemoryStateStore { async fn get_table_replication_state( &self, table_id: TableId, - ) -> ETLResult> { + ) -> EtlResult> { let inner = self.inner.lock().await; Ok(inner.table_replication_states.get(&table_id).cloned()) @@ -47,13 +47,13 @@ impl StateStore for MemoryStateStore { async fn get_table_replication_states( &self, - ) -> ETLResult> { + ) -> EtlResult> { let inner = self.inner.lock().await; Ok(inner.table_replication_states.clone()) } - async fn load_table_replication_states(&self) -> ETLResult { + async fn load_table_replication_states(&self) -> EtlResult { let inner = self.inner.lock().await; Ok(inner.table_replication_states.len()) @@ -63,7 +63,7 @@ impl StateStore for MemoryStateStore { &self, table_id: TableId, state: TableReplicationPhase, - ) -> ETLResult<()> { + ) -> EtlResult<()> { let mut inner = self.inner.lock().await; inner.table_replication_states.insert(table_id, state); diff --git a/etl/src/state/store/notify.rs b/etl/src/state/store/notify.rs index befcd8bfd..299d7be2d 100644 --- a/etl/src/state/store/notify.rs +++ b/etl/src/state/store/notify.rs @@ -6,7 +6,7 @@ use tokio::{ sync::{Notify, RwLock}, }; -use crate::error::ETLResult; +use crate::error::EtlResult; use crate::state::{ store::base::StateStore, table::{TableReplicationPhase, TableReplicationPhaseType}, @@ -106,7 +106,7 @@ impl StateStore for NotifyingStateStore { async fn get_table_replication_state( &self, table_id: TableId, - ) -> ETLResult> { + ) -> EtlResult> { let inner = self.inner.read().await; let result = Ok(inner.table_replication_states.get(&table_id).cloned()); @@ -119,7 +119,7 @@ impl StateStore for NotifyingStateStore { async fn get_table_replication_states( &self, - ) -> ETLResult> { + ) -> EtlResult> { let inner = self.inner.read().await; let result = Ok(inner.table_replication_states.clone()); @@ -130,7 +130,7 @@ impl StateStore for NotifyingStateStore { result } - async fn load_table_replication_states(&self) -> ETLResult { + async fn load_table_replication_states(&self) -> EtlResult { let inner = self.inner.read().await; let result = Ok(inner.table_replication_states.clone()); @@ -145,7 +145,7 @@ impl StateStore for NotifyingStateStore { &self, table_id: TableId, state: TableReplicationPhase, - ) -> ETLResult<()> { + ) -> EtlResult<()> { let mut inner = self.inner.write().await; inner.table_replication_states.insert(table_id, state); inner.check_conditions().await; diff --git a/etl/src/state/store/postgres.rs b/etl/src/state/store/postgres.rs index c479162df..1eaca3911 100644 --- a/etl/src/state/store/postgres.rs +++ b/etl/src/state/store/postgres.rs @@ -11,7 +11,7 @@ use tokio::sync::Mutex; use tokio_postgres::types::PgLsn; use tracing::{debug, info}; -use crate::error::{ETLError, ETLResult, ErrorKind}; +use crate::error::{EtlError, EtlResult, ErrorKind}; use crate::pipeline::PipelineId; use crate::state::store::base::StateStore; use crate::state::table::TableReplicationPhase; @@ -20,7 +20,7 @@ use crate::{bail, etl_error}; const NUM_POOL_CONNECTIONS: u32 = 1; impl TryFrom for (TableReplicationState, Option) { - type Error = ETLError; + type Error = EtlError; fn try_from(value: TableReplicationPhase) -> Result { Ok(match value { @@ -108,7 +108,7 @@ impl PostgresStateStore { &self, state: &TableReplicationState, sync_done_lsn: Option, - ) -> ETLResult { + ) -> EtlResult { Ok(match state { TableReplicationState::Init => TableReplicationPhase::Init, TableReplicationState::DataSync => TableReplicationPhase::DataSync, @@ -143,19 +143,19 @@ impl StateStore for PostgresStateStore { async fn get_table_replication_state( &self, table_id: TableId, - ) -> ETLResult> { + ) -> EtlResult> { let inner = self.inner.lock().await; Ok(inner.table_states.get(&table_id).cloned()) } async fn get_table_replication_states( &self, - ) -> ETLResult> { + ) -> EtlResult> { let inner = self.inner.lock().await; Ok(inner.table_states.clone()) } - async fn load_table_replication_states(&self) -> ETLResult { + async fn load_table_replication_states(&self) -> EtlResult { debug!("loading table replication states from postgres state store"); let pool = self.connect_to_source().await?; @@ -184,7 +184,7 @@ impl StateStore for PostgresStateStore { &self, table_id: TableId, state: TableReplicationPhase, - ) -> ETLResult<()> { + ) -> EtlResult<()> { let (table_state, sync_done_lsn) = state.try_into()?; self.update_replication_state(self.pipeline_id, table_id, table_state, sync_done_lsn) .await?; diff --git a/etl/src/test_utils/test_destination_wrapper.rs b/etl/src/test_utils/test_destination_wrapper.rs index 40064b4de..d4cc00d45 100644 --- a/etl/src/test_utils/test_destination_wrapper.rs +++ b/etl/src/test_utils/test_destination_wrapper.rs @@ -8,7 +8,7 @@ use tokio::sync::{Notify, RwLock}; use crate::conversions::event::{Event, EventType}; use crate::conversions::table_row::TableRow; use crate::destination::base::Destination; -use crate::error::ETLResult; +use crate::error::EtlResult; use crate::schema::cache::SchemaCache; use crate::test_utils::event::check_events_count; @@ -157,7 +157,7 @@ impl TestDestinationWrapper { } impl Destination for TestDestinationWrapper { - async fn inject(&self, schema_cache: SchemaCache) -> ETLResult<()> { + async fn inject(&self, schema_cache: SchemaCache) -> EtlResult<()> { let destination = { let inner = self.inner.read().await; inner.wrapped_destination.clone() @@ -166,7 +166,7 @@ impl Destination for TestDestinationWrappe destination.inject(schema_cache).await } - async fn write_table_schema(&self, table_schema: TableSchema) -> ETLResult<()> { + async fn write_table_schema(&self, table_schema: TableSchema) -> EtlResult<()> { let destination = { let inner = self.inner.read().await; inner.wrapped_destination.clone() @@ -186,7 +186,7 @@ impl Destination for TestDestinationWrappe result } - async fn load_table_schemas(&self) -> ETLResult> { + async fn load_table_schemas(&self) -> EtlResult> { let destination = { let inner = self.inner.read().await; inner.wrapped_destination.clone() @@ -199,7 +199,7 @@ impl Destination for TestDestinationWrappe &self, table_id: TableId, table_rows: Vec, - ) -> ETLResult<()> { + ) -> EtlResult<()> { let destination = { let inner = self.inner.read().await; inner.wrapped_destination.clone() @@ -225,7 +225,7 @@ impl Destination for TestDestinationWrappe result } - async fn write_events(&self, events: Vec) -> ETLResult<()> { + async fn write_events(&self, events: Vec) -> EtlResult<()> { let destination = { let inner = self.inner.read().await; inner.wrapped_destination.clone() diff --git a/etl/src/workers/apply.rs b/etl/src/workers/apply.rs index bbeaeeb44..6f38b2f75 100644 --- a/etl/src/workers/apply.rs +++ b/etl/src/workers/apply.rs @@ -1,4 +1,4 @@ -use crate::error::{ETLError, ETLResult, ErrorKind}; +use crate::error::{EtlError, EtlResult, ErrorKind}; use config::shared::PipelineConfig; use postgres::schema::TableId; use std::sync::Arc; @@ -25,13 +25,13 @@ use crate::workers::table_sync::{TableSyncWorker, TableSyncWorkerState}; #[derive(Debug)] pub struct ApplyWorkerHandle { - handle: Option>>, + handle: Option>>, } impl WorkerHandle<()> for ApplyWorkerHandle { fn state(&self) {} - async fn wait(mut self) -> ETLResult<()> { + async fn wait(mut self) -> EtlResult<()> { let Some(handle) = self.handle.take() else { return Ok(()); }; @@ -93,9 +93,9 @@ where S: StateStore + Clone + Send + Sync + 'static, D: Destination + Clone + Send + Sync + 'static, { - type Error = ETLError; + type Error = EtlError; - async fn start(self) -> ETLResult { + async fn start(self) -> EtlResult { info!("starting apply worker"); let apply_worker_span = tracing::info_span!( @@ -149,7 +149,7 @@ where async fn get_start_lsn( pipeline_id: PipelineId, replication_client: &PgReplicationClient, -) -> ETLResult { +) -> EtlResult { let slot_name = get_slot_name(pipeline_id, WorkerType::Apply)?; // TODO: validate that we only create the slot when we first start replication which // means when all tables are in the Init state. In any other case we should raise an @@ -228,7 +228,7 @@ where ) } - async fn handle_syncing_table(&self, table_id: TableId, current_lsn: PgLsn) -> ETLResult { + async fn handle_syncing_table(&self, table_id: TableId, current_lsn: PgLsn) -> EtlResult { let mut pool = self.pool.lock().await; let table_sync_worker_state = pool.get_active_worker_state(table_id); @@ -249,7 +249,7 @@ where table_id: TableId, table_sync_worker_state: TableSyncWorkerState, current_lsn: PgLsn, - ) -> ETLResult { + ) -> EtlResult { let mut catchup_started = false; { let mut inner = table_sync_worker_state.get_inner().lock().await; @@ -301,7 +301,7 @@ where S: StateStore + Clone + Send + Sync + 'static, D: Destination + Clone + Send + Sync + 'static, { - async fn before_loop(&self, _start_lsn: PgLsn) -> ETLResult { + async fn before_loop(&self, _start_lsn: PgLsn) -> EtlResult { info!("starting table sync workers before the main apply loop"); let active_table_replication_states = @@ -339,7 +339,7 @@ where &self, current_lsn: PgLsn, update_state: bool, - ) -> ETLResult { + ) -> EtlResult { let active_table_replication_states = get_table_replication_states(&self.state_store, false).await?; debug!( @@ -387,7 +387,7 @@ where Ok(true) } - async fn skip_table(&self, table_id: TableId) -> ETLResult { + async fn skip_table(&self, table_id: TableId) -> EtlResult { let table_sync_worker_state = { let pool = self.pool.lock().await; pool.get_active_worker_state(table_id) @@ -412,7 +412,7 @@ where &self, table_id: TableId, remote_final_lsn: PgLsn, - ) -> ETLResult { + ) -> EtlResult { let pool = self.pool.lock().await; // We try to load the state first from memory, if we don't find it, we try to load from the diff --git a/etl/src/workers/base.rs b/etl/src/workers/base.rs index ef105bdb9..f50622fdb 100644 --- a/etl/src/workers/base.rs +++ b/etl/src/workers/base.rs @@ -1,7 +1,7 @@ use postgres::schema::TableId; use std::future::Future; -use crate::error::ETLResult; +use crate::error::EtlResult; /// The type of worker that is currently running. /// @@ -45,5 +45,5 @@ pub trait WorkerHandle { /// /// The future resolves to a [`Result`] indicating whether the worker completed successfully /// or encountered an error. - fn wait(self) -> impl Future> + Send; + fn wait(self) -> impl Future> + Send; } diff --git a/etl/src/workers/pool.rs b/etl/src/workers/pool.rs index ddf0f7b03..759cd9db5 100644 --- a/etl/src/workers/pool.rs +++ b/etl/src/workers/pool.rs @@ -8,8 +8,8 @@ use tracing::{debug, warn}; use crate::concurrency::future::ReactiveFutureCallback; use crate::destination::base::Destination; -use crate::error::ETLError; -use crate::error::{ETLResult, ErrorKind}; +use crate::error::EtlError; +use crate::error::{EtlResult, ErrorKind}; use crate::etl_error; use crate::state::store::base::StateStore; use crate::workers::base::{Worker, WorkerHandle}; @@ -44,7 +44,7 @@ impl TableSyncWorkerPoolInner { } } - pub async fn start_worker(&mut self, worker: TableSyncWorker) -> ETLResult + pub async fn start_worker(&mut self, worker: TableSyncWorker) -> EtlResult where S: StateStore + Clone + Send + Sync + 'static, D: Destination + Clone + Send + Sync + 'static, @@ -93,7 +93,7 @@ impl TableSyncWorkerPoolInner { } } - pub async fn wait_all(&mut self) -> ETLResult>> { + pub async fn wait_all(&mut self) -> EtlResult>> { // If there are active workers, we return the notify, signaling that not all of them are // ready. // @@ -165,7 +165,7 @@ impl TableSyncWorkerPool { self.inner.clone() } - pub async fn wait_all(&self) -> ETLResult<()> { + pub async fn wait_all(&self) -> EtlResult<()> { loop { // We try first to wait for all workers to be finished, in case there are still active // workers, we get back a `Notify` which we will use to try again once new workers reported diff --git a/etl/src/workers/table_sync.rs b/etl/src/workers/table_sync.rs index 7185f85f5..339a190a3 100644 --- a/etl/src/workers/table_sync.rs +++ b/etl/src/workers/table_sync.rs @@ -11,7 +11,7 @@ use crate::concurrency::future::ReactiveFuture; use crate::concurrency::shutdown::{ShutdownResult, ShutdownRx}; use crate::concurrency::signal::SignalTx; use crate::destination::base::Destination; -use crate::error::{ETLError, ETLResult, ErrorKind}; +use crate::error::{EtlError, EtlResult, ErrorKind}; use crate::pipeline::PipelineId; use crate::replication::apply::{ApplyLoopHook, start_apply_loop}; use crate::replication::client::PgReplicationClient; @@ -63,7 +63,7 @@ impl TableSyncWorkerStateInner { &mut self, phase: TableReplicationPhase, state_store: S, - ) -> ETLResult<()> { + ) -> EtlResult<()> { self.set_phase(phase); // If we should store this phase change, we want to do it via the supplied state store. @@ -189,7 +189,7 @@ impl TableSyncWorkerState { #[derive(Debug)] pub struct TableSyncWorkerHandle { state: TableSyncWorkerState, - handle: Option>>, + handle: Option>>, } impl WorkerHandle for TableSyncWorkerHandle { @@ -197,7 +197,7 @@ impl WorkerHandle for TableSyncWorkerHandle { self.state.clone() } - async fn wait(mut self) -> ETLResult<()> { + async fn wait(mut self) -> EtlResult<()> { let Some(handle) = self.handle.take() else { return Ok(()); }; @@ -266,9 +266,9 @@ where S: StateStore + Clone + Send + Sync + 'static, D: Destination + Clone + Send + Sync + 'static, { - type Error = ETLError; + type Error = EtlError; - async fn start(mut self) -> ETLResult { + async fn start(mut self) -> EtlResult { info!("starting table sync worker for table {}", self.table_id); let Some(table_replication_phase) = self @@ -451,7 +451,7 @@ where /// /// Returns `Ok(false)` when the worker is done with its work, signaling the caller that the apply /// loop should be stopped. - async fn try_advance_phase(&self, current_lsn: PgLsn, update_state: bool) -> ETLResult { + async fn try_advance_phase(&self, current_lsn: PgLsn, update_state: bool) -> EtlResult { let mut inner = self.table_sync_worker_state.get_inner().lock().await; // If we caught up with the lsn, we mark this table as `SyncDone` and stop the worker. @@ -487,7 +487,7 @@ impl ApplyLoopHook for TableSyncWorkerHook where S: StateStore + Clone + Send + Sync + 'static, { - async fn before_loop(&self, start_lsn: PgLsn) -> ETLResult { + async fn before_loop(&self, start_lsn: PgLsn) -> EtlResult { info!("checking if the table sync worker is already caught up with the apply worker"); self.try_advance_phase(start_lsn, true).await @@ -504,7 +504,7 @@ where &self, current_lsn: PgLsn, update_state: bool, - ) -> ETLResult { + ) -> EtlResult { info!( "processing syncing tables for table sync worker with lsn {}", current_lsn @@ -513,7 +513,7 @@ where self.try_advance_phase(current_lsn, update_state).await } - async fn skip_table(&self, table_id: TableId) -> ETLResult { + async fn skip_table(&self, table_id: TableId) -> EtlResult { if self.table_id != table_id { return Ok(true); } @@ -530,7 +530,7 @@ where &self, table_id: TableId, _remote_final_lsn: PgLsn, - ) -> ETLResult { + ) -> EtlResult { let inner = self.table_sync_worker_state.get_inner().lock().await; let is_skipped = matches!( inner.table_replication_phase.as_type(), From 830fc3270e8f6655ac5aa838ce7e6b388b8dee1e Mon Sep 17 00:00:00 2001 From: Riccardo Busetti Date: Mon, 28 Jul 2025 09:25:20 +0200 Subject: [PATCH 18/18] Improve --- etl/src/conversions/bool.rs | 2 +- etl/src/conversions/event.rs | 6 ++---- etl/src/conversions/hex.rs | 2 +- etl/src/conversions/table_row.rs | 2 +- etl/src/conversions/text.rs | 2 +- etl/src/destination/bigquery.rs | 2 +- etl/src/error.rs | 2 +- etl/src/pipeline.rs | 2 +- etl/src/replication/apply.rs | 2 +- etl/src/replication/client.rs | 2 +- etl/src/replication/slot.rs | 2 +- etl/src/replication/stream.rs | 2 +- etl/src/replication/table_sync.rs | 2 +- etl/src/state/store/postgres.rs | 2 +- etl/src/workers/apply.rs | 2 +- etl/src/workers/pool.rs | 2 +- etl/src/workers/table_sync.rs | 2 +- 17 files changed, 18 insertions(+), 20 deletions(-) diff --git a/etl/src/conversions/bool.rs b/etl/src/conversions/bool.rs index f3e00d2b9..d71cff542 100644 --- a/etl/src/conversions/bool.rs +++ b/etl/src/conversions/bool.rs @@ -1,6 +1,6 @@ use crate::bail; use crate::error::EtlResult; -use crate::error::{EtlError, ErrorKind}; +use crate::error::{ErrorKind, EtlError}; pub fn parse_bool(s: &str) -> EtlResult { if s == "t" { diff --git a/etl/src/conversions/event.rs b/etl/src/conversions/event.rs index eff419a1c..568f3e5d9 100644 --- a/etl/src/conversions/event.rs +++ b/etl/src/conversions/event.rs @@ -10,7 +10,7 @@ use crate::conversions::Cell; use crate::conversions::table_row::TableRow; use crate::conversions::text::TextFormatConverter; use crate::error::EtlError; -use crate::error::{EtlResult, ErrorKind}; +use crate::error::{ErrorKind, EtlResult}; use crate::schema::cache::SchemaCache; use crate::{bail, etl_error}; @@ -246,9 +246,7 @@ async fn get_table_schema(schema_cache: &SchemaCache, table_id: TableId) -> EtlR etl_error!( ErrorKind::MissingTableSchema, "Table not found in the schema cache", - format!( - "The table schema for table {table_id} was not found in the cache" - ) + format!("The table schema for table {table_id} was not found in the cache") ) }) } diff --git a/etl/src/conversions/hex.rs b/etl/src/conversions/hex.rs index 762751f6b..dfcd78a34 100644 --- a/etl/src/conversions/hex.rs +++ b/etl/src/conversions/hex.rs @@ -1,6 +1,6 @@ use crate::bail; use crate::error::EtlError; -use crate::error::{EtlResult, ErrorKind}; +use crate::error::{ErrorKind, EtlResult}; pub fn from_bytea_hex(s: &str) -> EtlResult> { if s.len() < 2 || &s[..2] != "\\x" { diff --git a/etl/src/conversions/table_row.rs b/etl/src/conversions/table_row.rs index 000a30885..0c0aa2d20 100644 --- a/etl/src/conversions/table_row.rs +++ b/etl/src/conversions/table_row.rs @@ -2,7 +2,7 @@ use super::Cell; use crate::bail; use crate::conversions::text::TextFormatConverter; use crate::error::EtlError; -use crate::error::{EtlResult, ErrorKind}; +use crate::error::{ErrorKind, EtlResult}; use core::str; use postgres::schema::ColumnSchema; use tracing::error; diff --git a/etl/src/conversions/text.rs b/etl/src/conversions/text.rs index d6f2a99ec..543de19d6 100644 --- a/etl/src/conversions/text.rs +++ b/etl/src/conversions/text.rs @@ -6,7 +6,7 @@ use uuid::Uuid; use crate::bail; use crate::conversions::{bool::parse_bool, hex}; -use crate::error::{EtlResult, ErrorKind}; +use crate::error::{ErrorKind, EtlResult}; use super::{ArrayCell, Cell, numeric::PgNumeric}; diff --git a/etl/src/destination/bigquery.rs b/etl/src/destination/bigquery.rs index b7bf02a27..9154b2e20 100644 --- a/etl/src/destination/bigquery.rs +++ b/etl/src/destination/bigquery.rs @@ -13,7 +13,7 @@ use crate::conversions::Cell; use crate::conversions::event::{Event, TruncateEvent}; use crate::conversions::table_row::TableRow; use crate::destination::base::Destination; -use crate::error::{EtlError, EtlResult, ErrorKind}; +use crate::error::{ErrorKind, EtlError, EtlResult}; use crate::schema::cache::SchemaCache; /// Table name for storing ETL table schema metadata in BigQuery. diff --git a/etl/src/error.rs b/etl/src/error.rs index e2c36325c..9304eb659 100644 --- a/etl/src/error.rs +++ b/etl/src/error.rs @@ -235,7 +235,7 @@ impl fmt::Display for EtlError { } } -impl error::Error for EtlError { } +impl error::Error for EtlError {} // Ergonomic constructors following Redis pattern diff --git a/etl/src/pipeline.rs b/etl/src/pipeline.rs index 09b0c7038..2eb58b647 100644 --- a/etl/src/pipeline.rs +++ b/etl/src/pipeline.rs @@ -6,7 +6,7 @@ use tracing::{error, info}; use crate::bail; use crate::concurrency::shutdown::{ShutdownTx, create_shutdown_channel}; use crate::destination::base::Destination; -use crate::error::{EtlError, EtlResult, ErrorKind}; +use crate::error::{ErrorKind, EtlError, EtlResult}; use crate::replication::client::PgReplicationClient; use crate::schema::cache::SchemaCache; use crate::state::store::base::StateStore; diff --git a/etl/src/replication/apply.rs b/etl/src/replication/apply.rs index 13d0c5982..92f94732e 100644 --- a/etl/src/replication/apply.rs +++ b/etl/src/replication/apply.rs @@ -15,7 +15,7 @@ use crate::concurrency::shutdown::ShutdownRx; use crate::concurrency::signal::SignalRx; use crate::conversions::event::{Event, EventType, convert_message_to_event}; use crate::destination::base::Destination; -use crate::error::{EtlError, EtlResult, ErrorKind}; +use crate::error::{ErrorKind, EtlError, EtlResult}; use crate::pipeline::PipelineId; use crate::replication::client::PgReplicationClient; use crate::replication::slot::get_slot_name; diff --git a/etl/src/replication/client.rs b/etl/src/replication/client.rs index 00d001928..300b2e3d5 100644 --- a/etl/src/replication/client.rs +++ b/etl/src/replication/client.rs @@ -1,4 +1,4 @@ -use crate::error::{EtlError, EtlResult, ErrorKind}; +use crate::error::{ErrorKind, EtlError, EtlResult}; use crate::{bail, etl_error}; use config::shared::{IntoConnectOptions, PgConnectionConfig}; use pg_escape::{quote_identifier, quote_literal}; diff --git a/etl/src/replication/slot.rs b/etl/src/replication/slot.rs index 29b2cff29..d7cf82867 100644 --- a/etl/src/replication/slot.rs +++ b/etl/src/replication/slot.rs @@ -1,6 +1,6 @@ use crate::bail; use crate::error::EtlError; -use crate::error::{EtlResult, ErrorKind}; +use crate::error::{ErrorKind, EtlResult}; use crate::pipeline::PipelineId; use crate::workers::base::WorkerType; diff --git a/etl/src/replication/stream.rs b/etl/src/replication/stream.rs index e179b5428..3f91d9bc2 100644 --- a/etl/src/replication/stream.rs +++ b/etl/src/replication/stream.rs @@ -13,7 +13,7 @@ use tracing::debug; use crate::conversions::table_row::{TableRow, TableRowConverter}; use crate::error::EtlError; -use crate::error::{EtlResult, ErrorKind}; +use crate::error::{ErrorKind, EtlResult}; use crate::etl_error; /// The amount of milliseconds between two consecutive status updates in case no forced update diff --git a/etl/src/replication/table_sync.rs b/etl/src/replication/table_sync.rs index f33974e09..8e7f6415c 100644 --- a/etl/src/replication/table_sync.rs +++ b/etl/src/replication/table_sync.rs @@ -13,7 +13,7 @@ use crate::concurrency::shutdown::{ShutdownResult, ShutdownRx}; use crate::concurrency::signal::SignalTx; use crate::concurrency::stream::BatchStream; use crate::destination::base::Destination; -use crate::error::{EtlError, EtlResult, ErrorKind}; +use crate::error::{ErrorKind, EtlError, EtlResult}; use crate::pipeline::PipelineId; use crate::replication::client::PgReplicationClient; use crate::replication::slot::get_slot_name; diff --git a/etl/src/state/store/postgres.rs b/etl/src/state/store/postgres.rs index 1eaca3911..c8c387701 100644 --- a/etl/src/state/store/postgres.rs +++ b/etl/src/state/store/postgres.rs @@ -11,7 +11,7 @@ use tokio::sync::Mutex; use tokio_postgres::types::PgLsn; use tracing::{debug, info}; -use crate::error::{EtlError, EtlResult, ErrorKind}; +use crate::error::{ErrorKind, EtlError, EtlResult}; use crate::pipeline::PipelineId; use crate::state::store::base::StateStore; use crate::state::table::TableReplicationPhase; diff --git a/etl/src/workers/apply.rs b/etl/src/workers/apply.rs index 6f38b2f75..d21fd76c3 100644 --- a/etl/src/workers/apply.rs +++ b/etl/src/workers/apply.rs @@ -1,4 +1,4 @@ -use crate::error::{EtlError, EtlResult, ErrorKind}; +use crate::error::{ErrorKind, EtlError, EtlResult}; use config::shared::PipelineConfig; use postgres::schema::TableId; use std::sync::Arc; diff --git a/etl/src/workers/pool.rs b/etl/src/workers/pool.rs index 759cd9db5..3e0671479 100644 --- a/etl/src/workers/pool.rs +++ b/etl/src/workers/pool.rs @@ -9,7 +9,7 @@ use tracing::{debug, warn}; use crate::concurrency::future::ReactiveFutureCallback; use crate::destination::base::Destination; use crate::error::EtlError; -use crate::error::{EtlResult, ErrorKind}; +use crate::error::{ErrorKind, EtlResult}; use crate::etl_error; use crate::state::store::base::StateStore; use crate::workers::base::{Worker, WorkerHandle}; diff --git a/etl/src/workers/table_sync.rs b/etl/src/workers/table_sync.rs index 339a190a3..c89cb9f0f 100644 --- a/etl/src/workers/table_sync.rs +++ b/etl/src/workers/table_sync.rs @@ -11,7 +11,7 @@ use crate::concurrency::future::ReactiveFuture; use crate::concurrency::shutdown::{ShutdownResult, ShutdownRx}; use crate::concurrency::signal::SignalTx; use crate::destination::base::Destination; -use crate::error::{EtlError, EtlResult, ErrorKind}; +use crate::error::{ErrorKind, EtlError, EtlResult}; use crate::pipeline::PipelineId; use crate::replication::apply::{ApplyLoopHook, start_apply_loop}; use crate::replication::client::PgReplicationClient;