diff --git a/etl-destinations/src/bigquery/client.rs b/etl-destinations/src/bigquery/client.rs index 683ba8e13..72c174086 100644 --- a/etl-destinations/src/bigquery/client.rs +++ b/etl-destinations/src/bigquery/client.rs @@ -837,7 +837,7 @@ mod tests { TableName::new("public".to_string(), "test_table".to_string()), columns, )); - let replication_mask = ReplicationMask::build(&table_schema, &column_names).unwrap(); + let replication_mask = ReplicationMask::build_or_all(&table_schema, &column_names); ReplicatedTableSchema::from_mask(table_schema, replication_mask) } diff --git a/etl-postgres/src/replication/schema.rs b/etl-postgres/src/replication/schema.rs index 8ef235d60..a540f88ba 100644 --- a/etl-postgres/src/replication/schema.rs +++ b/etl-postgres/src/replication/schema.rs @@ -4,7 +4,7 @@ use sqlx::{PgExecutor, PgPool, Row}; use std::collections::HashMap; use tokio_postgres::types::Type as PgType; -use crate::types::{ColumnSchema, TableId, TableName, TableSchema}; +use crate::types::{ColumnSchema, SnapshotId, TableId, TableName, TableSchema}; macro_rules! define_type_mappings { ( @@ -134,10 +134,11 @@ define_type_mappings! { DATE_RANGE => "DATE_RANGE" } -/// Stores a table schema in the database. +/// Stores a table schema in the database with a specific snapshot ID. /// -/// Inserts or updates table schema and column information in schema storage tables -/// using a transaction to ensure atomicity. +/// Upserts table schema and replaces all column information in schema storage tables +/// using a transaction to ensure atomicity. If a schema version already exists for +/// the same (pipeline_id, table_id, snapshot_id), columns are deleted and re-inserted. pub async fn store_table_schema( pool: &PgPool, pipeline_id: i64, @@ -145,16 +146,13 @@ pub async fn store_table_schema( ) -> Result<(), sqlx::Error> { let mut tx = pool.begin().await?; - // Insert or update table schema record + // Upsert table schema version let table_schema_id: i64 = sqlx::query( r#" - insert into etl.table_schemas (pipeline_id, table_id, schema_name, table_name) - values ($1, $2, $3, $4) - on conflict (pipeline_id, table_id) - do update set - schema_name = excluded.schema_name, - table_name = excluded.table_name, - updated_at = now() + insert into etl.table_schemas (pipeline_id, table_id, schema_name, table_name, snapshot_id) + values ($1, $2, $3, $4, $5::pg_lsn) + on conflict (pipeline_id, table_id, snapshot_id) + do update set schema_name = excluded.schema_name, table_name = excluded.table_name returning id "#, ) @@ -162,6 +160,7 @@ pub async fn store_table_schema( .bind(table_schema.id.into_inner() as i64) .bind(&table_schema.name.schema) .bind(&table_schema.name.name) + .bind(table_schema.snapshot_id.to_pg_lsn_string()) .fetch_one(&mut *tx) .await? .get(0); @@ -199,20 +198,34 @@ pub async fn store_table_schema( Ok(()) } -/// Loads all table schemas for a pipeline from the database. +/// Loads all table schemas for a pipeline from the database at the latest snapshot. /// /// Retrieves table schemas and columns from schema storage tables, -/// reconstructing complete [`TableSchema`] objects. +/// reconstructing complete [`TableSchema`] objects. This is equivalent to +/// calling [`load_table_schemas_at_snapshot`] with the maximum LSN value. pub async fn load_table_schemas( pool: &PgPool, pipeline_id: i64, ) -> Result, sqlx::Error> { + load_table_schemas_at_snapshot(pool, pipeline_id, SnapshotId::max()).await +} + +/// Loads a single table schema with the largest snapshot_id <= the requested snapshot. +/// +/// Returns `None` if no schema version exists for the table at or before the given snapshot. +pub async fn load_table_schema_at_snapshot( + pool: &PgPool, + pipeline_id: i64, + table_id: TableId, + snapshot_id: SnapshotId, +) -> Result, sqlx::Error> { let rows = sqlx::query( r#" select ts.table_id, ts.schema_name, ts.table_name, + ts.snapshot_id::text as snapshot_id, tc.column_name, tc.column_type, tc.type_modifier, @@ -222,11 +235,93 @@ pub async fn load_table_schemas( tc.primary_key_ordinal_position from etl.table_schemas ts inner join etl.table_columns tc on ts.id = tc.table_schema_id - where ts.pipeline_id = $1 - order by ts.table_id, tc.column_order + where ts.id = ( + select id from etl.table_schemas + where pipeline_id = $1 and table_id = $2 and snapshot_id <= $3::pg_lsn + order by snapshot_id desc + limit 1 + ) + order by tc.column_order + "#, + ) + .bind(pipeline_id) + .bind(SqlxTableId(table_id.into_inner())) + .bind(snapshot_id.to_pg_lsn_string()) + .fetch_all(pool) + .await?; + + if rows.is_empty() { + return Ok(None); + } + + let first_row = &rows[0]; + let table_oid: SqlxTableId = first_row.get("table_id"); + let table_id = TableId::new(table_oid.0); + let schema_name: String = first_row.get("schema_name"); + let table_name: String = first_row.get("table_name"); + let snapshot_id_str: String = first_row.get("snapshot_id"); + let snapshot_id = SnapshotId::from_pg_lsn_string(&snapshot_id_str) + .map_err(|e| sqlx::Error::Protocol(e.to_string()))?; + + let mut table_schema = TableSchema::with_snapshot_id( + table_id, + TableName::new(schema_name, table_name), + vec![], + snapshot_id, + ); + + for row in rows { + table_schema.add_column_schema(parse_column_schema(&row)); + } + + Ok(Some(table_schema)) +} + +/// Loads all table schemas for a pipeline at a specific snapshot point. +/// +/// For each table, retrieves the schema version with the largest snapshot_id +/// that is <= the requested snapshot_id. Tables without any schema version +/// at or before the snapshot are excluded from the result. +pub async fn load_table_schemas_at_snapshot( + pool: &PgPool, + pipeline_id: i64, + snapshot_id: SnapshotId, +) -> Result, sqlx::Error> { + // Use DISTINCT ON to efficiently find the latest schema version for each table. + // PostgreSQL optimizes DISTINCT ON with ORDER BY using index scans when possible. + let rows = sqlx::query( + r#" + with latest_schemas as ( + select distinct on (ts.table_id) + ts.id, + ts.table_id, + ts.schema_name, + ts.table_name, + ts.snapshot_id + from etl.table_schemas ts + where ts.pipeline_id = $1 + and ts.snapshot_id <= $2::pg_lsn + order by ts.table_id, ts.snapshot_id desc + ) + select + ls.table_id, + ls.schema_name, + ls.table_name, + ls.snapshot_id::text as snapshot_id, + tc.column_name, + tc.column_type, + tc.type_modifier, + tc.nullable, + tc.primary_key, + tc.column_order, + tc.primary_key_ordinal_position + from latest_schemas ls + inner join etl.table_columns tc on ls.id = tc.table_schema_id + order by ls.table_id, tc.column_order "#, ) .bind(pipeline_id) + .bind(snapshot_id.to_pg_lsn_string()) .fetch_all(pool) .await?; @@ -237,9 +332,17 @@ pub async fn load_table_schemas( let table_id = TableId::new(table_oid.0); let schema_name: String = row.get("schema_name"); let table_name: String = row.get("table_name"); + let snapshot_id_str: String = row.get("snapshot_id"); + let row_snapshot_id = SnapshotId::from_pg_lsn_string(&snapshot_id_str) + .map_err(|e| sqlx::Error::Protocol(e.to_string()))?; let entry = table_schemas.entry(table_id).or_insert_with(|| { - TableSchema::new(table_id, TableName::new(schema_name, table_name), vec![]) + TableSchema::with_snapshot_id( + table_id, + TableName::new(schema_name, table_name), + vec![], + row_snapshot_id, + ) }); entry.add_column_schema(parse_column_schema(&row)); diff --git a/etl-postgres/src/tokio/test_utils.rs b/etl-postgres/src/tokio/test_utils.rs index ccdeaf101..1b15c097b 100644 --- a/etl-postgres/src/tokio/test_utils.rs +++ b/etl-postgres/src/tokio/test_utils.rs @@ -22,11 +22,16 @@ pub enum TableModification<'a> { DropColumn { name: &'a str, }, - /// Alter an existing column with the specified alteration. + /// Alter an existing column with the specified alteration (e.g., "type bigint"). AlterColumn { name: &'a str, alteration: &'a str, }, + /// Rename an existing column. + RenameColumn { + old_name: &'a str, + new_name: &'a str, + }, ReplicaIdentity { value: &'a str, }, @@ -211,6 +216,9 @@ impl PgDatabase { TableModification::AlterColumn { name, alteration } => { format!("alter column {name} {alteration}") } + TableModification::RenameColumn { old_name, new_name } => { + format!("rename column {old_name} to {new_name}") + } TableModification::ReplicaIdentity { value } => { format!("replica identity {value}") } diff --git a/etl-postgres/src/types/schema.rs b/etl-postgres/src/types/schema.rs index 2fb2f1583..6171fbaa1 100644 --- a/etl-postgres/src/types/schema.rs +++ b/etl-postgres/src/types/schema.rs @@ -1,10 +1,11 @@ use pg_escape::quote_identifier; use std::collections::HashSet; use std::fmt; +use std::hash::{Hash, Hasher}; use std::str::FromStr; use std::sync::Arc; use thiserror::Error; -use tokio_postgres::types::{FromSql, ToSql, Type}; +use tokio_postgres::types::{FromSql, PgLsn, ToSql, Type}; /// Errors that can occur during schema operations. #[derive(Debug, Error)] @@ -12,11 +13,103 @@ pub enum SchemaError { /// Columns were received during replication that do not exist in the stored table schema. #[error("received columns during replication that are not in the stored table schema: {0:?}")] UnknownReplicatedColumns(Vec), + + /// A snapshot ID string could not be converted to the [`SnapshotId`] type. + #[error("invalid snapshot id '{0}'")] + InvalidSnapshotId(String), } /// An object identifier in Postgres. type Oid = u32; +/// Snapshot identifier for schema versioning. +/// +/// Wraps a [`PgLsn`] to represent the start_lsn of the DDL message that created a schema version. +/// A value of 0/0 indicates the initial schema before any DDL changes. +/// Stored as `pg_lsn` in the database. +#[derive(Debug, Clone, Copy, Eq, PartialEq, Ord, PartialOrd)] +pub struct SnapshotId(PgLsn); + +impl SnapshotId { + /// Returns the initial snapshot ID (0/0) for the first schema version. + pub fn initial() -> Self { + Self(PgLsn::from(0)) + } + + /// Returns the maximum possible snapshot ID. + pub fn max() -> Self { + Self(PgLsn::from(u64::MAX)) + } + + /// Creates a new [`SnapshotId`] from a [`PgLsn`]. + pub fn new(lsn: PgLsn) -> Self { + Self(lsn) + } + + /// Returns the inner [`PgLsn`] value. + pub fn into_inner(self) -> PgLsn { + self.0 + } + + /// Returns the underlying `u64` representation. + pub fn as_u64(self) -> u64 { + self.0.into() + } + + /// Converts to a `pg_lsn` string. + pub fn to_pg_lsn_string(self) -> String { + self.0.to_string() + } + + /// Parses a `pg_lsn` string. + /// + /// # Errors + /// + /// Returns [`SchemaError::InvalidSnapshotId`] if the string is not a valid `pg_lsn` format. + pub fn from_pg_lsn_string(s: &str) -> Result { + s.parse::() + .map(Self) + .map_err(|_| SchemaError::InvalidSnapshotId(s.to_string())) + } +} + +impl Hash for SnapshotId { + fn hash(&self, state: &mut H) { + let value: u64 = self.0.into(); + value.hash(state); + } +} + +impl From for SnapshotId { + fn from(lsn: PgLsn) -> Self { + Self(lsn) + } +} + +impl From for PgLsn { + fn from(snapshot_id: SnapshotId) -> Self { + snapshot_id.0 + } +} + +impl From for SnapshotId { + fn from(value: u64) -> Self { + Self(PgLsn::from(value)) + } +} + +impl From for u64 { + fn from(snapshot_id: SnapshotId) -> Self { + snapshot_id.0.into() + } +} + +impl fmt::Display for SnapshotId { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + write!(f, "{}", self.0) + } +} + /// A fully qualified Postgres table name consisting of a schema and table name. /// /// This type represents a table identifier in Postgres, which requires both a schema name @@ -189,23 +282,39 @@ impl ToSql for TableId { /// Represents the complete schema of a Postgres table. /// /// This type contains all metadata about a table including its name, OID, -/// and the schemas of all its columns. +/// the schemas of all its columns, and a snapshot identifier for versioning. #[derive(Debug, Clone, Eq, PartialEq)] pub struct TableSchema { - /// The Postgres OID of the table + /// The Postgres OID of the table. pub id: TableId, - /// The fully qualified name of the table + /// The fully qualified name of the table. pub name: TableName, - /// The schemas of all columns in the table + /// The schemas of all columns in the table. pub column_schemas: Vec, + /// The snapshot identifier for this schema version. + /// + /// Value 0 indicates the initial schema, other values are start_lsn positions of DDL changes. + pub snapshot_id: SnapshotId, } impl TableSchema { + /// Creates a new [`TableSchema`] with the initial snapshot ID (0/0). pub fn new(id: TableId, name: TableName, column_schemas: Vec) -> Self { + Self::with_snapshot_id(id, name, column_schemas, SnapshotId::initial()) + } + + /// Creates a new [`TableSchema`] with a specific snapshot ID. + pub fn with_snapshot_id( + id: TableId, + name: TableName, + column_schemas: Vec, + snapshot_id: SnapshotId, + ) -> Self { Self { id, name, column_schemas, + snapshot_id, } } @@ -231,7 +340,7 @@ impl TableSchema { pub struct ReplicationMask(Arc>); impl ReplicationMask { - /// Creates a new [`ReplicationMask`] from a table schema and column names. + /// Tries to create a new [`ReplicationMask`] from a table schema and column names. /// /// The mask is constructed by checking which column names from the schema are present /// in the provided set of replicated column names. @@ -243,7 +352,7 @@ impl ReplicationMask { /// /// The column validation occurs because we have to make sure that the stored table schema is always /// up to date, if not, it's a critical problem. - pub fn build( + pub fn try_build( table_schema: &TableSchema, replicated_column_names: &HashSet, ) -> Result { @@ -259,10 +368,52 @@ impl ReplicationMask { .cloned() .collect(); + // This check ensures all replicated columns are present in the schema. + // + // Limitation: If a column exists in the schema but is absent from the replicated columns, + // we assume publication-level column filtering is enabled. However, this is indistinguishable + // from an invalid state where the schema has diverged, we cannot detect the difference. + // + // How schema divergence occurs: When progress tracking fails and the system restarts, + // we may receive a `Relation` message reflecting the *current* table schema rather than + // the schema at the time the in-flight events were emitted. This is how Postgres handles + // initial `Relation` messages on reconnection. It's not the wrong behavior since the data + // has the columns that it announces, but it conflicts with our schema management logic. + // + // Invariant: Our schema management assumes the schema in `Relation` messages is consistent + // with the schema under which the corresponding row events were produced. + // + // In the future we might want to implement a system to go around this edge case. if !unknown_columns.is_empty() { return Err(SchemaError::UnknownReplicatedColumns(unknown_columns)); } + Ok(Self::build(table_schema, replicated_column_names)) + } + + /// Creates a new [`ReplicationMask`] from a table schema and column names, falling back + /// to an all-replicated mask if validation fails. + /// + /// This method attempts to validate that all replicated column names exist in the schema. + /// If validation succeeds, it builds a mask based on matching columns. If validation fails + /// (unknown columns are present), it returns a mask with all columns marked as replicated. + /// + /// This fallback behavior handles the case where Postgres sends a `Relation` message on + /// reconnection with the current schema, but the stored schema is from an earlier point + /// before DDL changes. Rather than failing, we enable all columns and let the system + /// converge when the actual DDL message is replayed. + pub fn build_or_all( + table_schema: &TableSchema, + replicated_column_names: &HashSet, + ) -> Self { + match Self::try_build(table_schema, replicated_column_names) { + Ok(mask) => mask, + Err(_) => Self::all(table_schema), + } + } + + /// Creates a new [`ReplicationMask`] from a table schema and column names. + pub fn build(table_schema: &TableSchema, replicated_column_names: &HashSet) -> Self { let mask = table_schema .column_schemas .iter() @@ -275,7 +426,13 @@ impl ReplicationMask { }) .collect(); - Ok(Self(Arc::new(mask))) + Self(Arc::new(mask)) + } + + /// Creates a [`ReplicationMask`] with all columns marked as replicated. + pub fn all(table_schema: &TableSchema) -> Self { + let mask = vec![1; table_schema.column_schemas.len()]; + Self(Arc::new(mask)) } /// Returns the underlying mask as a slice. @@ -378,48 +535,48 @@ mod tests { } #[test] - fn test_replication_mask_build_all_columns_replicated() { + fn test_replication_mask_try_build_all_columns_replicated() { let schema = create_test_table_schema(); let replicated_columns: HashSet = ["id", "name", "age"] .into_iter() .map(String::from) .collect(); - let mask = ReplicationMask::build(&schema, &replicated_columns).unwrap(); + let mask = ReplicationMask::try_build(&schema, &replicated_columns).unwrap(); assert_eq!(mask.as_slice(), &[1, 1, 1]); } #[test] - fn test_replication_mask_build_partial_columns_replicated() { + fn test_replication_mask_try_build_partial_columns_replicated() { let schema = create_test_table_schema(); let replicated_columns: HashSet = ["id", "age"].into_iter().map(String::from).collect(); - let mask = ReplicationMask::build(&schema, &replicated_columns).unwrap(); + let mask = ReplicationMask::try_build(&schema, &replicated_columns).unwrap(); assert_eq!(mask.as_slice(), &[1, 0, 1]); } #[test] - fn test_replication_mask_build_no_columns_replicated() { + fn test_replication_mask_try_build_no_columns_replicated() { let schema = create_test_table_schema(); let replicated_columns: HashSet = HashSet::new(); - let mask = ReplicationMask::build(&schema, &replicated_columns).unwrap(); + let mask = ReplicationMask::try_build(&schema, &replicated_columns).unwrap(); assert_eq!(mask.as_slice(), &[0, 0, 0]); } #[test] - fn test_replication_mask_build_unknown_column_error() { + fn test_replication_mask_try_build_unknown_column_error() { let schema = create_test_table_schema(); let replicated_columns: HashSet = ["id", "unknown_column"] .into_iter() .map(String::from) .collect(); - let result = ReplicationMask::build(&schema, &replicated_columns); + let result = ReplicationMask::try_build(&schema, &replicated_columns); assert!(result.is_err()); let err = result.unwrap_err(); @@ -427,16 +584,17 @@ mod tests { SchemaError::UnknownReplicatedColumns(columns) => { assert_eq!(columns, vec!["unknown_column".to_string()]); } + _ => panic!("expected UnknownReplicatedColumns error"), } } #[test] - fn test_replication_mask_build_multiple_unknown_columns_error() { + fn test_replication_mask_try_build_multiple_unknown_columns_error() { let schema = create_test_table_schema(); let replicated_columns: HashSet = ["id", "foo", "bar"].into_iter().map(String::from).collect(); - let result = ReplicationMask::build(&schema, &replicated_columns); + let result = ReplicationMask::try_build(&schema, &replicated_columns); assert!(result.is_err()); let err = result.unwrap_err(); @@ -445,6 +603,40 @@ mod tests { columns.sort(); assert_eq!(columns, vec!["bar".to_string(), "foo".to_string()]); } + _ => panic!("expected UnknownReplicatedColumns error"), } } + + #[test] + fn test_replication_mask_build_or_all_success() { + let schema = create_test_table_schema(); + let replicated_columns: HashSet = + ["id", "age"].into_iter().map(String::from).collect(); + + let mask = ReplicationMask::build_or_all(&schema, &replicated_columns); + + assert_eq!(mask.as_slice(), &[1, 0, 1]); + } + + #[test] + fn test_replication_mask_build_or_all_falls_back_to_all() { + let schema = create_test_table_schema(); + let replicated_columns: HashSet = ["id", "unknown_column"] + .into_iter() + .map(String::from) + .collect(); + + let mask = ReplicationMask::build_or_all(&schema, &replicated_columns); + + // Falls back to all columns being replicated. + assert_eq!(mask.as_slice(), &[1, 1, 1]); + } + + #[test] + fn test_replication_mask_all() { + let schema = create_test_table_schema(); + let mask = ReplicationMask::all(&schema); + + assert_eq!(mask.as_slice(), &[1, 1, 1]); + } } diff --git a/etl/Cargo.toml b/etl/Cargo.toml index b122549a2..f5386fcda 100644 --- a/etl/Cargo.toml +++ b/etl/Cargo.toml @@ -26,6 +26,7 @@ metrics = { workspace = true } pg_escape = { workspace = true } pin-project-lite = { workspace = true } postgres-replication = { workspace = true } +rand = { workspace = true, features = ["thread_rng"] } ring = { workspace = true, default-features = false } rustls = { workspace = true, features = ["aws-lc-rs", "logging"] } serde = { workspace = true, features = ["derive"] } @@ -50,5 +51,3 @@ etl-postgres = { workspace = true, features = [ "test-utils", ] } etl-telemetry = { workspace = true } - -rand = { workspace = true, features = ["thread_rng"] } diff --git a/etl/migrations/20251205000000_schema_versioning.sql b/etl/migrations/20251205000000_schema_versioning.sql new file mode 100644 index 000000000..4796029e7 --- /dev/null +++ b/etl/migrations/20251205000000_schema_versioning.sql @@ -0,0 +1,19 @@ +-- Add snapshot_id column to table_schemas for schema versioning. +-- The snapshot_id value is the start_lsn of the DDL message that created this schema version. +-- Initial schemas use snapshot_id='0/0'. + +ALTER TABLE etl.table_schemas + ADD COLUMN IF NOT EXISTS snapshot_id PG_LSN NOT NULL DEFAULT '0/0'; + +-- Change unique constraint from (pipeline_id, table_id) to (pipeline_id, table_id, snapshot_id) +-- to allow multiple schema versions per table. +ALTER TABLE etl.table_schemas + DROP CONSTRAINT IF EXISTS table_schemas_pipeline_id_table_id_key; + +ALTER TABLE etl.table_schemas + ADD CONSTRAINT table_schemas_pipeline_id_table_id_snapshot_id_key + UNIQUE (pipeline_id, table_id, snapshot_id); + +-- Index for efficient "find largest snapshot_id <= X" queries. +CREATE INDEX IF NOT EXISTS idx_table_schemas_pipeline_table_snapshot_id + ON etl.table_schemas (pipeline_id, table_id, snapshot_id DESC); diff --git a/etl/src/conversions/event.rs b/etl/src/conversions/event.rs index 5212f54b3..6391b0048 100644 --- a/etl/src/conversions/event.rs +++ b/etl/src/conversions/event.rs @@ -2,7 +2,8 @@ use core::str; use std::collections::HashSet; use etl_postgres::types::{ - ColumnSchema, ReplicatedTableSchema, TableId, TableName, TableSchema, convert_type_oid_to_type, + ColumnSchema, ReplicatedTableSchema, SnapshotId, TableId, TableName, TableSchema, + convert_type_oid_to_type, }; use postgres_replication::protocol; use serde::Deserialize; @@ -19,12 +20,12 @@ use crate::{bail, etl_error}; /// event trigger. Messages with this prefix contain JSON-encoded schema information. pub const DDL_MESSAGE_PREFIX: &str = "supabase_etl_ddl"; -/// Represents a DDL schema change message emitted by Postgres event trigger. +/// Represents a schema change message emitted by Postgres event trigger. /// /// This message is emitted when ALTER TABLE commands are executed on tables /// that are part of a publication. #[derive(Debug, Clone, Deserialize)] -pub struct DdlSchemaChangeMessage { +pub struct SchemaChangeMessage { /// The DDL command that triggered this message (e.g., "ALTER TABLE"). pub event: String, /// The schema name of the affected table. @@ -32,15 +33,51 @@ pub struct DdlSchemaChangeMessage { /// The name of the affected table. pub table_name: String, /// The OID of the affected table. + /// + /// PostgreSQL table OIDs are `u32` values, but JSON serialization from the event trigger + /// uses `bigint` (i64) for transmission. The cast back to `u32` in [`into_table_schema`] + /// is safe because PostgreSQL OIDs are always within the `u32` range. pub table_id: i64, /// The columns of the table after the schema change. - pub columns: Vec, + pub columns: Vec, } -/// Represents a column schema in a DDL schema change message. +impl SchemaChangeMessage { + /// Converts a [`SchemaChangeMessage`] to a [`TableSchema`] with a specific snapshot ID. + /// + /// This is used to update the stored table schema when a DDL change is detected. + /// The snapshot_id should be the start_lsn of the DDL message. + pub fn into_table_schema(self, snapshot_id: SnapshotId) -> TableSchema { + let table_name = TableName::new(self.schema_name, self.table_name); + let column_schemas = self + .columns + .into_iter() + .map(|column| { + let typ = convert_type_oid_to_type(column.type_oid); + ColumnSchema::new( + column.name, + typ, + column.type_modifier, + column.ordinal_position, + column.primary_key_ordinal_position, + column.nullable, + ) + }) + .collect(); + + TableSchema::with_snapshot_id( + TableId::new(self.table_id as u32), + table_name, + column_schemas, + snapshot_id, + ) + } +} + +/// Represents a column schema in a schema change message. #[allow(dead_code)] #[derive(Debug, Clone, Deserialize)] -pub struct DdlColumnSchema { +pub struct ColumnSchemaMessage { /// The name of the column. pub name: String, /// The OID of the column's data type. @@ -315,41 +352,12 @@ pub fn convert_tuple_to_row<'a>( /// Parses a DDL schema change message from its JSON content. /// /// Returns the parsed message if successful, or an error if the JSON is malformed. -pub fn parse_ddl_schema_change_message(content: &str) -> EtlResult { +pub fn parse_schema_change_message(content: &str) -> EtlResult { serde_json::from_str(content).map_err(|e| { etl_error!( ErrorKind::ConversionError, - "Failed to parse DDL schema change message", - format!("Invalid JSON in DDL message: {}", e) + "Failed to parse schema change message", + format!("Invalid JSON in schema change message: {}", e) ) }) } - -/// Converts a [`DdlSchemaChangeMessage`] to a [`TableSchema`]. -/// -/// This is used to update the stored table schema when a DDL change is detected. -#[allow(dead_code)] -pub fn ddl_message_to_table_schema(message: &DdlSchemaChangeMessage) -> TableSchema { - let table_name = TableName::new(message.schema_name.clone(), message.table_name.clone()); - let column_schemas = message - .columns - .iter() - .map(|col| { - let typ = convert_type_oid_to_type(col.type_oid); - ColumnSchema::new( - col.name.clone(), - typ, - col.type_modifier, - col.ordinal_position, - col.primary_key_ordinal_position, - col.nullable, - ) - }) - .collect(); - - TableSchema::new( - TableId::new(message.table_id as u32), - table_name, - column_schemas, - ) -} diff --git a/etl/src/error.rs b/etl/src/error.rs index bd61a8372..7175134eb 100644 --- a/etl/src/error.rs +++ b/etl/src/error.rs @@ -100,6 +100,7 @@ pub enum ErrorKind { SourceSchemaError, MissingTableSchema, MissingTableMapping, + CorruptedTableSchema, DestinationTableNameInvalid, DestinationNamespaceAlreadyExists, DestinationTableAlreadyExists, @@ -1032,19 +1033,32 @@ impl From for EtlErro } } -/// Converts [`etl_postgres::types::SchemaError`] to [`EtlError`] with [`ErrorKind::InvalidState`]. +/// Converts [`etl_postgres::types::SchemaError`] to [`EtlError`] with [`ErrorKind::CorruptedTableSchema`]. impl From for EtlError { #[track_caller] fn from(err: etl_postgres::types::SchemaError) -> EtlError { match err { etl_postgres::types::SchemaError::UnknownReplicatedColumns(columns) => { EtlError::from_components( - ErrorKind::InvalidState, + ErrorKind::CorruptedTableSchema, Cow::Borrowed( "Received columns during replication that are not in the stored table schema", ), Some(Cow::Owned(format!( - "The columns that are not in the table schema are: {columns:?}" + "Unknown columns: {columns:?}\n\n\ + Cause: The pipeline crashed after a schema change but before reporting progress \ + back to Postgres. On restart, event streaming resumed from past events with an \ + outdated schema." + ))), + None, + ) + } + etl_postgres::types::SchemaError::InvalidSnapshotId(lsn_str) => { + EtlError::from_components( + ErrorKind::CorruptedTableSchema, + Cow::Borrowed("Invalid snapshot id"), + Some(Cow::Owned(format!( + "Failed to parse snapshot '{lsn_str}' as PgLsn." ))), None, ) diff --git a/etl/src/failpoints.rs b/etl/src/failpoints.rs index 7c3073a31..b4c4551a9 100644 --- a/etl/src/failpoints.rs +++ b/etl/src/failpoints.rs @@ -8,9 +8,10 @@ use fail::fail_point; use crate::bail; use crate::error::{ErrorKind, EtlResult}; -pub const START_TABLE_SYNC_BEFORE_DATA_SYNC_SLOT_CREATION: &str = - "start_table_sync.before_data_sync_slot_creation"; -pub const START_TABLE_SYNC_DURING_DATA_SYNC: &str = "start_table_sync.during_data_sync"; +pub const START_TABLE_SYNC_BEFORE_DATA_SYNC_SLOT_CREATION_FP: &str = + "start_table_sync.before_data_sync_slot_creation_fp"; +pub const START_TABLE_SYNC_DURING_DATA_SYNC_FP: &str = "start_table_sync.during_data_sync_fp"; +pub const SEND_STATUS_UPDATE_FP: &str = "send_status_update_fp"; /// Executes a configurable failpoint for testing error scenarios. /// @@ -45,3 +46,10 @@ pub fn etl_fail_point(name: &str) -> EtlResult<()> { Ok(()) } + +/// Returns `true` if a specific failpoint is active, `false` otherwise. +/// +/// A failpoint is considered active if it throws an error. +pub fn etl_fail_point_active(name: &str) -> bool { + etl_fail_point(name).is_err() +} diff --git a/etl/src/replication/apply.rs b/etl/src/replication/apply.rs index 04b0aff27..e9bed5c0e 100644 --- a/etl/src/replication/apply.rs +++ b/etl/src/replication/apply.rs @@ -3,10 +3,10 @@ use crate::concurrency::shutdown::ShutdownRx; use crate::concurrency::signal::SignalRx; use crate::concurrency::stream::{TimeoutStream, TimeoutStreamResult}; use crate::conversions::event::{ - DDL_MESSAGE_PREFIX, parse_ddl_schema_change_message, parse_event_from_begin_message, - parse_event_from_commit_message, parse_event_from_delete_message, - parse_event_from_insert_message, parse_event_from_truncate_message, - parse_event_from_update_message, parse_replicated_column_names, + DDL_MESSAGE_PREFIX, parse_event_from_begin_message, parse_event_from_commit_message, + parse_event_from_delete_message, parse_event_from_insert_message, + parse_event_from_truncate_message, parse_event_from_update_message, + parse_replicated_column_names, parse_schema_change_message, }; use crate::destination::Destination; use crate::error::{ErrorKind, EtlResult}; @@ -19,10 +19,12 @@ use crate::replication::client::PgReplicationClient; use crate::replication::masks::ReplicationMasks; use crate::replication::stream::EventsStream; use crate::store::schema::SchemaStore; -use crate::types::{Event, PipelineId}; +use crate::types::{Event, PipelineId, RelationEvent}; use etl_config::shared::PipelineConfig; use etl_postgres::replication::worker::WorkerType; -use etl_postgres::types::{ReplicatedTableSchema, ReplicationMask, TableId, TableSchema}; +use etl_postgres::types::{ + ReplicatedTableSchema, ReplicationMask, SnapshotId, TableId, TableSchema, +}; use futures::StreamExt; use metrics::histogram; use postgres_replication::protocol; @@ -291,14 +293,22 @@ struct ApplyLoopState { /// transaction boundary is found. If not found, the process will continue until it is killed via /// a `SIGKILL`. shutdown_discarded: bool, + /// The current schema snapshot being tracked. + /// + /// This is updated when DDL messages are processed, tracking the latest schema version. + current_schema_snapshot_id: SnapshotId, } impl ApplyLoopState { - /// Creates a new [`ApplyLoopState`] with initial status update and event batch. + /// Creates a new [`ApplyLoopState`] with initial status update, event batch, and schema snapshot. /// /// This constructor initializes the state tracking structure used throughout /// the apply loop to maintain replication progress and coordinate batching. - fn new(next_status_update: StatusUpdate, events_batch: Vec) -> Self { + fn new( + next_status_update: StatusUpdate, + events_batch: Vec, + current_schema_snapshot_id: SnapshotId, + ) -> Self { Self { last_commit_end_lsn: None, remote_final_lsn: None, @@ -308,9 +318,15 @@ impl ApplyLoopState { current_tx_begin_ts: None, current_tx_events: 0, shutdown_discarded: false, + current_schema_snapshot_id, } } + /// Updates the current schema snapshot to a new value. + fn update_schema_snapshot_id(&mut self, snapshot_id: SnapshotId) { + self.current_schema_snapshot_id = snapshot_id; + } + /// Updates the last commit end LSN to track transaction boundaries. /// /// This method maintains the highest commit end LSN seen, which represents @@ -477,6 +493,12 @@ where return Ok(result); } + // Initialize the current schema snapshot from the start LSN (this is fine to do even if the + // start lsn is not like any of the existing snapshot ids since the system is designed to return + // the biggest snapshot id <= the current snapshot id). + // Schemas will be loaded on-demand when get_table_schema is called. + let current_schema_snapshot_id: SnapshotId = start_lsn.into(); + // The first status update is defaulted from the start lsn since at this point we haven't // processed anything. let first_status_update = StatusUpdate { @@ -514,6 +536,7 @@ where let mut state = ApplyLoopState::new( first_status_update, Vec::with_capacity(config.batch.max_size), + current_schema_snapshot_id, ); // Main event processing loop - continues until shutdown or fatal error @@ -871,21 +894,28 @@ where .await } -/// Retrieves a table schema from the schema store by table ID. +/// Retrieves a table schema from the schema store by table ID and snapshot. /// /// Returns an error if the schema is not found in the store. -async fn get_table_schema(schema_store: &S, table_id: &TableId) -> EtlResult> +async fn get_table_schema( + schema_store: &S, + table_id: &TableId, + snapshot_id: SnapshotId, +) -> EtlResult> where S: SchemaStore, { schema_store - .get_table_schema(table_id) + .get_table_schema(table_id, snapshot_id) .await? .ok_or_else(|| { crate::etl_error!( ErrorKind::MissingTableSchema, - "Table schema not found in cache", - format!("Table schema for table {} not found in cache", table_id) + "Table schema not found", + format!( + "Table schema for table {} at snapshot {} not found", + table_id, snapshot_id + ) ) }) } @@ -993,8 +1023,15 @@ where handle_commit_message(state, start_lsn, commit_body, hook, pipeline_id).await } LogicalReplicationMessage::Relation(relation_body) => { - handle_relation_message(state, relation_body, hook, schema_store, replication_masks) - .await + handle_relation_message( + state, + start_lsn, + relation_body, + hook, + schema_store, + replication_masks, + ) + .await } LogicalReplicationMessage::Insert(insert_body) => { handle_insert_message( @@ -1041,7 +1078,15 @@ where .await } LogicalReplicationMessage::Message(message_body) => { - handle_logical_message(state, message_body, hook).await + handle_logical_message( + state, + start_lsn, + message_body, + hook, + schema_store, + replication_masks, + ) + .await } message => { debug!("received unsupported message: {:?}", message); @@ -1194,11 +1239,21 @@ where /// a [`ReplicationMask`] which is stored in the shared [`ReplicationMasks`] container. /// /// The mask is built by matching the replicated column names from the RELATION message against -/// the table schema from the schema store. If the schema is not yet available (e.g., during -/// initial table sync before the schema is stored), the mask building is skipped and will be -/// handled when the table sync worker completes and sets the mask. +/// the table schema from the schema store. +/// +/// Emits an [`Event::Relation`] containing the [`ReplicatedTableSchema`] to notify downstream +/// consumers about which columns are being replicated for this table. +/// +/// # Errors +/// +/// Returns [`ErrorKind::CorruptedTableSchema`] if the replicated columns in the `Relation` +/// message do not match the stored table schema. This can occur when DDL changes happen +/// but the pipeline crashes before acknowledging progress, causing the stored schema to +/// be out of sync with the source. Manual intervention is required to update the stored +/// schema before the pipeline can continue. async fn handle_relation_message( state: &ApplyLoopState, + start_lsn: PgLsn, message: &protocol::RelationBody, hook: &T, schema_store: &S, @@ -1212,7 +1267,7 @@ where bail!( ErrorKind::InvalidState, "Invalid transaction state", - "Transaction must be active before processing INSERT message" + "Transaction must be active before processing RELATION message" ); }; @@ -1228,7 +1283,8 @@ where let replicated_columns = parse_replicated_column_names(message)?; - let table_schema = get_table_schema(schema_store, &table_id).await?; + let table_schema = + get_table_schema(schema_store, &table_id, state.current_schema_snapshot_id).await?; info!( table_id = %table_id, @@ -1236,10 +1292,31 @@ where "received relation message, building replication mask" ); - let replication_mask = ReplicationMask::build(&table_schema, &replicated_columns)?; - replication_masks.set(table_id, replication_mask).await; + // Build the replication mask by validating that all replicated columns exist in the schema. + // If validation fails, it indicates the stored schema is out of sync with the source + // database and requires manual intervention to update the stored schema. + // + // TODO: Currently we fail and require manual intervention. In the future, we might want to + // handle this case automatically (e.g., by rebuilding the schema from the source) if this + // error becomes common. + let replication_mask = ReplicationMask::try_build(&table_schema, &replicated_columns)?; - Ok(HandleMessageResult::no_event()) + replication_masks + .set(table_id, replication_mask.clone()) + .await; + + // Build the ReplicatedTableSchema and emit a Relation event. + let replicated_table_schema = ReplicatedTableSchema::from_mask(table_schema, replication_mask); + + let relation_event = RelationEvent { + start_lsn, + commit_lsn: remote_final_lsn, + replicated_table_schema, + }; + + Ok(HandleMessageResult::return_event(Event::Relation( + relation_event, + ))) } /// Handles Postgres INSERT messages for row insertion events. @@ -1272,8 +1349,13 @@ where return Ok(HandleMessageResult::no_event()); } - let replicated_table_schema = - get_replicated_table_schema(&table_id, schema_store, replication_masks).await?; + let replicated_table_schema = get_replicated_table_schema( + &table_id, + state.current_schema_snapshot_id, + schema_store, + replication_masks, + ) + .await?; // Convert event from the protocol message. let event = parse_event_from_insert_message( @@ -1316,8 +1398,13 @@ where return Ok(HandleMessageResult::no_event()); } - let replicated_table_schema = - get_replicated_table_schema(&table_id, schema_store, replication_masks).await?; + let replicated_table_schema = get_replicated_table_schema( + &table_id, + state.current_schema_snapshot_id, + schema_store, + replication_masks, + ) + .await?; // Convert event from the protocol message. let event = parse_event_from_update_message( @@ -1360,8 +1447,13 @@ where return Ok(HandleMessageResult::no_event()); } - let replicated_table_schema = - get_replicated_table_schema(&table_id, schema_store, replication_masks).await?; + let replicated_table_schema = get_replicated_table_schema( + &table_id, + state.current_schema_snapshot_id, + schema_store, + replication_masks, + ) + .await?; // Convert event from the protocol message. let event = parse_event_from_delete_message( @@ -1409,8 +1501,13 @@ where .should_apply_changes(table_id, remote_final_lsn) .await? { - let replicated_table_schema = - get_replicated_table_schema(&table_id, schema_store, replication_masks).await?; + let replicated_table_schema = get_replicated_table_schema( + &table_id, + state.current_schema_snapshot_id, + schema_store, + replication_masks, + ) + .await?; truncated_tables.push(replicated_table_schema); } } @@ -1430,59 +1527,94 @@ where /// Handles a logical replication message. /// /// Processes `pg_logical_emit_message` messages from the replication stream. -/// -/// Currently handles DDL schema change messages with the `supabase_etl_ddl` prefix -/// for tracking schema changes. -async fn handle_logical_message( - state: &ApplyLoopState, +/// Handles DDL schema change messages with the `supabase_etl_ddl` prefix by +/// storing the new schema version with the start_lsn as the snapshot_id. +async fn handle_logical_message( + state: &mut ApplyLoopState, + start_lsn: PgLsn, message: &protocol::MessageBody, hook: &T, + schema_store: &S, + replication_masks: &ReplicationMasks, ) -> EtlResult where + S: SchemaStore, T: ApplyLoopHook, { // If the prefix is unknown, we don't want to process it. let prefix = message.prefix()?; if prefix != DDL_MESSAGE_PREFIX { + info!( + prefix = %prefix, + "received logical message with unknown prefix, discarding" + ); + return Ok(HandleMessageResult::no_event()); } + // DDL messages must be transactional (emitted with transactional=true in pg_logical_emit_message). + // This ensures they are part of a transaction and have a valid commit LSN for ordering. + let Some(remote_final_lsn) = state.remote_final_lsn else { + bail!( + ErrorKind::InvalidState, + "Invalid transaction state", + "DDL schema change messages must be transactional (transactional=true). \ + Received a DDL message outside of a transaction boundary." + ); + }; + let content = message.content()?; - let Ok(message) = parse_ddl_schema_change_message(content) else { + let Ok(schema_change_message) = parse_schema_change_message(content) else { bail!( - ErrorKind::SourceConnectionFailed, - "PostgreSQL connection has been closed during the apply loop" + ErrorKind::InvalidData, + "Failed to parse DDL schema change message", + "Invalid JSON format in schema change message content" ); }; - let table_id = TableId::new(message.table_id as u32); - // TODO: check if this check is required or we can leverage the idempotency of schema writing and - // we always unconditionally update the schema. - if let Some(remote_final_lsn) = state.remote_final_lsn { - if !hook - .should_apply_changes(table_id, remote_final_lsn) - .await? - { - return Ok(HandleMessageResult::no_event()); - } + let table_id = TableId::new(schema_change_message.table_id as u32); + if !hook + .should_apply_changes(table_id, remote_final_lsn) + .await? + { + return Ok(HandleMessageResult::no_event()); } info!( - table_id = message.table_id, - table_name = %message.table_name, - schema_name = %message.schema_name, - event = %message.event, - columns = message.columns.len(), + table_id = schema_change_message.table_id, + table_name = %schema_change_message.table_name, + schema_name = %schema_change_message.schema_name, + event = %schema_change_message.event, + columns = schema_change_message.columns.len(), "received ddl schema change message" ); - // TODO: In the future, update the stored schema here based on the start_lsn of the - // event as identifier. + // Build table schema from DDL message with start_lsn as the snapshot_id. + let snapshot_id: SnapshotId = start_lsn.into(); + let table_schema = schema_change_message.into_table_schema(snapshot_id); + + // Store the new schema version in the store. + schema_store.store_table_schema(table_schema).await?; + + // Update the current schema snapshot in the state. + state.update_schema_snapshot_id(snapshot_id); + + // Invalidate the cached replication mask for this table. While PostgreSQL guarantees that + // a RELATION message will be sent before any DML events after a schema change, we + // proactively invalidate the mask to ensure consistency. + replication_masks.remove(&table_id).await; + + let table_id: u32 = table_id.into(); + info!( + table_id = table_id, + %snapshot_id, + "stored new schema version from ddl message" + ); Ok(HandleMessageResult::no_event()) } -/// Retrieves a [`ReplicatedTableSchema`] for the given table. +/// Retrieves a [`ReplicatedTableSchema`] for the given table at the specified snapshot. /// /// This function combines the table schema from the schema store with the replication mask /// from the shared [`ReplicationMasks`] to create a [`ReplicatedTableSchema`]. @@ -1493,6 +1625,7 @@ where /// container, or if the table schema is not found in the schema store. async fn get_replicated_table_schema( table_id: &TableId, + snapshot_id: SnapshotId, schema_store: &S, replication_masks: &ReplicationMasks, ) -> EtlResult @@ -1510,7 +1643,7 @@ where ); }; - let table_schema = get_table_schema(schema_store, table_id).await?; + let table_schema = get_table_schema(schema_store, table_id, snapshot_id).await?; Ok(ReplicatedTableSchema::from_mask( table_schema, diff --git a/etl/src/replication/client.rs b/etl/src/replication/client.rs index 43ad1af29..e6ab59f0e 100644 --- a/etl/src/replication/client.rs +++ b/etl/src/replication/client.rs @@ -700,11 +700,7 @@ impl PgReplicationClient { let table_name = self.get_table_name(table_id).await?; let column_schemas = self.get_column_schemas(table_id).await?; - Ok(TableSchema { - name: table_name, - id: table_id, - column_schemas, - }) + Ok(TableSchema::new(table_id, table_name, column_schemas)) } /// Loads the table name and schema information for a given table OID. diff --git a/etl/src/replication/masks.rs b/etl/src/replication/masks.rs index dd20fc5ae..06a946168 100644 --- a/etl/src/replication/masks.rs +++ b/etl/src/replication/masks.rs @@ -54,6 +54,17 @@ impl ReplicationMasks { let guard = self.inner.read().await; guard.get(table_id).cloned() } + + /// Removes the replication mask for a table. + /// + /// This is called after processing a DDL schema change message to invalidate the cached + /// mask. While PostgreSQL guarantees that a RELATION message will be sent before any DML + /// events after a schema change, we proactively invalidate the mask to ensure consistency. + /// The next RELATION message will rebuild the mask with the updated schema. + pub async fn remove(&self, table_id: &TableId) { + let mut guard = self.inner.write().await; + guard.remove(table_id); + } } #[cfg(test)] @@ -76,7 +87,7 @@ mod tests { let replicated_columns: HashSet = ["id".to_string(), "age".to_string()].into_iter().collect(); - ReplicationMask::build(&schema, &replicated_columns).unwrap() + ReplicationMask::build(&schema, &replicated_columns) } #[tokio::test] diff --git a/etl/src/replication/stream.rs b/etl/src/replication/stream.rs index 69950dc0d..2df9cb256 100644 --- a/etl/src/replication/stream.rs +++ b/etl/src/replication/stream.rs @@ -10,10 +10,14 @@ use std::time::{Duration, Instant}; use tokio_postgres::CopyOutStream; use tokio_postgres::types::PgLsn; use tracing::debug; +#[cfg(feature = "failpoints")] +use tracing::warn; use crate::conversions::table_row::parse_table_row_from_postgres_copy_bytes; use crate::error::{ErrorKind, EtlResult}; use crate::etl_error; +#[cfg(feature = "failpoints")] +use crate::failpoints::{SEND_STATUS_UPDATE_FP, etl_fail_point_active}; use crate::metrics::{ETL_COPIED_TABLE_ROW_SIZE_BYTES, PIPELINE_ID_LABEL}; use crate::types::{PipelineId, TableRow}; use metrics::histogram; @@ -127,6 +131,15 @@ impl EventsStream { mut flush_lsn: PgLsn, force: bool, ) -> EtlResult<()> { + // If the failpoint is active, we do not send any status update. This is useful for testing + // the system when we want to check what happens when no status updates are sent. + #[cfg(feature = "failpoints")] + if etl_fail_point_active(SEND_STATUS_UPDATE_FP) { + warn!("not sending status update due to active failpoint"); + + return Ok(()); + } + let this = self.project(); // If the new LSN is less than the last one, we can safely ignore it, since we only want diff --git a/etl/src/replication/table_sync.rs b/etl/src/replication/table_sync.rs index 7a68c234a..64082952b 100644 --- a/etl/src/replication/table_sync.rs +++ b/etl/src/replication/table_sync.rs @@ -1,6 +1,6 @@ use etl_config::shared::PipelineConfig; use etl_postgres::replication::slots::EtlReplicationSlot; -use etl_postgres::types::{ReplicatedTableSchema, ReplicationMask, TableId}; +use etl_postgres::types::{ReplicatedTableSchema, ReplicationMask, SchemaError, TableId}; use futures::StreamExt; use metrics::histogram; use std::sync::Arc; @@ -17,7 +17,7 @@ use crate::destination::Destination; use crate::error::{ErrorKind, EtlResult}; #[cfg(feature = "failpoints")] use crate::failpoints::{ - START_TABLE_SYNC_BEFORE_DATA_SYNC_SLOT_CREATION, START_TABLE_SYNC_DURING_DATA_SYNC, + START_TABLE_SYNC_BEFORE_DATA_SYNC_SLOT_CREATION_FP, START_TABLE_SYNC_DURING_DATA_SYNC_FP, etl_fail_point, }; use crate::metrics::{ @@ -163,7 +163,7 @@ where // Fail point to test when the table sync fails before copying data. #[cfg(feature = "failpoints")] - etl_fail_point(START_TABLE_SYNC_BEFORE_DATA_SYNC_SLOT_CREATION)?; + etl_fail_point(START_TABLE_SYNC_BEFORE_DATA_SYNC_SLOT_CREATION_FP)?; // We create the slot with a transaction, since we need to have a consistent snapshot of the database // before copying the schema and tables. @@ -202,7 +202,18 @@ where .await?; // Build and store the replication mask for use during CDC. - let replication_mask = ReplicationMask::build(&table_schema, &replicated_column_names)?; + // We use `try_build` here because the schema was just loaded and should match + // the publication's column filter. Any mismatch indicates a schema inconsistency. + let replication_mask = + ReplicationMask::try_build(&table_schema, &replicated_column_names).map_err( + |err: SchemaError| { + crate::etl_error!( + ErrorKind::InvalidState, + "Schema mismatch during table sync", + format!("{}", err) + ) + }, + )?; replication_masks .set(table_id, replication_mask.clone()) .await; @@ -298,7 +309,7 @@ where // Fail point to test when the table sync fails after copying one batch. #[cfg(feature = "failpoints")] - etl_fail_point(START_TABLE_SYNC_DURING_DATA_SYNC)?; + etl_fail_point(START_TABLE_SYNC_DURING_DATA_SYNC_FP)?; } ShutdownResult::Shutdown(_) => { // If we received a shutdown in the middle of a table copy, we bail knowing diff --git a/etl/src/state/table.rs b/etl/src/state/table.rs index fbe1d5c82..2c7a45c61 100644 --- a/etl/src/state/table.rs +++ b/etl/src/state/table.rs @@ -148,6 +148,12 @@ impl TableReplicationError { "Check replication slot status and database configuration.", RetryPolicy::ManualRetry, ), + ErrorKind::CorruptedTableSchema => Self::with_solution( + table_id, + error, + "Manually update the stored table schema with the current schema of the table.", + RetryPolicy::ManualRetry, + ), // Special handling for error kinds used during failure injection. #[cfg(feature = "failpoints")] diff --git a/etl/src/store/both/memory.rs b/etl/src/store/both/memory.rs index 34376e317..5efcb3bbb 100644 --- a/etl/src/store/both/memory.rs +++ b/etl/src/store/both/memory.rs @@ -1,4 +1,4 @@ -use etl_postgres::types::{TableId, TableSchema}; +use etl_postgres::types::{SnapshotId, TableId, TableSchema}; use std::collections::HashMap; use std::sync::Arc; use tokio::sync::Mutex; @@ -20,10 +20,8 @@ struct Inner { /// This is an append-only log that grows over time and provides visibility into /// table state evolution. Entries are chronologically ordered. table_state_history: HashMap>, - /// Cached table schema definitions, reference-counted for efficient sharing. - /// Schemas are expensive to fetch from Postgres, so they're cached here - /// once retrieved and shared via Arc across the application. - table_schemas: HashMap>, + /// Cached table schemas keyed by (TableId, SnapshotId) for versioning support. + table_schemas: HashMap<(TableId, SnapshotId), Arc>, /// Mapping from table IDs to human-readable table names for easier debugging /// and logging. These mappings are established during schema discovery. table_mappings: HashMap, @@ -172,10 +170,26 @@ impl StateStore for MemoryStore { } impl SchemaStore for MemoryStore { - async fn get_table_schema(&self, table_id: &TableId) -> EtlResult>> { + /// Returns the table schema for the given table at the specified snapshot point. + /// + /// Returns the schema version with the largest snapshot_id <= the requested snapshot_id. + /// For MemoryStore, this only looks in the in-memory cache. + async fn get_table_schema( + &self, + table_id: &TableId, + snapshot_id: SnapshotId, + ) -> EtlResult>> { let inner = self.inner.lock().await; - Ok(inner.table_schemas.get(table_id).cloned()) + // Find the best matching schema (largest snapshot_id <= requested) + let best_match = inner + .table_schemas + .iter() + .filter(|((tid, sid), _)| *tid == *table_id && *sid <= snapshot_id) + .max_by_key(|((_, sid), _)| *sid) + .map(|(_, schema)| schema.clone()); + + Ok(best_match) } async fn get_table_schemas(&self) -> EtlResult>> { @@ -193,10 +207,9 @@ impl SchemaStore for MemoryStore { async fn store_table_schema(&self, table_schema: TableSchema) -> EtlResult> { let mut inner = self.inner.lock().await; + let key = (table_schema.id, table_schema.snapshot_id); let table_schema = Arc::new(table_schema); - inner - .table_schemas - .insert(table_schema.id, table_schema.clone()); + inner.table_schemas.insert(key, table_schema.clone()); Ok(table_schema) } @@ -208,7 +221,8 @@ impl CleanupStore for MemoryStore { inner.table_replication_states.remove(&table_id); inner.table_state_history.remove(&table_id); - inner.table_schemas.remove(&table_id); + // Remove all schema versions for this table + inner.table_schemas.retain(|(tid, _), _| *tid != table_id); inner.table_mappings.remove(&table_id); Ok(()) diff --git a/etl/src/store/both/postgres.rs b/etl/src/store/both/postgres.rs index e754bd0ec..e50afb43f 100644 --- a/etl/src/store/both/postgres.rs +++ b/etl/src/store/both/postgres.rs @@ -2,7 +2,7 @@ use std::{collections::HashMap, sync::Arc}; use etl_config::shared::PgConnectionConfig; use etl_postgres::replication::{connect_to_source_database, schema, state, table_mappings}; -use etl_postgres::types::{TableId, TableSchema}; +use etl_postgres::types::{SnapshotId, TableId, TableSchema}; use metrics::gauge; use sqlx::PgPool; use tokio::sync::Mutex; @@ -19,6 +19,13 @@ use crate::{bail, etl_error}; const NUM_POOL_CONNECTIONS: u32 = 1; +/// Maximum number of schema snapshots to keep cached per table. +/// +/// This limits memory usage by evicting older snapshots when new ones are added. +/// In practice, during a single batch of events, it's highly unlikely to need +/// more than 2 schema versions for any given table. +const MAX_CACHED_SCHEMAS_PER_TABLE: usize = 2; + /// Converts ETL table replication phases to Postgres database state format. /// /// This conversion transforms internal ETL replication states into the format @@ -135,8 +142,14 @@ struct Inner { phase_counts: HashMap<&'static str, u64>, /// Cached table replication states indexed by table ID. table_states: HashMap, - /// Cached table schemas indexed by table ID. - table_schemas: HashMap>, + /// Cached table schemas indexed by (table_id, snapshot_id) for versioning support. + /// + /// This cache is optimized for keeping the most actively used schemas in memory, + /// not all historical snapshots. Schemas are loaded on-demand from the database + /// when not found in cache. During normal operation, this typically contains + /// only the latest schema version for each table, since that's what the + /// replication pipeline actively uses. + table_schemas: HashMap<(TableId, SnapshotId), Arc>, /// Cached table mappings from source table ID to destination table name. table_mappings: HashMap, } @@ -151,6 +164,39 @@ impl Inner { let count = self.phase_counts.entry(phase).or_default(); *count += 1; } + + /// Inserts a schema into the cache and evicts older snapshots if necessary. + /// + /// Maintains at most [`MAX_CACHED_SCHEMAS_PER_TABLE`] snapshots per table, + /// evicting the oldest snapshots when the limit is exceeded. + fn insert_schema_with_eviction(&mut self, table_schema: Arc) { + let table_id = table_schema.id; + let snapshot_id = table_schema.snapshot_id; + + // Insert the new schema + self.table_schemas + .insert((table_id, snapshot_id), table_schema); + + // Collect all snapshot_ids for this table + let mut snapshots_for_table: Vec = self + .table_schemas + .keys() + .filter(|(tid, _)| *tid == table_id) + .map(|(_, sid)| *sid) + .collect(); + + // If we exceed the limit, evict oldest snapshots + if snapshots_for_table.len() > MAX_CACHED_SCHEMAS_PER_TABLE { + // Sort ascending so oldest are first + snapshots_for_table.sort(); + + // Remove oldest entries until we're at the limit + let to_remove = snapshots_for_table.len() - MAX_CACHED_SCHEMAS_PER_TABLE; + for &old_snapshot_id in snapshots_for_table.iter().take(to_remove) { + self.table_schemas.remove(&(table_id, old_snapshot_id)); + } + } + } } /// Postgres-backed storage for ETL pipeline state and schema information. @@ -501,22 +547,84 @@ impl StateStore for PostgresStore { } impl SchemaStore for PostgresStore { - /// Retrieves a table schema from cache by table ID. + /// Retrieves a table schema at a specific snapshot point. /// - /// This method provides fast access to cached table schemas, which are - /// essential for processing replication events. Schemas are loaded during - /// startup and cached for the lifetime of the pipeline. - async fn get_table_schema(&self, table_id: &TableId) -> EtlResult>> { - let inner = self.inner.lock().await; + /// Returns the schema version with the largest snapshot_id <= the requested snapshot_id. + /// First checks the in-memory cache, then loads from the database if not found. + /// The loaded schema is cached for subsequent requests. Note that the cache is + /// optimized for active schemas, not historical snapshots. + async fn get_table_schema( + &self, + table_id: &TableId, + snapshot_id: SnapshotId, + ) -> EtlResult>> { + // First, check if we have a cached schema that matches the criteria. + // + // We can afford to hold the lock only for this short critical section since we assume that + // there is not really concurrency at the table level since each table is processed by exactly + // one worker. + { + let inner = self.inner.lock().await; + + // Find the best matching schema in the cache (largest snapshot_id <= requested). + let newest_table_schema = inner + .table_schemas + .iter() + .filter(|((tid, sid), _)| *tid == *table_id && *sid <= snapshot_id) + .max_by_key(|((_, sid), _)| *sid) + .map(|(_, schema)| schema.clone()); - Ok(inner.table_schemas.get(table_id).cloned()) + if newest_table_schema.is_some() { + return Ok(newest_table_schema); + } + } + + debug!( + "schema for table {} at snapshot {} not in cache, loading from database", + table_id, snapshot_id + ); + + let pool = self.connect_to_source().await?; + + // Load the schema at the requested snapshot. + let table_schema = schema::load_table_schema_at_snapshot( + &pool, + self.pipeline_id as i64, + *table_id, + snapshot_id, + ) + .await + .map_err(|err| { + etl_error!( + ErrorKind::SourceQueryFailed, + "Table schema loading failed", + format!( + "Failed to load table schema for table {} at snapshot {} from PostgreSQL: {}", + table_id, snapshot_id, err + ) + ) + })?; + + let Some(table_schema) = table_schema else { + return Ok(None); + }; + + let result = { + let mut inner = self.inner.lock().await; + + let table_schema = Arc::new(table_schema); + inner.insert_schema_with_eviction(table_schema.clone()); + + Some(table_schema) + }; + + Ok(result) } /// Retrieves all cached table schemas as a vector. /// /// This method returns all currently cached table schemas, providing a /// complete view of the schema information available to the pipeline. - /// Useful for operations that need to process or analyze all table schemas. async fn get_table_schemas(&self) -> EtlResult>> { let inner = self.inner.lock().await; @@ -525,8 +633,8 @@ impl SchemaStore for PostgresStore { /// Loads table schemas from Postgres into memory cache. /// - /// This method connects to the source database, retrieves schema information - /// for all tables in this pipeline, and populates the in-memory cache. + /// This method connects to the source database, retrieves the latest schema + /// version for all tables in this pipeline, and populates the in-memory cache. /// Called during pipeline initialization to establish the schema context /// needed for processing replication events. async fn load_table_schemas(&self) -> EtlResult { @@ -545,14 +653,11 @@ impl SchemaStore for PostgresStore { })?; let table_schemas_len = table_schemas.len(); - // For performance reasons, since we load the table schemas only once during startup - // and from a single thread, we can afford to have a super short critical section. let mut inner = self.inner.lock().await; inner.table_schemas.clear(); for table_schema in table_schemas { - inner - .table_schemas - .insert(table_schema.id, Arc::new(table_schema)); + let key = (table_schema.id, table_schema.snapshot_id); + inner.table_schemas.insert(key, Arc::new(table_schema)); } info!( @@ -566,15 +671,16 @@ impl SchemaStore for PostgresStore { /// Stores a table schema in both database and cache. /// /// This method persists a table schema to the database and updates the - /// in-memory cache atomically. Used when new tables are discovered during - /// replication or when schema definitions need to be updated. + /// in-memory cache atomically. The schema's snapshot_id determines which + /// version this schema represents. async fn store_table_schema(&self, table_schema: TableSchema) -> EtlResult> { - debug!("storing table schema for table '{}'", table_schema.name); + debug!( + "storing table schema for table '{}' at snapshot {}", + table_schema.name, table_schema.snapshot_id + ); let pool = self.connect_to_source().await?; - // We also lock the entire section to be consistent. - let mut inner = self.inner.lock().await; schema::store_table_schema(&pool, self.pipeline_id as i64, &table_schema) .await .map_err(|err| { @@ -584,10 +690,10 @@ impl SchemaStore for PostgresStore { format!("Failed to store table schema in PostgreSQL: {}", err) ) })?; + + let mut inner = self.inner.lock().await; let table_schema = Arc::new(table_schema); - inner - .table_schemas - .insert(table_schema.id, table_schema.clone()); + inner.insert_schema_with_eviction(table_schema.clone()); Ok(table_schema) } @@ -633,7 +739,8 @@ impl CleanupStore for PostgresStore { let mut inner = self.inner.lock().await; inner.table_states.remove(&table_id); - inner.table_schemas.remove(&table_id); + // Remove all schema versions for this table + inner.table_schemas.retain(|(tid, _), _| *tid != table_id); inner.table_mappings.remove(&table_id); emit_table_metrics( diff --git a/etl/src/store/schema/base.rs b/etl/src/store/schema/base.rs index c6e0917e0..d094336c4 100644 --- a/etl/src/store/schema/base.rs +++ b/etl/src/store/schema/base.rs @@ -1,4 +1,4 @@ -use etl_postgres::types::{TableId, TableSchema}; +use etl_postgres::types::{SnapshotId, TableId, TableSchema}; use std::sync::Arc; use crate::error::EtlResult; @@ -6,19 +6,25 @@ use crate::error::EtlResult; /// Trait for storing and retrieving database table schema information. /// /// [`SchemaStore`] implementations are responsible for defining how the schema information -/// is stored and retrieved. +/// is stored and retrieved. The store supports schema versioning where each schema version +/// is identified by a snapshot_id (the start_lsn of the DDL message that created it). /// /// Implementations should ensure thread-safety and handle concurrent access to the data. pub trait SchemaStore { - /// Returns table schema for table with id `table_id` from the cache. + /// Returns the table schema for the given table at the specified snapshot point. /// - /// Does not load any new data into the cache. + /// Returns the schema version with the largest snapshot_id that is <= the requested + /// snapshot_id. If not found in cache, loads from the persistent store. As an optimization, + /// also loads the latest schema version when fetching from the database. + /// + /// Returns `None` if no schema version exists for the table at or before the given snapshot. fn get_table_schema( &self, table_id: &TableId, + snapshot_id: SnapshotId, ) -> impl Future>>> + Send; - /// Returns all table schemas from the cache. + /// Returns all cached table schemas. /// /// Does not read from the persistent store. fn get_table_schemas(&self) -> impl Future>>> + Send; @@ -29,6 +35,8 @@ pub trait SchemaStore { fn load_table_schemas(&self) -> impl Future> + Send; /// Stores a table schema in both the cache and the persistent store. + /// + /// The schema's `snapshot_id` field determines which version this schema represents. fn store_table_schema( &self, table_schema: TableSchema, diff --git a/etl/src/test_utils/event.rs b/etl/src/test_utils/event.rs index 657c95bf9..80eae1a7b 100644 --- a/etl/src/test_utils/event.rs +++ b/etl/src/test_utils/event.rs @@ -21,8 +21,9 @@ pub fn group_events_by_type_and_table_id( let mut grouped = HashMap::new(); for event in events { let event_type = EventType::from(event); - // This grouping only works on simple DML operations. + // This grouping works on DML operations and Relation events. let table_ids = match event { + Event::Relation(event) => vec![event.replicated_table_schema.id()], Event::Insert(event) => vec![event.replicated_table_schema.id()], Event::Update(event) => vec![event.replicated_table_schema.id()], Event::Delete(event) => vec![event.replicated_table_schema.id()], diff --git a/etl/src/test_utils/mod.rs b/etl/src/test_utils/mod.rs index 9381d2b03..613748440 100644 --- a/etl/src/test_utils/mod.rs +++ b/etl/src/test_utils/mod.rs @@ -9,6 +9,6 @@ pub mod event; pub mod materialize; pub mod notify; pub mod pipeline; -pub mod table; +pub mod schema; pub mod test_destination_wrapper; pub mod test_schema; diff --git a/etl/src/test_utils/notify.rs b/etl/src/test_utils/notify.rs index c5600ebeb..c1adc840c 100644 --- a/etl/src/test_utils/notify.rs +++ b/etl/src/test_utils/notify.rs @@ -1,6 +1,6 @@ use std::{collections::HashMap, fmt, sync::Arc}; -use etl_postgres::types::{TableId, TableSchema}; +use etl_postgres::types::{SnapshotId, TableId, TableSchema}; use tokio::sync::{Notify, RwLock}; use crate::error::{ErrorKind, EtlResult}; @@ -29,7 +29,8 @@ type TableStateCondition = ( struct Inner { table_replication_states: HashMap, table_state_history: HashMap>, - table_schemas: HashMap>, + /// Stores table schemas in insertion order per table. + table_schemas: HashMap>>, table_mappings: HashMap, table_state_type_conditions: Vec, table_state_conditions: Vec, @@ -103,12 +104,35 @@ impl NotifyingStore { inner.table_replication_states.clone() } - pub async fn get_table_schemas(&self) -> HashMap { + pub async fn get_latest_table_schemas(&self) -> HashMap { let inner = self.inner.read().await; + + // Return the latest schema version for each table (last in the Vec). + inner + .table_schemas + .iter() + .filter_map(|(table_id, schemas)| { + schemas + .last() + .map(|schema| (*table_id, Arc::as_ref(schema).clone())) + }) + .collect() + } + + pub async fn get_table_schemas(&self) -> HashMap> { + let inner = self.inner.read().await; + + // Return schemas in insertion order per table. inner .table_schemas .iter() - .map(|(id, schema)| (*id, Arc::as_ref(schema).clone())) + .map(|(table_id, schemas)| { + let schemas_with_ids: Vec<_> = schemas + .iter() + .map(|schema| (schema.snapshot_id, Arc::as_ref(schema).clone())) + .collect(); + (*table_id, schemas_with_ids) + }) .collect() } @@ -290,30 +314,50 @@ impl StateStore for NotifyingStore { } impl SchemaStore for NotifyingStore { - async fn get_table_schema(&self, table_id: &TableId) -> EtlResult>> { + async fn get_table_schema( + &self, + table_id: &TableId, + snapshot_id: SnapshotId, + ) -> EtlResult>> { let inner = self.inner.read().await; - Ok(inner.table_schemas.get(table_id).cloned()) + // Find the best matching schema (largest snapshot_id <= requested). + let best_match = inner.table_schemas.get(table_id).and_then(|schemas| { + schemas + .iter() + .filter(|schema| schema.snapshot_id <= snapshot_id) + .max_by_key(|schema| schema.snapshot_id) + .cloned() + }); + + Ok(best_match) } async fn get_table_schemas(&self) -> EtlResult>> { let inner = self.inner.read().await; - Ok(inner.table_schemas.values().cloned().collect()) + Ok(inner + .table_schemas + .values() + .flat_map(|schemas| schemas.iter().cloned()) + .collect()) } async fn load_table_schemas(&self) -> EtlResult { let inner = self.inner.read().await; - Ok(inner.table_schemas.len()) + Ok(inner.table_schemas.values().map(|v| v.len()).sum()) } async fn store_table_schema(&self, table_schema: TableSchema) -> EtlResult> { let mut inner = self.inner.write().await; + let table_id = table_schema.id; let table_schema = Arc::new(table_schema); inner .table_schemas - .insert(table_schema.id, table_schema.clone()); + .entry(table_id) + .or_default() + .push(table_schema.clone()); Ok(table_schema) } diff --git a/etl/src/test_utils/pipeline.rs b/etl/src/test_utils/pipeline.rs index 045d21a87..4f316e3cb 100644 --- a/etl/src/test_utils/pipeline.rs +++ b/etl/src/test_utils/pipeline.rs @@ -1,12 +1,20 @@ -use etl_config::shared::{BatchConfig, PgConnectionConfig, PipelineConfig}; -use uuid::Uuid; - use crate::destination::Destination; +use crate::destination::memory::MemoryDestination; use crate::pipeline::Pipeline; +use crate::state::table::TableReplicationPhaseType; use crate::store::cleanup::CleanupStore; use crate::store::schema::SchemaStore; use crate::store::state::StateStore; +use crate::test_utils::database::{spawn_source_database, test_table_name}; +use crate::test_utils::notify::NotifyingStore; +use crate::test_utils::test_destination_wrapper::TestDestinationWrapper; use crate::types::PipelineId; +use etl_config::shared::{BatchConfig, PgConnectionConfig, PipelineConfig}; +use etl_postgres::tokio::test_utils::PgDatabase; +use etl_postgres::types::{TableId, TableName}; +use rand::random; +use tokio_postgres::Client; +use uuid::Uuid; /// Generates a test-specific replication slot name with a random component. /// @@ -73,3 +81,69 @@ where Pipeline::new(config, store, destination) } + +pub async fn create_database_and_pipeline_with_table( + table_suffix: &str, + columns: &[(&str, &str)], +) -> ( + PgDatabase, + TableName, + TableId, + NotifyingStore, + TestDestinationWrapper, + Pipeline>, + PipelineId, + String, +) { + let database = spawn_source_database().await; + + let table_name = test_table_name(table_suffix); + let table_id = database + .create_table(table_name.clone(), true, columns) + .await + .unwrap(); + + let publication_name = format!("pub_{}", random::()); + database + .create_publication(&publication_name, &[table_name.clone()]) + .await + .unwrap(); + + let store = NotifyingStore::new(); + let destination = TestDestinationWrapper::wrap(MemoryDestination::new()); + + let pipeline_id: PipelineId = random(); + let mut pipeline = create_pipeline( + &database.config, + pipeline_id, + publication_name.clone(), + store.clone(), + destination.clone(), + ); + + // We wait for sync done so that we have the apply worker dealing with events, this is the common + // testing condition which ensures that the table is ready to be streamed from the main apply worker. + // + // The rationale for wanting to test ETL mainly on the apply worker is that it's really hard to test + // ETL in a state before `SyncDone` since the system will advance on its own. To properly test all + // the table sync worker states, we would need a way to programmatically drive execution, but we deemed + // it too much work compared to the benefit it brings. + let sync_done = store + .notify_on_table_state_type(table_id, TableReplicationPhaseType::SyncDone) + .await; + + pipeline.start().await.unwrap(); + + sync_done.notified().await; + + ( + database, + table_name, + table_id, + store, + destination, + pipeline, + pipeline_id, + publication_name, + ) +} diff --git a/etl/src/test_utils/schema.rs b/etl/src/test_utils/schema.rs new file mode 100644 index 000000000..237b7f1a2 --- /dev/null +++ b/etl/src/test_utils/schema.rs @@ -0,0 +1,231 @@ +use crate::types::Type; +use etl_postgres::types::{ColumnSchema, ReplicatedTableSchema, SnapshotId, TableSchema}; + +/// Asserts that two column schemas are equal. +pub fn assert_column_schema_eq(actual: &ColumnSchema, expected: &ColumnSchema) { + assert_eq!( + actual.name, expected.name, + "column name mismatch: got '{}', expected '{}'", + actual.name, expected.name + ); + assert_eq!( + actual.typ, expected.typ, + "column '{}' type mismatch: got {:?}, expected {:?}", + actual.name, actual.typ, expected.typ + ); + assert_eq!( + actual.modifier, expected.modifier, + "column '{}' modifier mismatch: got {}, expected {}", + actual.name, actual.modifier, expected.modifier + ); + assert_eq!( + actual.nullable, expected.nullable, + "column '{}' nullable mismatch: got {}, expected {}", + actual.name, actual.nullable, expected.nullable + ); + assert_eq!( + actual.primary_key(), + expected.primary_key(), + "column '{}' primary_key mismatch: got {}, expected {}", + actual.name, + actual.primary_key(), + expected.primary_key() + ); +} + +/// Asserts that a column has the expected name and type. +pub fn assert_column_name_type(column: &ColumnSchema, expected_name: &str, expected_type: &Type) { + assert_eq!( + column.name, expected_name, + "column name mismatch: got '{}', expected '{expected_name}'", + column.name + ); + assert_eq!( + &column.typ, expected_type, + "column '{expected_name}' type mismatch: got {:?}, expected {expected_type:?}", + column.typ + ); +} + +/// Asserts that a column has the expected name. +pub fn assert_column_name(column: &ColumnSchema, expected_name: &str) { + assert_eq!( + column.name, expected_name, + "column name mismatch: got '{}', expected '{expected_name}'", + column.name + ); +} + +/// Asserts that columns match the expected column schemas. +pub fn assert_columns_eq<'a>( + columns: impl Iterator, + expected_columns: &[ColumnSchema], +) { + let columns: Vec<_> = columns.collect(); + assert_eq!( + columns.len(), + expected_columns.len(), + "column count mismatch: got {}, expected {}", + columns.len(), + expected_columns.len() + ); + + for (actual, expected) in columns.iter().zip(expected_columns.iter()) { + assert_column_schema_eq(actual, expected); + } +} + +/// Asserts that columns have the expected names and types. +pub fn assert_columns_names_types<'a>( + columns: impl Iterator, + expected_columns: &[(&str, Type)], +) { + let columns: Vec<_> = columns.collect(); + assert_eq!( + columns.len(), + expected_columns.len(), + "column count mismatch: got {}, expected {}", + columns.len(), + expected_columns.len() + ); + + for (i, (actual, (expected_name, expected_type))) in + columns.iter().zip(expected_columns.iter()).enumerate() + { + assert_eq!( + actual.name, *expected_name, + "column name mismatch at index {i}: got '{}', expected '{expected_name}'", + actual.name + ); + assert_eq!( + actual.typ, *expected_type, + "column '{expected_name}' type mismatch at index {i}: got {:?}, expected {expected_type:?}", + actual.typ + ); + } +} + +/// Asserts that columns have the expected names. +pub fn assert_columns_names<'a>( + columns: impl Iterator, + expected_names: &[&str], +) { + let columns: Vec<_> = columns.collect(); + assert_eq!( + columns.len(), + expected_names.len(), + "column count mismatch: got {}, expected {}", + columns.len(), + expected_names.len() + ); + + for (i, (actual, expected_name)) in columns.iter().zip(expected_names.iter()).enumerate() { + assert_eq!( + actual.name, *expected_name, + "column name mismatch at index {i}: got '{}', expected '{expected_name}'", + actual.name + ); + } +} + +/// Asserts that a table schema has columns matching the expected column schemas. +pub fn assert_table_schema_columns(schema: &TableSchema, expected_columns: &[ColumnSchema]) { + assert_columns_eq(schema.column_schemas.iter(), expected_columns); +} + +/// Asserts that a table schema has columns with the expected names and types. +pub fn assert_table_schema_column_names_types( + schema: &TableSchema, + expected_columns: &[(&str, Type)], +) { + assert_columns_names_types(schema.column_schemas.iter(), expected_columns); +} + +/// Asserts that a table schema has columns with the expected names. +pub fn assert_table_schema_column_names(schema: &TableSchema, expected_names: &[&str]) { + assert_columns_names(schema.column_schemas.iter(), expected_names); +} + +/// Asserts that a replicated table schema has columns matching the expected column schemas, +/// and that all columns are replicated. +pub fn assert_replicated_schema_columns( + schema: &ReplicatedTableSchema, + expected_columns: &[ColumnSchema], +) { + assert_columns_eq(schema.column_schemas(), expected_columns); + assert_all_columns_replicated(schema, expected_columns.len()); +} + +/// Asserts that a replicated table schema has columns with the expected names and types, +/// and that all columns are replicated. +pub fn assert_replicated_schema_column_names_types( + schema: &ReplicatedTableSchema, + expected_columns: &[(&str, Type)], +) { + assert_columns_names_types(schema.column_schemas(), expected_columns); + assert_all_columns_replicated(schema, expected_columns.len()); +} + +/// Asserts that a replicated table schema has columns with the expected names, +/// and that all columns are replicated. +pub fn assert_replicated_schema_column_names( + schema: &ReplicatedTableSchema, + expected_names: &[&str], +) { + assert_columns_names(schema.column_schemas(), expected_names); + assert_all_columns_replicated(schema, expected_names.len()); +} + +/// Asserts that all columns in the replication mask are set to 1. +fn assert_all_columns_replicated(schema: &ReplicatedTableSchema, expected_len: usize) { + let mask = schema.replication_mask().as_slice(); + assert_eq!( + mask.len(), + expected_len, + "replication mask length mismatch: got {}, expected {}", + mask.len(), + expected_len + ); + assert!( + mask.iter().all(|&bit| bit == 1), + "expected all columns to be replicated, but mask is {mask:?}" + ); +} + +/// Asserts that schema snapshots are in strictly increasing order by snapshot ID. +/// +/// If `first_is_zero` is true, the first snapshot ID must be 0. +/// If `first_is_zero` is false, the first snapshot ID must be > 0. +/// Each subsequent snapshot ID must be strictly greater than the previous one. +pub fn assert_schema_snapshots_ordering( + snapshots: &[(SnapshotId, TableSchema)], + first_is_zero: bool, +) { + assert!( + !snapshots.is_empty(), + "expected at least one schema snapshot" + ); + + let (first_snapshot_id, _) = &snapshots[0]; + if first_is_zero { + assert_eq!( + *first_snapshot_id, + SnapshotId::initial(), + "first snapshot_id is {first_snapshot_id}, expected 0" + ); + } else { + assert!( + *first_snapshot_id > SnapshotId::initial(), + "first snapshot_id is {first_snapshot_id}, expected > 0" + ); + } + + for i in 1..snapshots.len() { + let (prev_snapshot_id, _) = &snapshots[i - 1]; + let (snapshot_id, _) = &snapshots[i]; + assert!( + *snapshot_id > *prev_snapshot_id, + "snapshot at index {i} has snapshot_id {snapshot_id} which is not greater than previous snapshot_id {prev_snapshot_id}" + ); + } +} diff --git a/etl/src/test_utils/table.rs b/etl/src/test_utils/table.rs deleted file mode 100644 index 3a9835a5b..000000000 --- a/etl/src/test_utils/table.rs +++ /dev/null @@ -1,34 +0,0 @@ -use etl_postgres::types::{ColumnSchema, TableId, TableName, TableSchema}; -use std::collections::HashMap; - -/// Asserts that a table schema matches the expected schema. -/// -/// Compares all aspects of the table schema including table ID, name, and column -/// definitions. Each column's properties (name, type, modifier, nullability, and -/// primary key status) are verified. -/// -/// # Panics -/// -/// Panics if the table ID doesn't exist in the provided schemas, or if any aspect -/// of the schema doesn't match the expected values. -pub fn assert_table_schema( - table_schemas: &HashMap, - table_id: TableId, - expected_table_name: TableName, - expected_columns: &[ColumnSchema], -) { - let table_schema = table_schemas.get(&table_id).unwrap(); - assert_eq!(table_schema.id, table_id); - assert_eq!(table_schema.name, expected_table_name); - - let columns = &table_schema.column_schemas; - assert_eq!(columns.len(), expected_columns.len()); - - for (actual, expected) in columns.iter().zip(expected_columns.iter()) { - assert_eq!(actual.name, expected.name); - assert_eq!(actual.typ, expected.typ); - assert_eq!(actual.modifier, expected.modifier); - assert_eq!(actual.nullable, expected.nullable); - assert_eq!(actual.primary_key(), expected.primary_key()); - } -} diff --git a/etl/src/test_utils/test_schema.rs b/etl/src/test_utils/test_schema.rs index bf1a523a0..a6f501fb4 100644 --- a/etl/src/test_utils/test_schema.rs +++ b/etl/src/test_utils/test_schema.rs @@ -348,7 +348,7 @@ pub fn build_expected_users_inserts( .collect::>(); let replicated_table_schema = ReplicatedTableSchema::from_mask( Arc::new(users_table_schema.clone()), - ReplicationMask::build(users_table_schema, &users_table_column_names).unwrap(), + ReplicationMask::build_or_all(users_table_schema, &users_table_column_names), ); for (name, age) in expected_rows { @@ -386,7 +386,7 @@ pub fn build_expected_orders_inserts( .collect::>(); let replicated_table_schema = ReplicatedTableSchema::from_mask( Arc::new(orders_table_schema.clone()), - ReplicationMask::build(orders_table_schema, &orders_table_column_names).unwrap(), + ReplicationMask::build_or_all(orders_table_schema, &orders_table_column_names), ); for name in expected_rows { diff --git a/etl/src/types/event.rs b/etl/src/types/event.rs index df73d9e94..6559922e0 100644 --- a/etl/src/types/event.rs +++ b/etl/src/types/event.rs @@ -116,6 +116,22 @@ pub struct TruncateEvent { pub truncated_tables: Vec, } +/// Relation (schema) event from Postgres logical replication. +/// +/// [`RelationEvent`] represents a table schema notification in the replication stream. +/// It is emitted when a RELATION message is received, containing the current +/// replication mask for the table. This event notifies downstream consumers +/// about which columns are being replicated for a table. +#[derive(Debug, Clone)] +pub struct RelationEvent { + /// LSN position where the event started. + pub start_lsn: PgLsn, + /// LSN position where the transaction of this event will commit. + pub commit_lsn: PgLsn, + /// The replicated table schema containing the table schema and replication mask. + pub replicated_table_schema: ReplicatedTableSchema, +} + /// Represents a single replication event from Postgres logical replication. /// /// [`Event`] encapsulates all possible events that can occur in a Postgres replication @@ -135,6 +151,8 @@ pub enum Event { Delete(DeleteEvent), /// Table truncation event clearing all rows from tables. Truncate(TruncateEvent), + /// Relation (schema) event notifying about table schema and replication mask. + Relation(RelationEvent), /// Unsupported event type that cannot be processed. Unsupported, } @@ -159,6 +177,7 @@ impl Event { Event::Update(e) => e.replicated_table_schema.id() == *table_id, Event::Delete(e) => e.replicated_table_schema.id() == *table_id, Event::Truncate(e) => e.truncated_tables.iter().any(|s| s.id() == *table_id), + Event::Relation(e) => e.replicated_table_schema.id() == *table_id, _ => false, } } @@ -213,6 +232,7 @@ impl From<&Event> for EventType { Event::Update(_) => EventType::Update, Event::Delete(_) => EventType::Delete, Event::Truncate(_) => EventType::Truncate, + Event::Relation(_) => EventType::Relation, Event::Unsupported => EventType::Unsupported, } } diff --git a/etl/src/workers/apply.rs b/etl/src/workers/apply.rs index 5463031bb..312bc0e48 100644 --- a/etl/src/workers/apply.rs +++ b/etl/src/workers/apply.rs @@ -144,7 +144,7 @@ where // We create the signal used to notify the apply worker that it should force syncing tables. let (force_syncing_tables_tx, force_syncing_tables_rx) = create_signal(); - start_apply_loop( + let result = start_apply_loop( self.pipeline_id, start_lsn, self.config.clone(), @@ -166,11 +166,19 @@ where self.shutdown_rx, Some(force_syncing_tables_rx), ) - .await?; + .await; - info!("apply worker completed successfully"); - - Ok(()) + match result { + Ok(_) => { + info!("apply worker completed successfully"); + Ok(()) + } + Err(err) => { + // We log the error here, this way it's logged even if the worker is not awaited. + error!("apply worker failed: {}", err); + Err(err) + } + } } .instrument(apply_worker_span.or_current()); diff --git a/etl/tests/failpoints_pipeline.rs b/etl/tests/failpoints_pipeline.rs deleted file mode 100644 index 0f597b64d..000000000 --- a/etl/tests/failpoints_pipeline.rs +++ /dev/null @@ -1,279 +0,0 @@ -#![cfg(all(feature = "test-utils", feature = "failpoints"))] - -use etl::destination::memory::MemoryDestination; -use etl::error::ErrorKind; -use etl::failpoints::{ - START_TABLE_SYNC_BEFORE_DATA_SYNC_SLOT_CREATION, START_TABLE_SYNC_DURING_DATA_SYNC, -}; -use etl::state::table::{RetryPolicy, TableReplicationPhase, TableReplicationPhaseType}; -use etl::test_utils::database::spawn_source_database; -use etl::test_utils::notify::NotifyingStore; -use etl::test_utils::pipeline::create_pipeline; -use etl::test_utils::test_destination_wrapper::TestDestinationWrapper; -use etl::test_utils::test_schema::{TableSelection, insert_users_data, setup_test_database_schema}; -use etl::types::PipelineId; -use etl_telemetry::tracing::init_test_tracing; -use fail::FailScenario; -use rand::random; - -#[tokio::test(flavor = "multi_thread")] -async fn table_copy_fails_after_data_sync_threw_an_error_with_no_retry() { - let _scenario = FailScenario::setup(); - fail::cfg( - START_TABLE_SYNC_BEFORE_DATA_SYNC_SLOT_CREATION, - "1*return(no_retry)", - ) - .unwrap(); - - init_test_tracing(); - - let mut database = spawn_source_database().await; - let database_schema = setup_test_database_schema(&database, TableSelection::UsersOnly).await; - - // Insert initial test data. - let rows_inserted = 10; - insert_users_data( - &mut database, - &database_schema.users_schema().name, - 1..=rows_inserted, - ) - .await; - - let store = NotifyingStore::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(), - store.clone(), - destination.clone(), - ); - - // Register notifications for table sync phases. - let users_state_notify = store - .notify_on_table_state_type( - database_schema.users_schema().id, - TableReplicationPhaseType::Errored, - ) - .await; - - pipeline.start().await.unwrap(); - - users_state_notify.notified().await; - - // We expect to have a no retry error which is generated by the failpoint. - let err = pipeline.shutdown_and_wait().await.err().unwrap(); - assert_eq!(err.kinds().len(), 1); - assert_eq!(err.kinds()[0], ErrorKind::WithNoRetry); - - // Verify no data is there. - let table_rows = destination.get_table_rows().await; - assert!(table_rows.is_empty()); - - // Verify table schemas were correctly stored. - let table_schemas = store.get_table_schemas().await; - assert!(table_schemas.is_empty()); -} - -#[tokio::test(flavor = "multi_thread")] -async fn table_copy_fails_after_timed_retry_exceeded_max_attempts() { - let _scenario = FailScenario::setup(); - // Since we have table_error_retry_max_attempts: 2, we want to fail 3 times, so that on the 3rd - // time, the system switches to manual retry. - fail::cfg( - START_TABLE_SYNC_BEFORE_DATA_SYNC_SLOT_CREATION, - "3*return(timed_retry)", - ) - .unwrap(); - - init_test_tracing(); - - let mut database = spawn_source_database().await; - let database_schema = setup_test_database_schema(&database, TableSelection::UsersOnly).await; - - // Insert initial test data. - let rows_inserted = 10; - insert_users_data( - &mut database, - &database_schema.users_schema().name, - 1..=rows_inserted, - ) - .await; - - let store = NotifyingStore::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(), - store.clone(), - destination.clone(), - ); - - // Register notifications for waiting on the manual retry which is expected to be flipped by the - // max attempts handling. - let users_state_notify = store - .notify_on_table_state(database_schema.users_schema().id, |phase| { - matches!( - phase, - TableReplicationPhase::Errored { - retry_policy: RetryPolicy::ManualRetry, - .. - } - ) - }) - .await; - - pipeline.start().await.unwrap(); - - users_state_notify.notified().await; - - // We expect to still have the timed retry kind since this is the kind of error that we triggered. - let err = pipeline.shutdown_and_wait().await.err().unwrap(); - assert_eq!(err.kinds().len(), 1); - assert_eq!(err.kinds()[0], ErrorKind::WithTimedRetry); - - // Verify no data is there. - let table_rows = destination.get_table_rows().await; - assert!(table_rows.is_empty()); - - // Verify table schemas were correctly stored. - let table_schemas = store.get_table_schemas().await; - assert!(table_schemas.is_empty()); -} - -#[tokio::test(flavor = "multi_thread")] -async fn table_copy_is_consistent_after_data_sync_threw_an_error_with_timed_retry() { - let _scenario = FailScenario::setup(); - fail::cfg( - START_TABLE_SYNC_BEFORE_DATA_SYNC_SLOT_CREATION, - "1*return(timed_retry)", - ) - .unwrap(); - - init_test_tracing(); - - let mut database = spawn_source_database().await; - let database_schema = setup_test_database_schema(&database, TableSelection::UsersOnly).await; - - // Insert initial test data. - let rows_inserted = 10; - insert_users_data( - &mut database, - &database_schema.users_schema().name, - 1..=rows_inserted, - ) - .await; - - let store = NotifyingStore::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(), - store.clone(), - destination.clone(), - ); - - // We register the interest in waiting for both table syncs to have started. - let users_state_notify = store - .notify_on_table_state_type( - database_schema.users_schema().id, - TableReplicationPhaseType::SyncDone, - ) - .await; - - pipeline.start().await.unwrap(); - - users_state_notify.notified().await; - - // We expect no errors, since the same table sync worker task is retried. - pipeline.shutdown_and_wait().await.unwrap(); - - // Verify copied data. - let table_rows = destination.get_table_rows().await; - let users_table_rows = table_rows.get(&database_schema.users_schema().id).unwrap(); - assert_eq!(users_table_rows.len(), rows_inserted); - - // Verify table schemas were correctly stored. - let table_schemas = store.get_table_schemas().await; - assert_eq!(table_schemas.len(), 1); - assert_eq!( - *table_schemas - .get(&database_schema.users_schema().id) - .unwrap(), - database_schema.users_schema() - ); -} - -#[tokio::test(flavor = "multi_thread")] -async fn table_copy_is_consistent_during_data_sync_threw_an_error_with_timed_retry() { - let _scenario = FailScenario::setup(); - fail::cfg(START_TABLE_SYNC_DURING_DATA_SYNC, "1*return(timed_retry)").unwrap(); - - init_test_tracing(); - - let mut database = spawn_source_database().await; - let database_schema = setup_test_database_schema(&database, TableSelection::UsersOnly).await; - - // Insert initial test data. - let rows_inserted = 10; - insert_users_data( - &mut database, - &database_schema.users_schema().name, - 1..=rows_inserted, - ) - .await; - - let store = NotifyingStore::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(), - store.clone(), - destination.clone(), - ); - - // We register the interest in waiting for both table syncs to have started. - let users_state_notify = store - .notify_on_table_state_type( - database_schema.users_schema().id, - TableReplicationPhaseType::SyncDone, - ) - .await; - - pipeline.start().await.unwrap(); - - users_state_notify.notified().await; - - // We expect no errors, since the same table sync worker task is retried. - pipeline.shutdown_and_wait().await.unwrap(); - - // Verify copied data. - let table_rows = destination.get_table_rows().await; - let users_table_rows = table_rows.get(&database_schema.users_schema().id).unwrap(); - assert_eq!(users_table_rows.len(), rows_inserted); - - // Verify table schemas were correctly stored. - let table_schemas = store.get_table_schemas().await; - assert_eq!(table_schemas.len(), 1); - assert_eq!( - *table_schemas - .get(&database_schema.users_schema().id) - .unwrap(), - database_schema.users_schema() - ); -} diff --git a/etl/tests/pipeline.rs b/etl/tests/pipeline.rs index 3ebbdce0f..804188e3a 100644 --- a/etl/tests/pipeline.rs +++ b/etl/tests/pipeline.rs @@ -7,7 +7,7 @@ use etl::test_utils::database::{spawn_source_database, test_table_name}; use etl::test_utils::event::group_events_by_type_and_table_id; use etl::test_utils::notify::NotifyingStore; use etl::test_utils::pipeline::{create_pipeline, create_pipeline_with}; -use etl::test_utils::table::assert_table_schema; +use etl::test_utils::schema::assert_table_schema_columns; use etl::test_utils::test_destination_wrapper::TestDestinationWrapper; use etl::test_utils::test_schema::{ TableSelection, assert_events_equal, build_expected_orders_inserts, @@ -203,7 +203,7 @@ async fn table_schema_copy_survives_pipeline_restarts() { ); // We check that the table schemas have been stored. - let table_schemas = store.get_table_schemas().await; + let table_schemas = store.get_latest_table_schemas().await; assert_eq!(table_schemas.len(), 2); assert_eq!( *table_schemas @@ -501,7 +501,7 @@ async fn publication_for_all_tables_in_schema_ignores_new_tables_until_restart() pipeline.shutdown_and_wait().await.unwrap(); // Check that only the schemas of the first table were stored. - let table_schemas = store.get_table_schemas().await; + let table_schemas = store.get_latest_table_schemas().await; assert_eq!(table_schemas.len(), 1); assert!(table_schemas.contains_key(&table_1_id)); assert!(!table_schemas.contains_key(&table_2_id)); @@ -552,7 +552,7 @@ async fn publication_for_all_tables_in_schema_ignores_new_tables_until_restart() pipeline.shutdown_and_wait().await.unwrap(); // Check that both schemas exist. - let table_schemas = store.get_table_schemas().await; + let table_schemas = store.get_latest_table_schemas().await; assert_eq!(table_schemas.len(), 2); assert!(table_schemas.contains_key(&table_1_id)); assert!(table_schemas.contains_key(&table_2_id)); @@ -960,147 +960,6 @@ async fn table_processing_converges_to_apply_loop_with_no_events_coming() { assert_eq!(age_sum, expected_age_sum); } -// TOOD: re-implement once we clarified the semantics for table change. -// #[tokio::test(flavor = "multi_thread")] -// async fn table_processing_with_schema_change_errors_table() { -// init_test_tracing(); -// let database = spawn_source_database().await; -// let database_schema = setup_test_database_schema(&database, TableSelection::OrdersOnly).await; -// -// // Insert data in the table. -// database -// .insert_values( -// database_schema.orders_schema().name.clone(), -// &["description"], -// &[&"description_1"], -// ) -// .await -// .unwrap(); -// -// let store = NotifyingStore::new(); -// let destination = TestDestinationWrapper::wrap(MemoryDestination::new()); -// -// // Start pipeline from scratch. -// let pipeline_id: PipelineId = random(); -// let mut pipeline = create_pipeline( -// &database.config, -// pipeline_id, -// database_schema.publication_name(), -// store.clone(), -// destination.clone(), -// ); -// -// // Register notifications for initial table copy completion. -// let orders_state_notify = store -// .notify_on_table_state_type( -// database_schema.orders_schema().id, -// TableReplicationPhaseType::FinishedCopy, -// ) -// .await; -// -// pipeline.start().await.unwrap(); -// -// orders_state_notify.notified().await; -// -// // Register notification for the sync done state. -// let orders_state_notify = store -// .notify_on_table_state_type( -// database_schema.orders_schema().id, -// TableReplicationPhaseType::SyncDone, -// ) -// .await; -// -// // Insert new data in the table. -// database -// .insert_values( -// database_schema.orders_schema().name.clone(), -// &["description"], -// &[&"description_2"], -// ) -// .await -// .unwrap(); -// -// orders_state_notify.notified().await; -// -// // Register notification for the ready state. -// let orders_state_notify = store -// .notify_on_table_state_type( -// database_schema.orders_schema().id, -// TableReplicationPhaseType::Ready, -// ) -// .await; -// -// // Insert new data in the table. -// database -// .insert_values( -// database_schema.orders_schema().name.clone(), -// &["description"], -// &[&"description_3"], -// ) -// .await -// .unwrap(); -// -// orders_state_notify.notified().await; -// -// // Register notification for the errored state. -// let orders_state_notify = store -// .notify_on_table_state_type( -// database_schema.orders_schema().id, -// TableReplicationPhaseType::Errored, -// ) -// .await; -// -// // Change the schema of orders by adding a new column. -// database -// .alter_table( -// database_schema.orders_schema().name.clone(), -// &[TableModification::AddColumn { -// name: "date", -// data_type: "integer", -// }], -// ) -// .await -// .unwrap(); -// -// // Insert new data in the table. -// database -// .insert_values( -// database_schema.orders_schema().name.clone(), -// &["description", "date"], -// &[&"description_with_date", &10], -// ) -// .await -// .unwrap(); -// -// orders_state_notify.notified().await; -// -// pipeline.shutdown_and_wait().await.unwrap(); -// -// // We assert that the schema is the initial one. -// let table_schemas = store.get_table_schemas().await; -// assert_eq!(table_schemas.len(), 1); -// assert_eq!( -// *table_schemas -// .get(&database_schema.orders_schema().id) -// .unwrap(), -// database_schema.orders_schema() -// ); -// -// // We check that we got the insert events after the first data of the table has been copied. -// let events = destination.get_events().await; -// let grouped_events = group_events_by_type_and_table_id(&events); -// let orders_inserts = grouped_events -// .get(&(EventType::Insert, database_schema.orders_schema().id)) -// .unwrap(); -// -// let expected_orders_inserts = build_expected_orders_inserts( -// 2, -// database_schema.orders_schema().id, -// vec!["description_2", "description_3"], -// ); -// assert_events_equal(orders_inserts, &expected_orders_inserts); -// } - #[tokio::test(flavor = "multi_thread")] async fn table_without_primary_key_is_errored() { init_test_tracing(); @@ -1223,7 +1082,7 @@ async fn pipeline_respects_column_level_publication() { // Wait for an insert event to be processed. let insert_events_notify = destination - .wait_for_events_count(vec![(EventType::Insert, 1)]) + .wait_for_events_count(vec![(EventType::Relation, 1), (EventType::Insert, 1)]) .await; // Insert test data with all columns (including email and phone). @@ -1243,6 +1102,39 @@ async fn pipeline_respects_column_level_publication() { let insert_events = grouped_events.get(&(EventType::Insert, table_id)).unwrap(); assert_eq!(insert_events.len(), 1); + let initial_relation_event = events + .iter() + .rev() + .find_map(|event| match event { + Event::Relation(relation) if relation.replicated_table_schema.id() == table_id => { + Some(relation.clone()) + } + _ => None, + }) + .expect("Expected relation event for initial publication state"); + + let initial_relation_columns: Vec<&str> = initial_relation_event + .replicated_table_schema + .column_schemas() + .map(|c| c.name.as_str()) + .collect(); + assert_eq!(initial_relation_columns, vec!["id", "name", "age"]); + assert_eq!( + initial_relation_event + .replicated_table_schema + .replication_mask() + .as_slice(), + &[1, 1, 1, 0, 0] + ); + assert_eq!( + initial_relation_event + .replicated_table_schema + .get_inner() + .column_schemas + .len(), + 5 + ); + // Check that each insert event contains only the published columns (id, name, age) and that the // schema used is correct. for event in insert_events { @@ -1282,7 +1174,7 @@ async fn pipeline_respects_column_level_publication() { // Wait for 1 insert event with 4 columns. let insert_notify = destination - .wait_for_events_count(vec![(EventType::Insert, 1)]) + .wait_for_events_count(vec![(EventType::Relation, 1), (EventType::Insert, 1)]) .await; database @@ -1301,6 +1193,17 @@ async fn pipeline_respects_column_level_publication() { let inserts = grouped.get(&(EventType::Insert, table_id)).unwrap(); assert_eq!(inserts.len(), 1); + let relation_after_adding_email = events + .iter() + .rev() + .find_map(|event| match event { + Event::Relation(relation) if relation.replicated_table_schema.id() == table_id => { + Some(relation.clone()) + } + _ => None, + }) + .expect("Expected relation event after adding email to publication"); + if let Event::Insert(InsertEvent { replicated_table_schema, table_row, @@ -1317,6 +1220,28 @@ async fn pipeline_respects_column_level_publication() { panic!("Expected Insert event"); } + let relation_columns: Vec<&str> = relation_after_adding_email + .replicated_table_schema + .column_schemas() + .map(|c| c.name.as_str()) + .collect(); + assert_eq!(relation_columns, vec!["id", "name", "age", "email"]); + assert_eq!( + relation_after_adding_email + .replicated_table_schema + .replication_mask() + .as_slice(), + &[1, 1, 1, 1, 0] + ); + assert_eq!( + relation_after_adding_email + .replicated_table_schema + .get_inner() + .column_schemas + .len(), + 5 + ); + // Remove age column from publication -> (id, name, email). database .run_sql(&format!( @@ -1331,7 +1256,7 @@ async fn pipeline_respects_column_level_publication() { // Wait for 1 insert event with 3 columns (different set than before). let insert_notify = destination - .wait_for_events_count(vec![(EventType::Insert, 1)]) + .wait_for_events_count(vec![(EventType::Relation, 1), (EventType::Insert, 1)]) .await; database @@ -1349,6 +1274,16 @@ async fn pipeline_respects_column_level_publication() { // Verify 3 columns arrived (id, name, email) - age and phone excluded. let events = destination.get_events().await; + let relation_after_removing_age = events + .iter() + .rev() + .find_map(|event| match event { + Event::Relation(relation) if relation.replicated_table_schema.id() == table_id => { + Some(relation.clone()) + } + _ => None, + }) + .expect("Expected relation event after removing age from publication"); let grouped = group_events_by_type_and_table_id(&events); let inserts = grouped.get(&(EventType::Insert, table_id)).unwrap(); assert_eq!(inserts.len(), 1); @@ -1368,6 +1303,28 @@ async fn pipeline_respects_column_level_publication() { } else { panic!("Expected Insert event"); } + + let relation_columns: Vec<&str> = relation_after_removing_age + .replicated_table_schema + .column_schemas() + .map(|c| c.name.as_str()) + .collect(); + assert_eq!(relation_columns, vec!["id", "name", "email"]); + assert_eq!( + relation_after_removing_age + .replicated_table_schema + .replication_mask() + .as_slice(), + &[1, 1, 0, 1, 0] + ); + assert_eq!( + relation_after_removing_age + .replicated_table_schema + .get_inner() + .column_schemas + .len(), + 5 + ); } #[tokio::test(flavor = "multi_thread")] @@ -1422,11 +1379,12 @@ async fn empty_tables_are_created_at_destination() { pipeline.shutdown_and_wait().await.unwrap(); // Verify the table schema was stored. - let table_schemas = state_store.get_table_schemas().await; - assert_table_schema( - &table_schemas, - table_id, - table_name, + let table_schemas = state_store.get_latest_table_schemas().await; + let table_schema = table_schemas.get(&table_id).unwrap(); + assert_eq!(table_schema.id, table_id); + assert_eq!(table_schema.name, table_name); + assert_table_schema_columns( + table_schema, &[ id_column_schema(), test_column("name", Type::TEXT, 2, true, false), diff --git a/etl/tests/pipeline_with_failpoints.rs b/etl/tests/pipeline_with_failpoints.rs new file mode 100644 index 000000000..8ced24cec --- /dev/null +++ b/etl/tests/pipeline_with_failpoints.rs @@ -0,0 +1,858 @@ +#![cfg(all(feature = "test-utils", feature = "failpoints"))] + +use etl::destination::memory::MemoryDestination; +use etl::error::ErrorKind; +use etl::failpoints::{ + SEND_STATUS_UPDATE_FP, START_TABLE_SYNC_BEFORE_DATA_SYNC_SLOT_CREATION_FP, + START_TABLE_SYNC_DURING_DATA_SYNC_FP, +}; +use etl::state::table::{RetryPolicy, TableReplicationPhase, TableReplicationPhaseType}; +use etl::test_utils::database::{spawn_source_database, test_table_name}; +use etl::test_utils::event::group_events_by_type_and_table_id; +use etl::test_utils::notify::NotifyingStore; +use etl::test_utils::pipeline::{create_database_and_pipeline_with_table, create_pipeline}; +use etl::test_utils::schema::{ + assert_schema_snapshots_ordering, assert_table_schema_column_names_types, +}; +use etl::test_utils::test_destination_wrapper::TestDestinationWrapper; +use etl::test_utils::test_schema::{TableSelection, insert_users_data, setup_test_database_schema}; +use etl::types::Type; +use etl::types::{Event, EventType, InsertEvent, PipelineId, TableId}; +use etl_postgres::below_version; +use etl_postgres::tokio::test_utils::TableModification; +use etl_postgres::version::POSTGRES_15; +use etl_telemetry::tracing::init_test_tracing; +use fail::FailScenario; +use rand::random; + +#[tokio::test(flavor = "multi_thread")] +async fn table_copy_fails_after_data_sync_threw_an_error_with_no_retry() { + let _scenario = FailScenario::setup(); + fail::cfg( + START_TABLE_SYNC_BEFORE_DATA_SYNC_SLOT_CREATION_FP, + "1*return(no_retry)", + ) + .unwrap(); + + init_test_tracing(); + + let mut database = spawn_source_database().await; + let database_schema = setup_test_database_schema(&database, TableSelection::UsersOnly).await; + + // Insert initial test data. + let rows_inserted = 10; + insert_users_data( + &mut database, + &database_schema.users_schema().name, + 1..=rows_inserted, + ) + .await; + + let store = NotifyingStore::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(), + store.clone(), + destination.clone(), + ); + + // Register notifications for table sync phases. + let users_state_notify = store + .notify_on_table_state_type( + database_schema.users_schema().id, + TableReplicationPhaseType::Errored, + ) + .await; + + pipeline.start().await.unwrap(); + + users_state_notify.notified().await; + + // We expect to have a no retry error which is generated by the failpoint. + let err = pipeline.shutdown_and_wait().await.err().unwrap(); + assert_eq!(err.kinds().len(), 1); + assert_eq!(err.kinds()[0], ErrorKind::WithNoRetry); + + // Verify no data is there. + let table_rows = destination.get_table_rows().await; + assert!(table_rows.is_empty()); + + // Verify table schemas were correctly stored. + let table_schemas = store.get_latest_table_schemas().await; + assert!(table_schemas.is_empty()); +} + +#[tokio::test(flavor = "multi_thread")] +async fn table_copy_fails_after_timed_retry_exceeded_max_attempts() { + let _scenario = FailScenario::setup(); + // Since we have table_error_retry_max_attempts: 2, we want to fail 3 times, so that on the 3rd + // time, the system switches to manual retry. + fail::cfg( + START_TABLE_SYNC_BEFORE_DATA_SYNC_SLOT_CREATION_FP, + "3*return(timed_retry)", + ) + .unwrap(); + + init_test_tracing(); + + let mut database = spawn_source_database().await; + let database_schema = setup_test_database_schema(&database, TableSelection::UsersOnly).await; + + // Insert initial test data. + let rows_inserted = 10; + insert_users_data( + &mut database, + &database_schema.users_schema().name, + 1..=rows_inserted, + ) + .await; + + let store = NotifyingStore::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(), + store.clone(), + destination.clone(), + ); + + // Register notifications for waiting on the manual retry which is expected to be flipped by the + // max attempts handling. + let users_state_notify = store + .notify_on_table_state(database_schema.users_schema().id, |phase| { + matches!( + phase, + TableReplicationPhase::Errored { + retry_policy: RetryPolicy::ManualRetry, + .. + } + ) + }) + .await; + + pipeline.start().await.unwrap(); + + users_state_notify.notified().await; + + // We expect to still have the timed retry kind since this is the kind of error that we triggered. + let err = pipeline.shutdown_and_wait().await.err().unwrap(); + assert_eq!(err.kinds().len(), 1); + assert_eq!(err.kinds()[0], ErrorKind::WithTimedRetry); + + // Verify no data is there. + let table_rows = destination.get_table_rows().await; + assert!(table_rows.is_empty()); + + // Verify table schemas were correctly stored. + let table_schemas = store.get_latest_table_schemas().await; + assert!(table_schemas.is_empty()); +} + +#[tokio::test(flavor = "multi_thread")] +async fn table_copy_is_consistent_after_data_sync_threw_an_error_with_timed_retry() { + let _scenario = FailScenario::setup(); + fail::cfg( + START_TABLE_SYNC_BEFORE_DATA_SYNC_SLOT_CREATION_FP, + "1*return(timed_retry)", + ) + .unwrap(); + + init_test_tracing(); + + let mut database = spawn_source_database().await; + let database_schema = setup_test_database_schema(&database, TableSelection::UsersOnly).await; + + // Insert initial test data. + let rows_inserted = 10; + insert_users_data( + &mut database, + &database_schema.users_schema().name, + 1..=rows_inserted, + ) + .await; + + let store = NotifyingStore::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(), + store.clone(), + destination.clone(), + ); + + // We register the interest in waiting for both table syncs to have started. + let users_state_notify = store + .notify_on_table_state_type( + database_schema.users_schema().id, + TableReplicationPhaseType::SyncDone, + ) + .await; + + pipeline.start().await.unwrap(); + + users_state_notify.notified().await; + + // We expect no errors, since the same table sync worker task is retried. + pipeline.shutdown_and_wait().await.unwrap(); + + // Verify copied data. + let table_rows = destination.get_table_rows().await; + let users_table_rows = table_rows.get(&database_schema.users_schema().id).unwrap(); + assert_eq!(users_table_rows.len(), rows_inserted); + + // Verify table schemas were correctly stored. + let table_schemas = store.get_latest_table_schemas().await; + assert_eq!(table_schemas.len(), 1); + assert_eq!( + *table_schemas + .get(&database_schema.users_schema().id) + .unwrap(), + database_schema.users_schema() + ); +} + +#[tokio::test(flavor = "multi_thread")] +async fn table_copy_is_consistent_during_data_sync_threw_an_error_with_timed_retry() { + let _scenario = FailScenario::setup(); + fail::cfg( + START_TABLE_SYNC_DURING_DATA_SYNC_FP, + "1*return(timed_retry)", + ) + .unwrap(); + + init_test_tracing(); + + let mut database = spawn_source_database().await; + let database_schema = setup_test_database_schema(&database, TableSelection::UsersOnly).await; + + // Insert initial test data. + let rows_inserted = 10; + insert_users_data( + &mut database, + &database_schema.users_schema().name, + 1..=rows_inserted, + ) + .await; + + let store = NotifyingStore::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(), + store.clone(), + destination.clone(), + ); + + // We register the interest in waiting for both table syncs to have started. + let users_state_notify = store + .notify_on_table_state_type( + database_schema.users_schema().id, + TableReplicationPhaseType::SyncDone, + ) + .await; + + pipeline.start().await.unwrap(); + + users_state_notify.notified().await; + + // We expect no errors, since the same table sync worker task is retried. + pipeline.shutdown_and_wait().await.unwrap(); + + // Verify copied data. + let table_rows = destination.get_table_rows().await; + let users_table_rows = table_rows.get(&database_schema.users_schema().id).unwrap(); + assert_eq!(users_table_rows.len(), rows_inserted); + + // Verify table schemas were correctly stored. + let table_schemas = store.get_latest_table_schemas().await; + assert_eq!(table_schemas.len(), 1); + assert_eq!( + *table_schemas + .get(&database_schema.users_schema().id) + .unwrap(), + database_schema.users_schema() + ); +} + +#[ignore] +#[tokio::test(flavor = "multi_thread")] +async fn table_schema_snapshots_are_consistent_after_missing_status_update_with_interleaved_ddl() { + let _scenario = FailScenario::setup(); + fail::cfg(SEND_STATUS_UPDATE_FP, "return").unwrap(); + + init_test_tracing(); + + let (database, table_name, table_id, store, destination, pipeline, pipeline_id, publication) = + create_database_and_pipeline_with_table( + "schema_add_column", + &[("name", "text not null"), ("age", "integer not null")], + ) + .await; + + let notify = destination + .wait_for_events_count(vec![(EventType::Relation, 2), (EventType::Insert, 2)]) + .await; + + database + .insert_values(table_name.clone(), &["name", "age"], &[&"Alice", &25]) + .await + .unwrap(); + + database + .alter_table( + table_name.clone(), + &[TableModification::AddColumn { + name: "email", + data_type: "text null", + }], + ) + .await + .unwrap(); + + database + .insert_values( + table_name.clone(), + &["name", "age", "email"], + &[&"Bob", &28, &"bob@example.com"], + ) + .await + .unwrap(); + + notify.notified().await; + pipeline.shutdown_and_wait().await.unwrap(); + + // Assert that we got all the events correctly. + let events = destination.get_events().await; + let grouped = group_events_by_type_and_table_id(&events); + + assert_eq!( + grouped.get(&(EventType::Relation, table_id)).unwrap().len(), + 2 + ); + assert_eq!( + grouped.get(&(EventType::Insert, table_id)).unwrap().len(), + 2 + ); + + // Assert that we have 2 schema snapshots stored in order. + let table_schemas = store.get_table_schemas().await; + let table_schemas_snapshots = table_schemas.get(&table_id).unwrap(); + assert_eq!(table_schemas_snapshots.len(), 2); + assert_schema_snapshots_ordering(table_schemas_snapshots, true); + + // Verify the first snapshot has the original schema (id, name, age). + let (_, first_schema) = &table_schemas_snapshots[0]; + assert_table_schema_column_names_types( + first_schema, + &[ + ("id", Type::INT8), + ("name", Type::TEXT), + ("age", Type::INT4), + ], + ); + + // Verify the second snapshot has the new column added (id, name, age, email). + let (_, second_schema) = &table_schemas_snapshots[1]; + assert_table_schema_column_names_types( + second_schema, + &[ + ("id", Type::INT8), + ("name", Type::TEXT), + ("age", Type::INT4), + ("email", Type::TEXT), + ], + ); + + // Clear up the events. + destination.clear_events().await; + + // Restart the pipeline with the failpoint disabled to verify recovery. + let mut pipeline = create_pipeline( + &database.config, + pipeline_id, + publication, + store.clone(), + destination.clone(), + ); + + pipeline.start().await.unwrap(); + + database + .insert_values( + table_name.clone(), + &["name", "age", "email"], + &[&"Charlie", &35, &"charlie@example.com"], + ) + .await + .unwrap(); + + // TODO: figure out how to wait for errors in the apply worker and remove ignore. + + // We expect to have a corrupted table schema error since when we reprocess the events, Postgres + // sends a `Relation` message with the `email` column even for entries before the DDL that added + // the `email`. For now this is a limitation that we are acknowledging, but we would like to find + // a solution for this. + let err = pipeline.shutdown_and_wait().await.err().unwrap(); + assert_eq!(err.kinds().len(), 1); + assert_eq!(err.kinds()[0], ErrorKind::CorruptedTableSchema); +} + +#[tokio::test(flavor = "multi_thread")] +async fn table_schema_snapshots_are_consistent_after_missing_status_update_with_initial_ddl() { + let _scenario = FailScenario::setup(); + fail::cfg(SEND_STATUS_UPDATE_FP, "return").unwrap(); + + init_test_tracing(); + + let (database, table_name, table_id, store, destination, pipeline, pipeline_id, publication) = + create_database_and_pipeline_with_table( + "schema_add_column", + &[("name", "text not null"), ("age", "integer not null")], + ) + .await; + + // The reason for why we wait for two `Relation` messages is that since we have a DDL event before + // DML statements, Postgres likely avoids sending an initial `Relation` message since it's already + // sent given the DDL event. + let notify = destination + .wait_for_events_count(vec![(EventType::Relation, 2), (EventType::Insert, 2)]) + .await; + + // We immediately add a column to the table without any DML, to show the case where we can recover + // in case we immediately start with a DDL event. + database + .alter_table( + table_name.clone(), + &[TableModification::AddColumn { + name: "email", + data_type: "text null", + }], + ) + .await + .unwrap(); + + database + .insert_values( + table_name.clone(), + &["name", "age", "email"], + &[&"Bob", &28, &"bob@example.com"], + ) + .await + .unwrap(); + + database + .alter_table( + table_name.clone(), + &[TableModification::DropColumn { name: "age" }], + ) + .await + .unwrap(); + + database + .insert_values( + table_name.clone(), + &["name", "email"], + &[&"Matt", &"matt@example.com"], + ) + .await + .unwrap(); + + notify.notified().await; + pipeline.shutdown_and_wait().await.unwrap(); + + // Assert that we got all the events correctly. + let events = destination.get_events().await; + let grouped = group_events_by_type_and_table_id(&events); + + assert_eq!( + grouped.get(&(EventType::Relation, table_id)).unwrap().len(), + 2 + ); + assert_eq!( + grouped.get(&(EventType::Insert, table_id)).unwrap().len(), + 2 + ); + + // Assert that we have 3 schema snapshots stored in order (1 base snapshot + 2 relation changes). + let table_schemas = store.get_table_schemas().await; + let table_schemas_snapshots = table_schemas.get(&table_id).unwrap(); + assert_eq!(table_schemas_snapshots.len(), 3); + assert_schema_snapshots_ordering(table_schemas_snapshots, true); + + // Verify the first snapshot has the initial schema (id, name, age). + let (_, first_schema) = &table_schemas_snapshots[0]; + assert_table_schema_column_names_types( + first_schema, + &[ + ("id", Type::INT8), + ("name", Type::TEXT), + ("age", Type::INT4), + ], + ); + + // Verify the first snapshot has the new schema (id, name, age, email). + let (_, first_schema) = &table_schemas_snapshots[1]; + assert_table_schema_column_names_types( + first_schema, + &[ + ("id", Type::INT8), + ("name", Type::TEXT), + ("age", Type::INT4), + ("email", Type::TEXT), + ], + ); + + // Verify the second snapshot doesn't have the age column (id, name, email). + let (_, second_schema) = &table_schemas_snapshots[2]; + assert_table_schema_column_names_types( + second_schema, + &[ + ("id", Type::INT8), + ("name", Type::TEXT), + ("email", Type::TEXT), + ], + ); + + // Clear up the events. + destination.clear_events().await; + + // Restart the pipeline with the failpoint disabled to verify recovery. + let mut pipeline = create_pipeline( + &database.config, + pipeline_id, + publication, + store.clone(), + destination.clone(), + ); + + pipeline.start().await.unwrap(); + + let notify = destination + .wait_for_events_count(vec![(EventType::Relation, 2), (EventType::Insert, 3)]) + .await; + + database + .insert_values( + table_name.clone(), + &["name", "email"], + &[&"Charlie", &"charlie@example.com"], + ) + .await + .unwrap(); + + notify.notified().await; + pipeline.shutdown_and_wait().await.unwrap(); + + // Assert that we got all the events correctly. + let events = destination.get_events().await; + let grouped = group_events_by_type_and_table_id(&events); + + assert_eq!( + grouped.get(&(EventType::Relation, table_id)).unwrap().len(), + 2 + ); + assert_eq!( + grouped.get(&(EventType::Insert, table_id)).unwrap().len(), + 3 + ); +} + +#[tokio::test(flavor = "multi_thread")] +async fn table_schema_replication_masks_are_consistent_after_restart() { + let _scenario = FailScenario::setup(); + fail::cfg(SEND_STATUS_UPDATE_FP, "return").unwrap(); + + init_test_tracing(); + let database = spawn_source_database().await; + + // Column filters in publication are only available from Postgres 15+. + if below_version!(database.server_version(), POSTGRES_15) { + eprintln!("Skipping test: PostgreSQL 15+ required for column filters"); + return; + } + + // Create a table with 3 columns (plus auto-generated id). + let table_name = test_table_name("col_removal"); + let table_id = database + .create_table( + table_name.clone(), + true, + &[ + ("name", "text not null"), + ("age", "integer not null"), + ("email", "text not null"), + ], + ) + .await + .unwrap(); + + // Create publication with all 3 columns (plus id) initially. + let publication_name = format!("pub_{}", random::()); + database + .run_sql(&format!( + "create publication {publication_name} for table {} (id, name, age, email)", + table_name.as_quoted_identifier() + )) + .await + .expect("Failed to create publication with column filter"); + + let store = NotifyingStore::new(); + let destination = TestDestinationWrapper::wrap(MemoryDestination::new()); + + let pipeline_id: PipelineId = random(); + let mut pipeline = create_pipeline( + &database.config, + pipeline_id, + publication_name.clone(), + store.clone(), + destination.clone(), + ); + + // Wait for the table to finish syncing. + let sync_done_notify = store + .notify_on_table_state_type(table_id, TableReplicationPhaseType::SyncDone) + .await; + + pipeline.start().await.unwrap(); + + sync_done_notify.notified().await; + + // We expect 3 relation events (one per publication change) and 3 insert events. + let events_notify = destination + .wait_for_events_count(vec![(EventType::Relation, 3), (EventType::Insert, 3)]) + .await; + + // Phase 1: Insert with all 4 columns (id, name, age, email). + database + .run_sql(&format!( + "insert into {} (name, age, email) values ('Alice', 25, 'alice@example.com')", + table_name.as_quoted_identifier() + )) + .await + .unwrap(); + + // Phase 2: Remove email column -> (id, name, age), then insert. + database + .run_sql(&format!( + "alter publication {publication_name} set table {} (id, name, age)", + table_name.as_quoted_identifier() + )) + .await + .unwrap(); + + database + .run_sql(&format!( + "insert into {} (name, age, email) values ('Bob', 30, 'bob@example.com')", + table_name.as_quoted_identifier() + )) + .await + .unwrap(); + + // Phase 3: Remove age column -> (id, name), then insert. + database + .run_sql(&format!( + "alter publication {publication_name} set table {} (id, name)", + table_name.as_quoted_identifier() + )) + .await + .unwrap(); + + database + .run_sql(&format!( + "insert into {} (name, age, email) values ('Charlie', 35, 'charlie@example.com')", + table_name.as_quoted_identifier() + )) + .await + .unwrap(); + + events_notify.notified().await; + + // Helper to verify events after each run. + let verify_events = |events: &[Event], table_id: TableId| { + let grouped = group_events_by_type_and_table_id(events); + + // Verify we have 3 relation events. + let relation_events: Vec<_> = events + .iter() + .filter_map(|event| match event { + Event::Relation(relation) if relation.replicated_table_schema.id() == table_id => { + Some(relation.clone()) + } + _ => None, + }) + .collect(); + assert_eq!( + relation_events.len(), + 3, + "Expected 3 relation events, got {}", + relation_events.len() + ); + + // Verify relation events have decreasing column counts: 4 -> 3 -> 2. + let relation_column_counts: Vec = relation_events + .iter() + .map(|r| r.replicated_table_schema.column_schemas().count()) + .collect(); + assert_eq!( + relation_column_counts, + vec![4, 3, 2], + "Expected relation column counts [4, 3, 2], got {relation_column_counts:?}" + ); + + // Verify relation column names for each phase. + let relation_1_cols: Vec<&str> = relation_events[0] + .replicated_table_schema + .column_schemas() + .map(|c| c.name.as_str()) + .collect(); + assert_eq!(relation_1_cols, vec!["id", "name", "age", "email"]); + + let relation_2_cols: Vec<&str> = relation_events[1] + .replicated_table_schema + .column_schemas() + .map(|c| c.name.as_str()) + .collect(); + assert_eq!(relation_2_cols, vec!["id", "name", "age"]); + + let relation_3_cols: Vec<&str> = relation_events[2] + .replicated_table_schema + .column_schemas() + .map(|c| c.name.as_str()) + .collect(); + assert_eq!(relation_3_cols, vec!["id", "name"]); + + // Verify replication masks. + assert_eq!( + relation_events[0] + .replicated_table_schema + .replication_mask() + .as_slice(), + &[1, 1, 1, 1] + ); + assert_eq!( + relation_events[1] + .replicated_table_schema + .replication_mask() + .as_slice(), + &[1, 1, 1, 0] + ); + assert_eq!( + relation_events[2] + .replicated_table_schema + .replication_mask() + .as_slice(), + &[1, 1, 0, 0] + ); + + // Verify underlying schema always has 4 columns. + for relation in &relation_events { + assert_eq!( + relation + .replicated_table_schema + .get_inner() + .column_schemas + .len(), + 4 + ); + } + + // Verify we have 3 insert events. + let insert_events = grouped.get(&(EventType::Insert, table_id)).unwrap(); + assert_eq!( + insert_events.len(), + 3, + "Expected 3 insert events, got {}", + insert_events.len() + ); + + // Verify insert events have decreasing value counts: 4 -> 3 -> 2. + let insert_value_counts: Vec = insert_events + .iter() + .filter_map(|event| { + if let Event::Insert(InsertEvent { table_row, .. }) = event { + Some(table_row.values.len()) + } else { + None + } + }) + .collect(); + assert_eq!( + insert_value_counts, + vec![4, 3, 2], + "Expected insert value counts [4, 3, 2], got {insert_value_counts:?}" + ); + }; + + // Shutdown the pipeline. + pipeline.shutdown_and_wait().await.unwrap(); + + // Verify events from first run. + let events = destination.get_events().await; + verify_events(&events, table_id); + + // Verify schema snapshots are stored correctly. + let table_schemas = store.get_table_schemas().await; + let table_schemas_snapshots = table_schemas.get(&table_id).unwrap(); + assert!( + !table_schemas_snapshots.is_empty(), + "Expected at least 1 schema snapshot" + ); + assert_schema_snapshots_ordering(table_schemas_snapshots, true); + + // The underlying table schema should always have 4 columns. + for (_, schema) in table_schemas_snapshots { + assert_table_schema_column_names_types( + schema, + &[ + ("id", Type::INT8), + ("name", Type::TEXT), + ("age", Type::INT4), + ("email", Type::TEXT), + ], + ); + } + + // Clear up the events. + destination.clear_events().await; + + // Restart the pipeline - Postgres will resend the data since we don't track progress exactly. + let mut pipeline = create_pipeline( + &database.config, + pipeline_id, + publication_name.clone(), + store.clone(), + destination.clone(), + ); + + // Wait for 3 relation events and 3 insert events again after restart. + let events_notify_restart = destination + .wait_for_events_count(vec![(EventType::Relation, 3), (EventType::Insert, 3)]) + .await; + + pipeline.start().await.unwrap(); + + events_notify_restart.notified().await; + + // Verify the same events are received after restart. + let events_after_restart = destination.get_events().await; + verify_events(&events_after_restart, table_id); + + pipeline.shutdown_and_wait().await.unwrap(); +} diff --git a/etl/tests/pipeline_with_partitioned_table.rs b/etl/tests/pipeline_with_partitioned_table.rs index ab40498d9..d24d049d6 100644 --- a/etl/tests/pipeline_with_partitioned_table.rs +++ b/etl/tests/pipeline_with_partitioned_table.rs @@ -75,7 +75,7 @@ async fn partitioned_table_copy_replicates_existing_data() { let _ = pipeline.shutdown_and_wait().await; // Verify table schema was discovered correctly. - let table_schemas = state_store.get_table_schemas().await; + let table_schemas = state_store.get_latest_table_schemas().await; assert!(table_schemas.contains_key(&parent_table_id)); let parent_schema = &table_schemas[&parent_table_id]; @@ -1246,7 +1246,7 @@ async fn nested_partitioned_table_copy_and_cdc() { parent_sync_done.notified().await; // Verify table schema was discovered correctly for nested partitioned table. - let table_schemas = state_store.get_table_schemas().await; + let table_schemas = state_store.get_latest_table_schemas().await; assert!(table_schemas.contains_key(&parent_table_id)); let parent_schema = &table_schemas[&parent_table_id]; diff --git a/etl/tests/pipelines_with_schema_changes.rs b/etl/tests/pipelines_with_schema_changes.rs new file mode 100644 index 000000000..e67170dcf --- /dev/null +++ b/etl/tests/pipelines_with_schema_changes.rs @@ -0,0 +1,840 @@ +#![cfg(feature = "test-utils")] + +use std::time::Duration; + +use etl::destination::memory::MemoryDestination; +use etl::state::table::TableReplicationPhaseType; +use etl::test_utils::database::{spawn_source_database, test_table_name}; +use etl::test_utils::event::group_events_by_type_and_table_id; +use etl::test_utils::notify::NotifyingStore; +use etl::test_utils::pipeline::{create_database_and_pipeline_with_table, create_pipeline}; +use etl::test_utils::schema::{ + assert_replicated_schema_column_names_types, assert_schema_snapshots_ordering, + assert_table_schema_column_names_types, +}; +use etl::test_utils::test_destination_wrapper::TestDestinationWrapper; +use etl::test_utils::test_schema::create_partitioned_table; +use etl::types::{Event, EventType, PipelineId, Type}; +use etl_postgres::tokio::test_utils::TableModification; +use etl_postgres::types::TableId; +use etl_telemetry::tracing::init_test_tracing; +use rand::random; +use tokio::time::sleep; + +fn get_last_relation_event(events: &[Event], table_id: TableId) -> &Event { + events + .iter() + .rev() + .find(|e| matches!(e, Event::Relation(r) if r.replicated_table_schema.id() == table_id)) + .expect("no relation events for table") +} + +fn get_last_insert_event(events: &[Event], table_id: TableId) -> &Event { + events + .iter() + .rev() + .find(|e| matches!(e, Event::Insert(i) if i.replicated_table_schema.id() == table_id)) + .expect("no insert events for table") +} + +#[tokio::test(flavor = "multi_thread")] +async fn relation_message_updates_when_column_added() { + init_test_tracing(); + + let (database, table_name, table_id, store, destination, pipeline, _pipeline_id, _publication) = + create_database_and_pipeline_with_table( + "schema_add_column", + &[("name", "text not null"), ("age", "integer not null")], + ) + .await; + + let notify = destination + .wait_for_events_count(vec![(EventType::Relation, 1), (EventType::Insert, 1)]) + .await; + + database + .alter_table( + table_name.clone(), + &[TableModification::AddColumn { + name: "email", + data_type: "text not null", + }], + ) + .await + .unwrap(); + + database + .insert_values( + table_name.clone(), + &["name", "age", "email"], + &[&"Alice", &25, &"alice@example.com"], + ) + .await + .unwrap(); + + notify.notified().await; + pipeline.shutdown_and_wait().await.unwrap(); + + let events = destination.get_events().await; + let grouped = group_events_by_type_and_table_id(&events); + + assert_eq!( + grouped.get(&(EventType::Relation, table_id)).unwrap().len(), + 1 + ); + assert_eq!( + grouped.get(&(EventType::Insert, table_id)).unwrap().len(), + 1 + ); + + let Event::Relation(r) = get_last_relation_event(&events, table_id) else { + panic!("expected relation event"); + }; + assert_replicated_schema_column_names_types( + &r.replicated_table_schema, + &[ + ("id", Type::INT8), + ("name", Type::TEXT), + ("age", Type::INT4), + ("email", Type::TEXT), + ], + ); + let Event::Insert(i) = get_last_insert_event(&events, table_id) else { + panic!("expected insert event"); + }; + assert_eq!(i.table_row.values.len(), 4); + + // Verify schema snapshots are stored in order. + let table_schemas = store.get_table_schemas().await; + let snapshots = table_schemas.get(&table_id).unwrap(); + assert_eq!(snapshots.len(), 2); + assert_schema_snapshots_ordering(snapshots, true); + + let (_, first_schema) = &snapshots[0]; + assert_table_schema_column_names_types( + first_schema, + &[ + ("id", Type::INT8), + ("name", Type::TEXT), + ("age", Type::INT4), + ], + ); + + let (_, second_schema) = &snapshots[1]; + assert_table_schema_column_names_types( + second_schema, + &[ + ("id", Type::INT8), + ("name", Type::TEXT), + ("age", Type::INT4), + ("email", Type::TEXT), + ], + ); +} + +#[tokio::test(flavor = "multi_thread")] +async fn relation_message_updates_when_column_removed() { + init_test_tracing(); + + let (database, table_name, table_id, store, destination, pipeline, _pipeline_id, _publication) = + create_database_and_pipeline_with_table( + "schema_remove_column", + &[("name", "text not null"), ("age", "integer not null")], + ) + .await; + + let notify = destination + .wait_for_events_count(vec![(EventType::Relation, 1), (EventType::Insert, 1)]) + .await; + + database + .alter_table( + table_name.clone(), + &[TableModification::DropColumn { name: "age" }], + ) + .await + .unwrap(); + + database + .insert_values(table_name.clone(), &["name"], &[&"Bob"]) + .await + .unwrap(); + + notify.notified().await; + pipeline.shutdown_and_wait().await.unwrap(); + + let events = destination.get_events().await; + let grouped = group_events_by_type_and_table_id(&events); + + assert_eq!( + grouped.get(&(EventType::Relation, table_id)).unwrap().len(), + 1 + ); + assert_eq!( + grouped.get(&(EventType::Insert, table_id)).unwrap().len(), + 1 + ); + + let Event::Relation(r) = get_last_relation_event(&events, table_id) else { + panic!("expected relation event"); + }; + assert_replicated_schema_column_names_types( + &r.replicated_table_schema, + &[("id", Type::INT8), ("name", Type::TEXT)], + ); + let Event::Insert(i) = get_last_insert_event(&events, table_id) else { + panic!("expected insert event"); + }; + assert_eq!(i.table_row.values.len(), 2); + + // Verify schema snapshots are stored in order. + let table_schemas = store.get_table_schemas().await; + let snapshots = table_schemas.get(&table_id).unwrap(); + assert_eq!(snapshots.len(), 2); + assert_schema_snapshots_ordering(snapshots, true); + + let (_, first_schema) = &snapshots[0]; + assert_table_schema_column_names_types( + first_schema, + &[ + ("id", Type::INT8), + ("name", Type::TEXT), + ("age", Type::INT4), + ], + ); + + let (_, second_schema) = &snapshots[1]; + assert_table_schema_column_names_types( + second_schema, + &[("id", Type::INT8), ("name", Type::TEXT)], + ); +} + +#[tokio::test(flavor = "multi_thread")] +async fn relation_message_updates_when_column_renamed() { + init_test_tracing(); + + let (database, table_name, table_id, store, destination, pipeline, _pipeline_id, _publication) = + create_database_and_pipeline_with_table( + "schema_rename_column", + &[("name", "text not null"), ("age", "integer not null")], + ) + .await; + + let notify = destination + .wait_for_events_count(vec![(EventType::Relation, 1), (EventType::Insert, 1)]) + .await; + + database + .alter_table( + table_name.clone(), + &[TableModification::RenameColumn { + old_name: "name", + new_name: "full_name", + }], + ) + .await + .unwrap(); + + database + .insert_values(table_name.clone(), &["full_name", "age"], &[&"Carol", &41]) + .await + .unwrap(); + + notify.notified().await; + pipeline.shutdown_and_wait().await.unwrap(); + + let events = destination.get_events().await; + let grouped = group_events_by_type_and_table_id(&events); + + assert_eq!( + grouped.get(&(EventType::Relation, table_id)).unwrap().len(), + 1 + ); + assert_eq!( + grouped.get(&(EventType::Insert, table_id)).unwrap().len(), + 1 + ); + + let Event::Relation(r) = get_last_relation_event(&events, table_id) else { + panic!("expected relation event"); + }; + assert_replicated_schema_column_names_types( + &r.replicated_table_schema, + &[ + ("id", Type::INT8), + ("full_name", Type::TEXT), + ("age", Type::INT4), + ], + ); + let Event::Insert(i) = get_last_insert_event(&events, table_id) else { + panic!("expected insert event"); + }; + assert_eq!(i.table_row.values.len(), 3); + + // Verify schema snapshots are stored in order. + let table_schemas = store.get_table_schemas().await; + let snapshots = table_schemas.get(&table_id).unwrap(); + assert_eq!(snapshots.len(), 2); + assert_schema_snapshots_ordering(snapshots, true); + + let (_, first_schema) = &snapshots[0]; + assert_table_schema_column_names_types( + first_schema, + &[ + ("id", Type::INT8), + ("name", Type::TEXT), + ("age", Type::INT4), + ], + ); + + let (_, second_schema) = &snapshots[1]; + assert_table_schema_column_names_types( + second_schema, + &[ + ("id", Type::INT8), + ("full_name", Type::TEXT), + ("age", Type::INT4), + ], + ); +} + +#[tokio::test(flavor = "multi_thread")] +async fn relation_message_updates_when_column_type_changes() { + init_test_tracing(); + + let (database, table_name, table_id, store, destination, pipeline, _pipeline_id, _publication) = + create_database_and_pipeline_with_table( + "schema_change_type", + &[("name", "text not null"), ("age", "integer not null")], + ) + .await; + + let notify = destination + .wait_for_events_count(vec![(EventType::Relation, 1), (EventType::Insert, 1)]) + .await; + + database + .alter_table( + table_name.clone(), + &[TableModification::AlterColumn { + name: "age", + alteration: "type bigint", + }], + ) + .await + .unwrap(); + + database + .insert_values(table_name.clone(), &["name", "age"], &[&"Dave", &45_i64]) + .await + .unwrap(); + + notify.notified().await; + pipeline.shutdown_and_wait().await.unwrap(); + + let events = destination.get_events().await; + let grouped = group_events_by_type_and_table_id(&events); + + assert_eq!( + grouped.get(&(EventType::Relation, table_id)).unwrap().len(), + 1 + ); + assert_eq!( + grouped.get(&(EventType::Insert, table_id)).unwrap().len(), + 1 + ); + + let Event::Relation(r) = get_last_relation_event(&events, table_id) else { + panic!("expected relation event"); + }; + assert_replicated_schema_column_names_types( + &r.replicated_table_schema, + &[ + ("id", Type::INT8), + ("name", Type::TEXT), + ("age", Type::INT8), + ], + ); + let Event::Insert(i) = get_last_insert_event(&events, table_id) else { + panic!("expected insert event"); + }; + assert_eq!(i.table_row.values.len(), 3); + + // Verify schema snapshots are stored in order. + let table_schemas = store.get_table_schemas().await; + let snapshots = table_schemas.get(&table_id).unwrap(); + assert_eq!(snapshots.len(), 2); + assert_schema_snapshots_ordering(snapshots, true); + + let (_, first_schema) = &snapshots[0]; + assert_table_schema_column_names_types( + first_schema, + &[ + ("id", Type::INT8), + ("name", Type::TEXT), + ("age", Type::INT4), + ], + ); + + let (_, second_schema) = &snapshots[1]; + assert_table_schema_column_names_types( + second_schema, + &[ + ("id", Type::INT8), + ("name", Type::TEXT), + ("age", Type::INT8), + ], + ); +} + +#[tokio::test(flavor = "multi_thread")] +async fn pipeline_recovers_after_multiple_schema_changes_and_restart() { + init_test_tracing(); + + // Start with initial schema: id (auto), name (text), age (integer), status (text) + let (database, table_name, table_id, store, destination, pipeline, pipeline_id, publication) = + create_database_and_pipeline_with_table( + "schema_multi_change_restart", + &[ + ("name", "text not null"), + ("age", "integer not null"), + ("status", "text not null"), + ], + ) + .await; + + // Phase 1: Add column + insert, then restart + let notify = destination + .wait_for_events_count(vec![(EventType::Relation, 1), (EventType::Insert, 1)]) + .await; + + database + .alter_table( + table_name.clone(), + &[TableModification::AddColumn { + name: "email", + data_type: "text not null", + }], + ) + .await + .unwrap(); + + database + .insert_values( + table_name.clone(), + &["name", "age", "status", "email"], + &[&"Alice", &25, &"active", &"alice@example.com"], + ) + .await + .unwrap(); + + notify.notified().await; + sleep(Duration::from_secs(5)).await; + pipeline.shutdown_and_wait().await.unwrap(); + destination.clear_events().await; + + // Phase 2: Rename column + change type + insert, then restart + let mut pipeline = create_pipeline( + &database.config, + pipeline_id, + publication.clone(), + store.clone(), + destination.clone(), + ); + pipeline.start().await.unwrap(); + + let notify = destination + .wait_for_events_count(vec![(EventType::Relation, 1), (EventType::Insert, 1)]) + .await; + + database + .alter_table( + table_name.clone(), + &[TableModification::RenameColumn { + old_name: "age", + new_name: "years", + }], + ) + .await + .unwrap(); + + database + .alter_table( + table_name.clone(), + &[TableModification::AlterColumn { + name: "years", + alteration: "type bigint", + }], + ) + .await + .unwrap(); + + database + .insert_values( + table_name.clone(), + &["name", "years", "status", "email"], + &[&"Bob", &30_i64, &"pending", &"bob@example.com"], + ) + .await + .unwrap(); + + notify.notified().await; + sleep(Duration::from_secs(1)).await; + pipeline.shutdown_and_wait().await.unwrap(); + destination.clear_events().await; + + // Phase 3: Drop column + insert, then restart + let mut pipeline = create_pipeline( + &database.config, + pipeline_id, + publication.clone(), + store.clone(), + destination.clone(), + ); + pipeline.start().await.unwrap(); + + let notify = destination + .wait_for_events_count(vec![(EventType::Relation, 1), (EventType::Insert, 1)]) + .await; + + database + .alter_table( + table_name.clone(), + &[TableModification::DropColumn { name: "status" }], + ) + .await + .unwrap(); + + database + .insert_values( + table_name.clone(), + &["name", "years", "email"], + &[&"Carol", &35_i64, &"carol@example.com"], + ) + .await + .unwrap(); + + notify.notified().await; + sleep(Duration::from_secs(1)).await; + pipeline.shutdown_and_wait().await.unwrap(); + destination.clear_events().await; + + // Phase 4: Add another column + rename existing + insert, then verify + let mut pipeline = create_pipeline( + &database.config, + pipeline_id, + publication, + store.clone(), + destination.clone(), + ); + pipeline.start().await.unwrap(); + + let notify = destination + .wait_for_events_count(vec![(EventType::Relation, 1), (EventType::Insert, 1)]) + .await; + + database + .alter_table( + table_name.clone(), + &[TableModification::AddColumn { + name: "created_at", + data_type: "timestamp not null default now()", + }], + ) + .await + .unwrap(); + + database + .alter_table( + table_name.clone(), + &[TableModification::RenameColumn { + old_name: "email", + new_name: "contact_email", + }], + ) + .await + .unwrap(); + + database + .insert_values( + table_name.clone(), + &["name", "years", "contact_email"], + &[&"Dave", &40_i64, &"dave@example.com"], + ) + .await + .unwrap(); + + notify.notified().await; + pipeline.shutdown_and_wait().await.unwrap(); + + // Final schema should be: id (int8), name (text), years (int8), contact_email (text), created_at (timestamp) + let events = destination.get_events().await; + + let Event::Relation(r) = get_last_relation_event(&events, table_id) else { + panic!("expected relation event"); + }; + assert_replicated_schema_column_names_types( + &r.replicated_table_schema, + &[ + ("id", Type::INT8), + ("name", Type::TEXT), + ("years", Type::INT8), + ("contact_email", Type::TEXT), + ("created_at", Type::TIMESTAMP), + ], + ); + let Event::Insert(i) = get_last_insert_event(&events, table_id) else { + panic!("expected insert event"); + }; + assert_eq!(i.table_row.values.len(), 5); + + // Verify all schema snapshots are stored in order. + // We have 7 snapshots: + // - Initial (id, name, age, status) + // - After adding email + // - After renaming age -> years + // - After changing years type to bigint + // - After dropping status + // - After adding created_at + // - After renaming email -> contact_email (this is the final schema for the insert) + let table_schemas = store.get_table_schemas().await; + let snapshots = table_schemas.get(&table_id).unwrap(); + assert_eq!(snapshots.len(), 7); + assert_schema_snapshots_ordering(snapshots, true); + + // Initial schema: id, name, age, status + let (_, schema) = &snapshots[0]; + assert_table_schema_column_names_types( + schema, + &[ + ("id", Type::INT8), + ("name", Type::TEXT), + ("age", Type::INT4), + ("status", Type::TEXT), + ], + ); + + // After adding email: id, name, age, status, email + let (_, schema) = &snapshots[1]; + assert_table_schema_column_names_types( + schema, + &[ + ("id", Type::INT8), + ("name", Type::TEXT), + ("age", Type::INT4), + ("status", Type::TEXT), + ("email", Type::TEXT), + ], + ); + + // After renaming age -> years: id, name, years, status, email + let (_, schema) = &snapshots[2]; + assert_table_schema_column_names_types( + schema, + &[ + ("id", Type::INT8), + ("name", Type::TEXT), + ("years", Type::INT4), + ("status", Type::TEXT), + ("email", Type::TEXT), + ], + ); + + // After changing years type to bigint: id, name, years (int8), status, email + let (_, schema) = &snapshots[3]; + assert_table_schema_column_names_types( + schema, + &[ + ("id", Type::INT8), + ("name", Type::TEXT), + ("years", Type::INT8), + ("status", Type::TEXT), + ("email", Type::TEXT), + ], + ); + + // After dropping status: id, name, years, email + let (_, schema) = &snapshots[4]; + assert_table_schema_column_names_types( + schema, + &[ + ("id", Type::INT8), + ("name", Type::TEXT), + ("years", Type::INT8), + ("email", Type::TEXT), + ], + ); + + // After adding created_at: id, name, years, email, created_at + let (_, schema) = &snapshots[5]; + assert_table_schema_column_names_types( + schema, + &[ + ("id", Type::INT8), + ("name", Type::TEXT), + ("years", Type::INT8), + ("email", Type::TEXT), + ("created_at", Type::TIMESTAMP), + ], + ); + + // Final schema after renaming email -> contact_email: id, name, years, contact_email, created_at + let (_, schema) = &snapshots[6]; + assert_table_schema_column_names_types( + schema, + &[ + ("id", Type::INT8), + ("name", Type::TEXT), + ("years", Type::INT8), + ("contact_email", Type::TEXT), + ("created_at", Type::TIMESTAMP), + ], + ); +} + +#[tokio::test(flavor = "multi_thread")] +async fn partitioned_table_schema_change_updates_relation_message() { + init_test_tracing(); + let database = spawn_source_database().await; + + let table_name = test_table_name("partitioned_schema_change"); + let partition_specs = [("p1", "from (1) to (100)"), ("p2", "from (100) to (200)")]; + + let (parent_table_id, _partition_table_ids) = + create_partitioned_table(&database, table_name.clone(), &partition_specs) + .await + .unwrap(); + + // Insert initial data into partitions. + database + .run_sql(&format!( + "insert into {} (data, partition_key) values ('event1', 50), ('event2', 150)", + table_name.as_quoted_identifier() + )) + .await + .unwrap(); + + let publication_name = "test_partitioned_schema_change_pub".to_string(); + database + .create_publication(&publication_name, std::slice::from_ref(&table_name)) + .await + .unwrap(); + + let state_store = NotifyingStore::new(); + let destination = TestDestinationWrapper::wrap(MemoryDestination::new()); + + let pipeline_id: PipelineId = random(); + let mut pipeline = create_pipeline( + &database.config, + pipeline_id, + publication_name, + state_store.clone(), + destination.clone(), + ); + + let parent_sync_done = state_store + .notify_on_table_state_type(parent_table_id, TableReplicationPhaseType::SyncDone) + .await; + + pipeline.start().await.unwrap(); + + parent_sync_done.notified().await; + + // Wait for the Relation event (schema change) and Insert event. + let notify = destination + .wait_for_events_count(vec![(EventType::Relation, 1), (EventType::Insert, 1)]) + .await; + + // Add a new column to the partitioned table. + database + .alter_table( + table_name.clone(), + &[TableModification::AddColumn { + name: "category", + data_type: "text not null default 'default_category'", + }], + ) + .await + .unwrap(); + + // Insert a row with the new column into one of the partitions. + database + .run_sql(&format!( + "insert into {} (data, partition_key, category) values ('event3', 75, 'test_category')", + table_name.as_quoted_identifier() + )) + .await + .unwrap(); + + notify.notified().await; + pipeline.shutdown_and_wait().await.unwrap(); + + let events = destination.get_events().await; + let grouped = group_events_by_type_and_table_id(&events); + + // Verify we received exactly 1 Relation event for the parent table. + assert_eq!( + grouped + .get(&(EventType::Relation, parent_table_id)) + .unwrap() + .len(), + 1 + ); + assert_eq!( + grouped + .get(&(EventType::Insert, parent_table_id)) + .unwrap() + .len(), + 1 + ); + + // Verify the Relation event has the updated schema with the new column. + let Event::Relation(r) = get_last_relation_event(&events, parent_table_id) else { + panic!("expected relation event"); + }; + assert_replicated_schema_column_names_types( + &r.replicated_table_schema, + &[ + ("id", Type::INT8), + ("data", Type::TEXT), + ("partition_key", Type::INT4), + ("category", Type::TEXT), + ], + ); + + // Verify the Insert event has 4 columns. + let Event::Insert(i) = get_last_insert_event(&events, parent_table_id) else { + panic!("expected insert event"); + }; + assert_eq!(i.table_row.values.len(), 4); + + // Verify schema snapshots are stored in order. + let table_schemas = state_store.get_table_schemas().await; + let snapshots = table_schemas.get(&parent_table_id).unwrap(); + assert_eq!(snapshots.len(), 2); + assert_schema_snapshots_ordering(snapshots, true); + + // Initial schema: id, data, partition_key. + let (_, first_schema) = &snapshots[0]; + assert_table_schema_column_names_types( + first_schema, + &[ + ("id", Type::INT8), + ("data", Type::TEXT), + ("partition_key", Type::INT4), + ], + ); + + // After adding category: id, data, partition_key, category. + let (_, second_schema) = &snapshots[1]; + assert_table_schema_column_names_types( + second_schema, + &[ + ("id", Type::INT8), + ("data", Type::TEXT), + ("partition_key", Type::INT4), + ("category", Type::TEXT), + ], + ); +} diff --git a/etl/tests/postgres_store.rs b/etl/tests/postgres_store.rs index 6e932895a..c12806555 100644 --- a/etl/tests/postgres_store.rs +++ b/etl/tests/postgres_store.rs @@ -7,7 +7,7 @@ use etl::store::schema::SchemaStore; use etl::store::state::StateStore; use etl::test_utils::database::spawn_source_database; use etl_postgres::replication::connect_to_source_database; -use etl_postgres::types::{ColumnSchema, TableId, TableName, TableSchema}; +use etl_postgres::types::{ColumnSchema, SnapshotId, TableId, TableName, TableSchema}; use etl_telemetry::tracing::init_test_tracing; use sqlx::postgres::types::Oid as SqlxTableId; use tokio_postgres::types::{PgLsn, Type as PgType}; @@ -244,7 +244,10 @@ async fn test_schema_store_operations() { let table_id = table_schema.id; // Test initial state - should be empty - let schema = store.get_table_schema(&table_id).await.unwrap(); + let schema = store + .get_table_schema(&table_id, SnapshotId::max()) + .await + .unwrap(); assert!(schema.is_none()); let all_schemas = store.get_table_schemas().await.unwrap(); @@ -256,7 +259,10 @@ async fn test_schema_store_operations() { .await .unwrap(); - let schema = store.get_table_schema(&table_id).await.unwrap(); + let schema = store + .get_table_schema(&table_id, SnapshotId::max()) + .await + .unwrap(); assert!(schema.is_some()); let schema = schema.unwrap(); assert_eq!(schema.id, table_schema.id); @@ -316,13 +322,19 @@ async fn test_schema_store_load_schemas() { let schemas = new_store.get_table_schemas().await.unwrap(); assert_eq!(schemas.len(), 2); - let schema1 = new_store.get_table_schema(&table_schema1.id).await.unwrap(); + let schema1 = new_store + .get_table_schema(&table_schema1.id, SnapshotId::max()) + .await + .unwrap(); assert!(schema1.is_some()); let schema1 = schema1.unwrap(); assert_eq!(schema1.id, table_schema1.id); assert_eq!(schema1.name, table_schema1.name); - let schema2 = new_store.get_table_schema(&table_schema2.id).await.unwrap(); + let schema2 = new_store + .get_table_schema(&table_schema2.id, SnapshotId::max()) + .await + .unwrap(); assert!(schema2.is_some()); let schema2 = schema2.unwrap(); assert_eq!(schema2.id, table_schema2.id); @@ -330,7 +342,7 @@ async fn test_schema_store_load_schemas() { } #[tokio::test(flavor = "multi_thread")] -async fn test_schema_store_update_existing() { +async fn test_schema_store_versioning() { init_test_tracing(); let database = spawn_source_database().await; @@ -339,13 +351,13 @@ async fn test_schema_store_update_existing() { let store = PostgresStore::new(pipeline_id, database.config.clone()); let mut table_schema = create_sample_table_schema(); - // Store initial schema + // Store initial schema at snapshot 0 store .store_table_schema(table_schema.clone()) .await .unwrap(); - // Update schema by adding a column + // Create a new version with a higher snapshot_id table_schema.add_column_schema(test_column( "updated_at", PgType::TIMESTAMPTZ, @@ -354,20 +366,40 @@ async fn test_schema_store_update_existing() { true, false, )); + table_schema.snapshot_id = SnapshotId::from(100u64); // New snapshot for the schema change - // Store updated schema + // Store updated schema as new version store .store_table_schema(table_schema.clone()) .await .unwrap(); - // Verify updated schema - let schema = store.get_table_schema(&table_schema.id).await.unwrap(); + // Verify querying at snapshot 100+ returns the updated schema + let schema = store + .get_table_schema(&table_schema.id, SnapshotId::max()) + .await + .unwrap(); assert!(schema.is_some()); let schema = schema.unwrap(); assert_eq!(schema.column_schemas.len(), 4); // Original 3 + 1 new column + assert_eq!(schema.snapshot_id, SnapshotId::from(100u64)); - // Verify the new column was added + // Verify querying at snapshot 50 returns the original schema + let schema = store + .get_table_schema(&table_schema.id, SnapshotId::from(50u64)) + .await + .unwrap(); + assert!(schema.is_some()); + let schema = schema.unwrap(); + assert_eq!(schema.column_schemas.len(), 3); // Original 3 columns + assert_eq!(schema.snapshot_id, SnapshotId::initial()); + + // Verify the new column was added in the latest version + let schema = store + .get_table_schema(&table_schema.id, SnapshotId::max()) + .await + .unwrap() + .unwrap(); let updated_at_column = schema .column_schemas .iter() @@ -375,6 +407,165 @@ async fn test_schema_store_update_existing() { assert!(updated_at_column.is_some()); } +#[tokio::test(flavor = "multi_thread")] +async fn test_schema_store_upsert_replaces_columns() { + init_test_tracing(); + + let database = spawn_source_database().await; + let pipeline_id = 1; + + let store = PostgresStore::new(pipeline_id, database.config.clone()); + + // Create initial schema with 3 columns + let table_id = TableId::new(12345); + let table_name = TableName::new("public".to_string(), "test_table".to_string()); + let initial_columns = vec![ + test_column("id", PgType::INT4, -1, 1, false, true), + test_column("name", PgType::TEXT, -1, 2, true, false), + test_column("old_column", PgType::TEXT, -1, 3, true, false), + ]; + let table_schema = TableSchema::new(table_id, table_name.clone(), initial_columns); + + // Store initial schema + store + .store_table_schema(table_schema.clone()) + .await + .unwrap(); + + // Verify initial columns + let schema = store + .get_table_schema(&table_id, SnapshotId::max()) + .await + .unwrap() + .unwrap(); + assert_eq!(schema.column_schemas.len(), 3); + assert!(schema.column_schemas.iter().any(|c| c.name == "old_column")); + + // Create updated schema with SAME snapshot_id but different columns + // (simulating a retry or re-processing scenario) + let updated_columns = vec![ + test_column("id", PgType::INT4, -1, 1, false, true), + test_column("name", PgType::TEXT, -1, 2, true, false), + test_column("new_column", PgType::TEXT, -1, 3, true, false), // replaced old_column + test_column("extra_column", PgType::INT8, -1, 4, true, false), // added column + ]; + let updated_schema = TableSchema::new(table_id, table_name, updated_columns); + + // Store updated schema with same snapshot_id (upsert) + store + .store_table_schema(updated_schema.clone()) + .await + .unwrap(); + + // Verify columns were replaced, not accumulated + // Need to clear cache and reload from DB to verify DB state + let new_store = PostgresStore::new(pipeline_id, database.config.clone()); + let schema = new_store + .get_table_schema(&table_id, SnapshotId::max()) + .await + .unwrap() + .unwrap(); + + assert_eq!(schema.column_schemas.len(), 4); // Should be 4, not 3+4=7 + assert!( + !schema.column_schemas.iter().any(|c| c.name == "old_column"), + "old_column should have been deleted" + ); + assert!( + schema.column_schemas.iter().any(|c| c.name == "new_column"), + "new_column should exist" + ); + assert!( + schema + .column_schemas + .iter() + .any(|c| c.name == "extra_column"), + "extra_column should exist" + ); +} + +#[tokio::test(flavor = "multi_thread")] +async fn test_schema_cache_eviction() { + init_test_tracing(); + + let database = spawn_source_database().await; + let pipeline_id = 1; + + let store = PostgresStore::new(pipeline_id, database.config.clone()); + + // Store 3 schema versions for table 1 + let table_id_1 = TableId::new(12345); + let table_name_1 = TableName::new("public".to_string(), "test_table".to_string()); + for snapshot_id in [0u64, 100, 200] { + let columns = vec![ + test_column("id", PgType::INT4, -1, 1, false, true), + test_column( + &format!("col_at_{snapshot_id}"), + PgType::TEXT, + -1, + 2, + true, + false, + ), + ]; + let mut table_schema = TableSchema::new(table_id_1, table_name_1.clone(), columns); + table_schema.snapshot_id = SnapshotId::from(snapshot_id); + store + .store_table_schema(table_schema.clone()) + .await + .unwrap(); + } + + // Store 3 schemas for table 2 to verify eviction is per-table + let table_id_2 = TableId::new(67890); + let table_name_2 = TableName::new("public".to_string(), "table_2".to_string()); + for snapshot_id in [0u64, 100, 200] { + let columns = vec![test_column("id", PgType::INT4, -1, 1, false, true)]; + let mut schema = TableSchema::new(table_id_2, table_name_2.clone(), columns); + schema.snapshot_id = SnapshotId::from(snapshot_id); + store.store_table_schema(schema).await.unwrap(); + } + + // Check cache size - should have 2 schemas per table = 4 total + let cached_schemas = store.get_table_schemas().await.unwrap(); + assert_eq!(cached_schemas.len(), 4, "Should have 2 schemas per table"); + + // Verify eviction keeps newest snapshots (100 and 200), evicts oldest (0) + let table_1_snapshots: Vec = cached_schemas + .iter() + .filter(|s| s.id == table_id_1) + .map(|s| s.snapshot_id) + .collect(); + assert!( + table_1_snapshots.contains(&SnapshotId::from(100u64)) + && table_1_snapshots.contains(&SnapshotId::from(200u64)) + ); + assert!( + !table_1_snapshots.contains(&SnapshotId::initial()), + "Snapshot 0 should be evicted" + ); + + let table_2_snapshots: Vec = cached_schemas + .iter() + .filter(|s| s.id == table_id_2) + .map(|s| s.snapshot_id) + .collect(); + assert!( + !table_2_snapshots.contains(&SnapshotId::initial()), + "Table 2 snapshot 0 should be evicted" + ); + + // Evicted schemas should still be loadable from DB + let new_store = PostgresStore::new(pipeline_id, database.config.clone()); + let schema_0 = new_store + .get_table_schema(&table_id_1, SnapshotId::initial()) + .await + .unwrap() + .unwrap(); + assert_eq!(schema_0.snapshot_id, SnapshotId::initial()); + assert!(schema_0.column_schemas.iter().any(|c| c.name == "col_at_0")); +} + #[tokio::test(flavor = "multi_thread")] async fn test_multiple_pipelines_isolation() { init_test_tracing(); @@ -387,26 +578,27 @@ async fn test_multiple_pipelines_isolation() { let store1 = PostgresStore::new(pipeline_id1, database.config.clone()); let store2 = PostgresStore::new(pipeline_id2, database.config.clone()); - // Add state to pipeline 1 + // Test state isolation let init_phase = TableReplicationPhase::Init; store1 .update_table_replication_state(table_id, init_phase.clone()) .await .unwrap(); - // Add different state to pipeline 2 for the same table let data_sync_phase = TableReplicationPhase::DataSync; store2 .update_table_replication_state(table_id, data_sync_phase.clone()) .await .unwrap(); - // Verify isolation - each pipeline sees only its own state - let state1 = store1.get_table_replication_state(table_id).await.unwrap(); - assert_eq!(state1, Some(init_phase)); - - let state2 = store2.get_table_replication_state(table_id).await.unwrap(); - assert_eq!(state2, Some(data_sync_phase)); + assert_eq!( + store1.get_table_replication_state(table_id).await.unwrap(), + Some(init_phase) + ); + assert_eq!( + store2.get_table_replication_state(table_id).await.unwrap(), + Some(data_sync_phase) + ); // Test schema isolation let table_schema1 = create_sample_table_schema(); @@ -421,7 +613,6 @@ async fn test_multiple_pipelines_isolation() { .await .unwrap(); - // Each pipeline sees only its own schemas let schemas1 = store1.get_table_schemas().await.unwrap(); assert_eq!(schemas1.len(), 1); assert_eq!(schemas1[0].id, table_schema1.id); @@ -429,6 +620,33 @@ async fn test_multiple_pipelines_isolation() { let schemas2 = store2.get_table_schemas().await.unwrap(); assert_eq!(schemas2.len(), 1); assert_eq!(schemas2[0].id, table_schema2.id); + + // Test table mappings isolation + store1 + .store_table_mapping(table_id, "pipeline1_table".to_string()) + .await + .unwrap(); + store2 + .store_table_mapping(table_id, "pipeline2_table".to_string()) + .await + .unwrap(); + + assert_eq!( + store1.get_table_mapping(&table_id).await.unwrap(), + Some("pipeline1_table".to_string()) + ); + assert_eq!( + store2.get_table_mapping(&table_id).await.unwrap(), + Some("pipeline2_table".to_string()) + ); + + // Verify isolation persists after loading from database + let new_store1 = PostgresStore::new(pipeline_id1, database.config.clone()); + new_store1.load_table_mappings().await.unwrap(); + assert_eq!( + new_store1.get_table_mapping(&table_id).await.unwrap(), + Some("pipeline1_table".to_string()) + ); } #[tokio::test(flavor = "multi_thread")] @@ -641,45 +859,6 @@ async fn test_table_mappings_persistence_and_loading() { ); } -#[tokio::test(flavor = "multi_thread")] -async fn test_table_mappings_pipeline_isolation() { - init_test_tracing(); - - let database = spawn_source_database().await; - let pipeline_id1 = 1; - let pipeline_id2 = 2; - - let store1 = PostgresStore::new(pipeline_id1, database.config.clone()); - let store2 = PostgresStore::new(pipeline_id2, database.config.clone()); - - let table_id = TableId::new(12345); - - // Store different mappings for the same table ID in different pipelines - store1 - .store_table_mapping(table_id, "pipeline1_table".to_string()) - .await - .unwrap(); - - store2 - .store_table_mapping(table_id, "pipeline2_table".to_string()) - .await - .unwrap(); - - // Verify isolation - each pipeline sees only its own mapping - let mapping1 = store1.get_table_mapping(&table_id).await.unwrap(); - assert_eq!(mapping1, Some("pipeline1_table".to_string())); - - let mapping2 = store2.get_table_mapping(&table_id).await.unwrap(); - assert_eq!(mapping2, Some("pipeline2_table".to_string())); - - // Verify isolation persists after loading from database - let new_store1 = PostgresStore::new(pipeline_id1, database.config.clone()); - new_store1.load_table_mappings().await.unwrap(); - - let loaded_mapping1 = new_store1.get_table_mapping(&table_id).await.unwrap(); - assert_eq!(loaded_mapping1, Some("pipeline1_table".to_string())); -} - #[tokio::test(flavor = "multi_thread")] async fn test_cleanup_deletes_state_schema_and_mapping_for_table() { init_test_tracing(); @@ -689,6 +868,13 @@ async fn test_cleanup_deletes_state_schema_and_mapping_for_table() { let store = PostgresStore::new(pipeline_id, database.config.clone()); + // Test idempotency: cleanup on non-existent table should succeed + let nonexistent_table_id = TableId::new(99999); + store + .cleanup_table_state(nonexistent_table_id) + .await + .unwrap(); + // Prepare two tables: one we will delete, one we will keep let table_1_schema = create_sample_table_schema(); let table_1_id = table_1_schema.id; @@ -731,7 +917,13 @@ async fn test_cleanup_deletes_state_schema_and_mapping_for_table() { .unwrap() .is_some() ); - assert!(store.get_table_schema(&table_1_id).await.unwrap().is_some()); + assert!( + store + .get_table_schema(&table_1_id, SnapshotId::max()) + .await + .unwrap() + .is_some() + ); assert!( store .get_table_mapping(&table_1_id) @@ -751,7 +943,13 @@ async fn test_cleanup_deletes_state_schema_and_mapping_for_table() { .unwrap() .is_none() ); - assert!(store.get_table_schema(&table_1_id).await.unwrap().is_none()); + assert!( + store + .get_table_schema(&table_1_id, SnapshotId::max()) + .await + .unwrap() + .is_none() + ); assert!( store .get_table_mapping(&table_1_id) @@ -768,7 +966,13 @@ async fn test_cleanup_deletes_state_schema_and_mapping_for_table() { .unwrap() .is_some() ); - assert!(store.get_table_schema(&table_2_id).await.unwrap().is_some()); + assert!( + store + .get_table_schema(&table_2_id, SnapshotId::max()) + .await + .unwrap() + .is_some() + ); assert!( store .get_table_mapping(&table_2_id) @@ -793,7 +997,7 @@ async fn test_cleanup_deletes_state_schema_and_mapping_for_table() { ); assert!( new_store - .get_table_schema(&table_1_id) + .get_table_schema(&table_1_id, SnapshotId::max()) .await .unwrap() .is_none() @@ -816,7 +1020,7 @@ async fn test_cleanup_deletes_state_schema_and_mapping_for_table() { ); assert!( new_store - .get_table_schema(&table_2_id) + .get_table_schema(&table_2_id, SnapshotId::max()) .await .unwrap() .is_some() @@ -829,53 +1033,3 @@ async fn test_cleanup_deletes_state_schema_and_mapping_for_table() { .is_some() ); } - -#[tokio::test(flavor = "multi_thread")] -async fn test_cleanup_idempotent_when_no_state_present() { - init_test_tracing(); - - let database = spawn_source_database().await; - let pipeline_id = 1; - let store = PostgresStore::new(pipeline_id, database.config.clone()); - - let table_schema = create_sample_table_schema(); - let table_id = table_schema.id; - - // Ensure no state exists yet - assert!( - store - .get_table_replication_state(table_id) - .await - .unwrap() - .is_none() - ); - assert!(store.get_table_schema(&table_id).await.unwrap().is_none()); - assert!(store.get_table_mapping(&table_id).await.unwrap().is_none()); - - // Calling cleanup should succeed even if nothing exists - store.cleanup_table_state(table_id).await.unwrap(); - - // Add state and clean up again - store - .update_table_replication_state(table_id, TableReplicationPhase::Init) - .await - .unwrap(); - store.store_table_schema(table_schema).await.unwrap(); - store - .store_table_mapping(table_id, "dest_table".to_string()) - .await - .unwrap(); - - store.cleanup_table_state(table_id).await.unwrap(); - - // Verify everything is gone - assert!( - store - .get_table_replication_state(table_id) - .await - .unwrap() - .is_none() - ); - assert!(store.get_table_schema(&table_id).await.unwrap().is_none()); - assert!(store.get_table_mapping(&table_id).await.unwrap().is_none()); -} diff --git a/etl/tests/replication.rs b/etl/tests/replication.rs index e85fc3ef5..6ecaee891 100644 --- a/etl/tests/replication.rs +++ b/etl/tests/replication.rs @@ -6,7 +6,7 @@ use etl::error::ErrorKind; use etl::replication::client::PgReplicationClient; use etl::test_utils::database::{spawn_source_database, test_table_name}; use etl::test_utils::pipeline::test_slot_name; -use etl::test_utils::table::assert_table_schema; +use etl::test_utils::schema::assert_table_schema_columns; use etl::test_utils::test_schema::create_partitioned_table; use etl_postgres::below_version; use etl_postgres::tokio::test_utils::{TableModification, id_column_schema}; @@ -204,14 +204,12 @@ async fn test_table_schema_copy_is_consistent() { .unwrap(); // We use the transaction to consistently read the table schemas. - let table_1_schema = transaction.get_table_schemas(&[table_1_id]).await.unwrap(); + let table_1_schemas = transaction.get_table_schemas(&[table_1_id]).await.unwrap(); transaction.commit().await.unwrap(); - assert_table_schema( - &table_1_schema, - table_1_id, - test_table_name("table_1"), - &[id_column_schema(), age_schema.clone()], - ); + let table_1_schema = table_1_schemas.get(&table_1_id).unwrap(); + assert_eq!(table_1_schema.id, table_1_id); + assert_eq!(table_1_schema.name, test_table_name("table_1")); + assert_table_schema_columns(table_1_schema, &[id_column_schema(), age_schema.clone()]); } #[tokio::test(flavor = "multi_thread")] @@ -241,14 +239,12 @@ async fn test_table_schema_copy_across_multiple_connections() { .unwrap(); // We use the transaction to consistently read the table schemas. - let table_1_schema = transaction.get_table_schemas(&[table_1_id]).await.unwrap(); + let table_1_schemas = transaction.get_table_schemas(&[table_1_id]).await.unwrap(); transaction.commit().await.unwrap(); - assert_table_schema( - &table_1_schema, - table_1_id, - test_table_name("table_1"), - &[id_column_schema(), age_schema.clone()], - ); + let table_1_schema = table_1_schemas.get(&table_1_id).unwrap(); + assert_eq!(table_1_schema.id, table_1_id); + assert_eq!(table_1_schema.name, test_table_name("table_1")); + assert_table_schema_columns(table_1_schema, &[id_column_schema(), age_schema.clone()]); // We create a new table in the database and update the schema of the old one. let table_2_id = database @@ -273,21 +269,20 @@ async fn test_table_schema_copy_across_multiple_connections() { .unwrap(); // We use the transaction to consistently read the table schemas. - let table_1_schema = transaction.get_table_schemas(&[table_1_id]).await.unwrap(); - let table_2_schema = transaction.get_table_schemas(&[table_2_id]).await.unwrap(); + let table_1_schemas = transaction.get_table_schemas(&[table_1_id]).await.unwrap(); + let table_2_schemas = transaction.get_table_schemas(&[table_2_id]).await.unwrap(); transaction.commit().await.unwrap(); - assert_table_schema( - &table_1_schema, - table_1_id, - test_table_name("table_1"), + let table_1_schema = table_1_schemas.get(&table_1_id).unwrap(); + assert_eq!(table_1_schema.id, table_1_id); + assert_eq!(table_1_schema.name, test_table_name("table_1")); + assert_table_schema_columns( + table_1_schema, &[id_column_schema(), age_schema.clone(), year_schema.clone()], ); - assert_table_schema( - &table_2_schema, - table_2_id, - test_table_name("table_2"), - &[id_column_schema(), year_schema], - ); + let table_2_schema = table_2_schemas.get(&table_2_id).unwrap(); + assert_eq!(table_2_schema.id, table_2_id); + assert_eq!(table_2_schema.name, test_table_name("table_2")); + assert_table_schema_columns(table_2_schema, &[id_column_schema(), year_schema]); } #[tokio::test(flavor = "multi_thread")] @@ -469,10 +464,10 @@ async fn test_get_replicated_column_names_respects_column_filter() { let table_schema = &table_schemas[&test_table_id]; // Verify all columns are present in the schema. - assert_table_schema( - &table_schemas, - test_table_id, - test_table_name, + assert_eq!(table_schema.id, test_table_id); + assert_eq!(table_schema.name, test_table_name); + assert_table_schema_columns( + table_schema, &[ id_column_schema(), test_column("name", Type::TEXT, 2, true, false), diff --git a/scripts/docker-compose.yaml b/scripts/docker-compose.yaml index bc4d07fc0..91c5856a4 100644 --- a/scripts/docker-compose.yaml +++ b/scripts/docker-compose.yaml @@ -19,12 +19,16 @@ services: - "${POSTGRES_PORT:-5430}:5432" volumes: - ${POSTGRES_DATA_VOLUME:-postgres_data}:/var/lib/postgresql/data + # These parameters are configured for local testing only. + # In production, use PostgreSQL defaults or your own tuned values. + # `wal_sender_timeout/2` is the keepalive timeout for streaming replication. command: > postgres -N 1000 -c wal_level=logical -c max_wal_senders=100 -c max_replication_slots=100 + -c wal_sender_timeout=10s restart: unless-stopped healthcheck: test: ["CMD-SHELL", "pg_isready -U ${POSTGRES_USER:-postgres}"]