diff --git a/.config/nextest.toml b/.config/nextest.toml index 02c759624..bfe02928f 100644 --- a/.config/nextest.toml +++ b/.config/nextest.toml @@ -10,5 +10,5 @@ test-threads = "num-cpus" max-threads = 1 [[profile.default.overrides]] -filter = "test(exclusive_) | binary_id(etl::main) | (binary_id(etl-destinations::main) & test(/^(bigquery_pipeline|ducklake_destination|ducklake_pipeline|iceberg_destination)::/)) | (binary_id(etl-destinations) & test(/ducklake::core::tests::postgres_backed::/))" +filter = "test(exclusive_) | binary_id(etl::main) | (binary_id(etl-destinations::main) & test(/^(bigquery_pipeline|clickhouse_pipeline|ducklake_destination|ducklake_pipeline|iceberg_destination)::/)) | (binary_id(etl-destinations) & test(/ducklake::core::tests::postgres_backed::/))" test-group = "shared-pg" diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index 2855b9651..5c53edcca 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -177,6 +177,9 @@ jobs: NUM_LOCAL_DATABASES: 4 TESTS_DATABASE_USERNAME: postgres TESTS_DATABASE_PASSWORD: postgres + TESTS_CLICKHOUSE_URL: http://localhost:8123 + TESTS_CLICKHOUSE_USER: etl + TESTS_CLICKHOUSE_PASSWORD: etl ETL_DUCKDB_EXTENSION_ROOT: ${{ github.workspace }}/vendor/duckdb/extensions steps: - name: Checkout diff --git a/Cargo.lock b/Cargo.lock index b245df38c..b086e7771 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -945,6 +945,12 @@ dependencies = [ "objc2", ] +[[package]] +name = "bnum" +version = "0.13.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "119771309b95163ec7aaf79810da82f7cd0599c19722d48b9c03894dca833966" + [[package]] name = "bon" version = "3.9.1" @@ -1015,6 +1021,15 @@ dependencies = [ "alloc-stdlib", ] +[[package]] +name = "bstr" +version = "1.12.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "63044e1ae8e69f3b5a92c736ca6269b8d12fa7efe39bf34ddb06d102cf0e2cab" +dependencies = [ + "memchr", +] + [[package]] name = "bumpalo" version = "3.20.2" @@ -1060,6 +1075,9 @@ name = "bytes" version = "1.11.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1e748733b7cbc798e1434b6ac524f0c1ff2ab456fe201501e6497c8417a4fc33" +dependencies = [ + "serde", +] [[package]] name = "bytestring" @@ -1114,6 +1132,12 @@ dependencies = [ "windows-link", ] +[[package]] +name = "cityhash-rs" +version = "1.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "93a719913643003b84bd13022b4b7e703c09342cd03b679c4641c7d2e50dc34d" + [[package]] name = "clap" version = "4.6.1" @@ -1154,6 +1178,56 @@ version = "1.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c8d4a3bb8b1e0c1050499d1815f5ab16d04f0959b233085fb31653fbfc9d98f9" +[[package]] +name = "clickhouse" +version = "0.14.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0bfb36e41b644dcd5be4ef54a3b7d2abc9bb07eda777ab3f90d1b0dbb97c940a" +dependencies = [ + "bnum", + "bstr", + "bytes", + "cityhash-rs", + "clickhouse-macros", + "clickhouse-types", + "futures-channel", + "futures-util", + "http-body-util", + "hyper", + "hyper-rustls", + "hyper-util", + "lz4_flex 0.11.6", + "polonius-the-crab", + "quanta", + "rustls", + "serde", + "thiserror 2.0.18", + "tokio", + "url", +] + +[[package]] +name = "clickhouse-macros" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ff6669899e23cb87b43daf7996f0ea3b9c07d0fb933d745bb7b815b052515ae3" +dependencies = [ + "proc-macro2", + "quote", + "serde_derive_internals", + "syn 2.0.117", +] + +[[package]] +name = "clickhouse-types" +version = "0.1.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "30a5efddc880ce9e2573bd867413d9056fa2bea0206af88dec21e72178b9dc74" +dependencies = [ + "bytes", + "thiserror 2.0.18", +] + [[package]] name = "cmake" version = "0.1.58" @@ -1823,6 +1897,7 @@ dependencies = [ "tokio", "tracing", "tracing-actix-web", + "url", "utoipa", "utoipa-swagger-ui", "uuid", @@ -1871,6 +1946,7 @@ dependencies = [ "async-trait", "base64", "chrono", + "clickhouse", "duckdb", "etl", "etl-postgres", @@ -1907,7 +1983,9 @@ version = "0.1.0" dependencies = [ "clap", "etl", + "etl-config", "etl-destinations", + "etl-telemetry", "rustls", "tokio", "tracing", @@ -2509,6 +2587,17 @@ version = "0.4.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7f24254aa9a54b5c858eaee2f5bccdb46aaf0e486a595ed5fd8f86ba55232a70" +[[package]] +name = "higher-kinded-types" +version = "0.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e690f8474c6c5d8ff99656fcbc195a215acc3949481a8b0b3351c838972dc776" +dependencies = [ + "macro_rules_attribute", + "never-say-never", + "paste", +] + [[package]] name = "hkdf" version = "0.12.4" @@ -3422,6 +3511,12 @@ version = "0.1.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "112b39cec0b298b6c1999fee3e31427f74f676e4cb9879ed1a121b43661a4154" +[[package]] +name = "lz4_flex" +version = "0.11.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "373f5eceeeab7925e0c1098212f2fbc4d416adec9d35051a6ab251e824c1854a" + [[package]] name = "lz4_flex" version = "0.12.1" @@ -3431,6 +3526,22 @@ dependencies = [ "twox-hash", ] +[[package]] +name = "macro_rules_attribute" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "65049d7923698040cd0b1ddcced9b0eb14dd22c5f86ae59c3740eab64a676520" +dependencies = [ + "macro_rules_attribute-proc_macro", + "paste", +] + +[[package]] +name = "macro_rules_attribute-proc_macro" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "670fdfda89751bc4a84ac13eaa63e205cf0fd22b4c9a5fbfa085b63c1f1d3a30" + [[package]] name = "matchers" version = "0.2.0" @@ -3601,6 +3712,12 @@ dependencies = [ "tempfile", ] +[[package]] +name = "never-say-never" +version = "6.6.666" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cf5a574dadd7941adeaa71823ecba5e28331b8313fb2e1c6a5c7e5981ea53ad6" + [[package]] name = "nix" version = "0.30.1" @@ -4086,7 +4203,7 @@ dependencies = [ "futures", "half", "hashbrown 0.16.1", - "lz4_flex", + "lz4_flex 0.12.1", "num-bigint", "num-integer", "num-traits", @@ -4310,6 +4427,16 @@ version = "0.2.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "b4596b6d070b27117e987119b4dac604f3c58cfb0b191112e24771b2faeac1a6" +[[package]] +name = "polonius-the-crab" +version = "0.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ec242d7eccbb2fd8b3b5b6e3cf89f94a91a800f469005b44d154359609f8af72" +dependencies = [ + "higher-kinded-types", + "never-say-never", +] + [[package]] name = "portable-atomic" version = "1.13.1" diff --git a/Cargo.toml b/Cargo.toml index ebb6d1900..26990f975 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -74,6 +74,7 @@ byteorder = { version = "1.5.0", default-features = false } bytes = { version = "1.10.1" } chrono = { version = "0.4.41", default-features = false } clap = { version = "4.5.42", default-features = false } +clickhouse = { version = "0.14", default-features = false } config = { version = "0.14", default-features = false } configcat = { version = "0.1.3", default-features = false } const-oid = { version = "0.9.6", default-features = false } @@ -132,7 +133,7 @@ tracing-actix-web = { version = "0.7.19", default-features = false } tracing-appender = { version = "0.2.3", default-features = false } tracing-log = { version = "0.2.0", default-features = false } tracing-subscriber = { version = "0.3.19", default-features = false } -url = { version = "2.5.8" } +url = { version = "2.5.8", features = ["serde"] } utoipa = { version = "5.4.0", default-features = false } utoipa-swagger-ui = { version = "9.0.2", default-features = false, features = [ "vendored", diff --git a/DEVELOPMENT.md b/DEVELOPMENT.md index cef064425..08e23d02d 100644 --- a/DEVELOPMENT.md +++ b/DEVELOPMENT.md @@ -66,10 +66,10 @@ The fastest way to get started is using the setup script: ``` This script will: -1. Start PostgreSQL via Docker Compose -2. Run etl-api migrations -3. Seed the default replicator image -4. Configure the Kubernetes environment (OrbStack) +1. Start PostgreSQL, ClickHouse, and the local Iceberg dependencies via Docker Compose. +2. Run etl-api migrations. +3. Seed the default replicator image. +4. Configure the Kubernetes environment (OrbStack). ## Database Setup @@ -100,12 +100,19 @@ POSTGRES_DATA_VOLUME=/path/to/data ./scripts/init.sh | `POSTGRES_DB` | `postgres` | Database name | | `POSTGRES_PORT` | `5430` | Database port | | `POSTGRES_HOST` | `localhost` | Database host | +| `CLICKHOUSE_HTTP_PORT` | `8123` | ClickHouse HTTP port | +| `CLICKHOUSE_NATIVE_PORT` | `9000` | ClickHouse native TCP port | +| `CLICKHOUSE_USER` | `etl` | ClickHouse user for the local Docker Compose setup | +| `CLICKHOUSE_PASSWORD` | `etl` | ClickHouse password for the local Docker Compose setup | | `SKIP_DOCKER` | (empty) | Skip Docker Compose if set | -| `POSTGRES_DATA_VOLUME` | (empty) | Path for persistent storage | +| `POSTGRES_DATA_VOLUME` | (empty) | Path for PostgreSQL persistent storage | +| `CLICKHOUSE_DATA_VOLUME` | (empty) | Path for ClickHouse persistent storage | | `REPLICATOR_IMAGE` | `ramsup/etl-replicator:latest` | Default replicator image | PostgreSQL 18+ containers store data under `/var/lib/postgresql//data`, so the Docker Compose setup mounts the parent `/var/lib/postgresql` directory to keep upgrades compatible. +The same Docker Compose stack also starts ClickHouse on `http://localhost:8123` by default, which is enough for local destination development and ClickHouse integration tests. + ### Manual Setup If you prefer manual setup or have an existing PostgreSQL instance: @@ -369,6 +376,18 @@ Iceberg destination tests use local MinIO and Lakekeeper instances. The followin **Note:** Iceberg tests are only run when the `iceberg` and `test-utils` features are enabled. These use hardcoded local URLs and do not require environment variables. +#### ClickHouse Test Variables + +ClickHouse destination tests require a reachable ClickHouse HTTP endpoint: + +| Variable | Required | Description | +|----------|----------|-------------| +| `TESTS_CLICKHOUSE_URL` | **Yes** | ClickHouse HTTP URL (for example, `http://localhost:8123`) | +| `TESTS_CLICKHOUSE_USER` | **Yes** | ClickHouse user name (for the local Docker Compose setup, use `etl`) | +| `TESTS_CLICKHOUSE_PASSWORD` | No | ClickHouse password; for the local Docker Compose setup, use `etl` | + +**Note:** ClickHouse tests are only run when the `clickhouse` and `test-utils` features are enabled. Each test creates a unique database in ClickHouse and drops it automatically when the test finishes. The Docker Compose setup started by `./scripts/init.sh` is sufficient for these tests. + #### Test Output and Logging | Variable | Description | @@ -407,6 +426,11 @@ export TESTS_DATABASE_PASSWORD=postgres export TESTS_BIGQUERY_PROJECT_ID=your-gcp-project-id export TESTS_BIGQUERY_SA_KEY_PATH=/path/to/service-account-key.json +# ClickHouse test configuration (optional - only needed for ClickHouse tests) +export TESTS_CLICKHOUSE_URL=http://localhost:8123 +export TESTS_CLICKHOUSE_USER=etl +export TESTS_CLICKHOUSE_PASSWORD=etl + # Enable test output (optional) export ENABLE_TRACING=1 export RUST_LOG=info @@ -432,6 +456,11 @@ TESTS_DATABASE_PASSWORD=postgres TESTS_BIGQUERY_PROJECT_ID=your-gcp-project-id TESTS_BIGQUERY_SA_KEY_PATH=/path/to/service-account-key.json +# ClickHouse (optional - only for ClickHouse tests) +TESTS_CLICKHOUSE_URL=http://localhost:8123 +TESTS_CLICKHOUSE_USER=etl +TESTS_CLICKHOUSE_PASSWORD=etl + # Test output (optional) ENABLE_TRACING=1 RUST_LOG=info @@ -462,6 +491,9 @@ TESTS_DATABASE_HOST=localhost TESTS_DATABASE_PORT=5430 TESTS_DATABASE_USERNAME=p # Run tests with tracing output for debugging TESTS_DATABASE_HOST=localhost TESTS_DATABASE_PORT=5430 TESTS_DATABASE_USERNAME=postgres TESTS_DATABASE_PASSWORD=postgres ENABLE_TRACING=1 RUST_LOG=info cargo test -p etl-api --test tenants tenant_can_be_created -- --nocapture + +# Run the ClickHouse destination integration test against the local Docker Compose service +TESTS_DATABASE_HOST=localhost TESTS_DATABASE_PORT=5430 TESTS_DATABASE_USERNAME=postgres TESTS_DATABASE_PASSWORD=postgres TESTS_CLICKHOUSE_URL=http://localhost:8123 TESTS_CLICKHOUSE_USER=etl TESTS_CLICKHOUSE_PASSWORD=etl cargo test -p etl-destinations --features clickhouse,test-utils clickhouse_pipeline -- --nocapture ``` **Packages requiring `--features test-utils`:** diff --git a/etl-api/Cargo.toml b/etl-api/Cargo.toml index 893a69ecc..6e28d7397 100644 --- a/etl-api/Cargo.toml +++ b/etl-api/Cargo.toml @@ -31,7 +31,7 @@ configcat = { workspace = true } constant_time_eq = { workspace = true } etl = { workspace = true } etl-config = { workspace = true, features = ["utoipa", "supabase"] } -etl-destinations = { workspace = true, features = ["bigquery", "iceberg", "ducklake"] } +etl-destinations = { workspace = true, features = ["bigquery", "clickhouse", "ducklake", "iceberg"] } etl-postgres = { workspace = true, features = ["replication"] } etl-telemetry = { workspace = true } k8s-openapi = { workspace = true, features = ["latest"] } @@ -49,11 +49,12 @@ thiserror = { workspace = true } tokio = { workspace = true, features = ["rt-multi-thread", "macros"] } tracing = { workspace = true, default-features = false } tracing-actix-web = { workspace = true, features = ["emit_event_on_error"] } +url = { workspace = true } utoipa = { workspace = true, features = ["actix_extras"] } utoipa-swagger-ui = { workspace = true, features = ["actix-web"] } [dev-dependencies] -etl-destinations = { workspace = true, features = ["test-utils", "iceberg", "bigquery", "ducklake"] } +etl-destinations = { workspace = true, features = ["test-utils", "bigquery", "clickhouse", "ducklake", "iceberg"] } etl-postgres = { workspace = true, features = ["test-utils", "sqlx"] } insta = { workspace = true, features = ["json", "redactions"] } diff --git a/etl-api/src/configs/destination.rs b/etl-api/src/configs/destination.rs index 0eb0d6b3e..f27d215b7 100644 --- a/etl-api/src/configs/destination.rs +++ b/etl-api/src/configs/destination.rs @@ -4,6 +4,7 @@ use etl_config::{ }; use secrecy::ExposeSecret; use serde::{Deserialize, Serialize}; +use url::Url; use utoipa::ToSchema; use crate::configs::{ @@ -43,6 +44,23 @@ pub enum FullApiDestinationConfig { #[serde(skip_serializing_if = "Option::is_none")] connection_pool_size: Option, }, + #[serde(rename = "clickhouse")] + ClickHouse { + /// ClickHouse HTTP(S) endpoint URL. + #[schema(value_type = String, example = "http://test:8123")] + #[serde(deserialize_with = "crate::utils::trim_http_url")] + url: Url, + /// ClickHouse user name + #[schema(example = "foo")] + #[serde(deserialize_with = "crate::utils::trim_string")] + user: String, + /// ClickHouse password (omit for passwordless access) + password: Option, + /// ClickHouse target database + #[schema(example = "my_db")] + #[serde(deserialize_with = "crate::utils::trim_string")] + database: String, + }, Iceberg { #[serde(flatten)] config: FullApiIcebergConfig, @@ -142,6 +160,9 @@ impl From for FullApiDestinationConfig { max_staleness_mins, connection_pool_size: Some(connection_pool_size), }, + StoredDestinationConfig::ClickHouse { url, user, password, database } => { + Self::ClickHouse { url, user, password, database } + } StoredDestinationConfig::Iceberg { config } => match config { StoredIcebergConfig::Supabase { project_ref, @@ -222,6 +243,12 @@ pub enum StoredDestinationConfig { max_staleness_mins: Option, connection_pool_size: usize, }, + ClickHouse { + url: Url, + user: String, + password: Option, + database: String, + }, Iceberg { config: StoredIcebergConfig, }, @@ -258,6 +285,12 @@ impl StoredDestinationConfig { max_staleness_mins, connection_pool_size, }, + Self::ClickHouse { url, user, password, database } => DestinationConfig::ClickHouse { + url, + user, + password: password.map(Into::into), + database, + }, Self::Iceberg { config } => match config { StoredIcebergConfig::Supabase { project_ref, @@ -346,6 +379,9 @@ impl From for StoredDestinationConfig { connection_pool_size: connection_pool_size .unwrap_or(DestinationConfig::DEFAULT_CONNECTION_POOL_SIZE), }, + FullApiDestinationConfig::ClickHouse { url, user, password, database } => { + Self::ClickHouse { url, user, password, database } + } FullApiDestinationConfig::Iceberg { config } => match config { FullApiIcebergConfig::Supabase { project_ref, @@ -441,6 +477,18 @@ impl Encrypt for StoredDestinationConfig { connection_pool_size, }) } + Self::ClickHouse { url, user, password, database } => { + let encrypted_password = password + .map(|p| encrypt_text(p.expose_secret().to_owned(), encryption_key)) + .transpose()?; + + Ok(EncryptedStoredDestinationConfig::ClickHouse { + url, + user, + password: encrypted_password, + database, + }) + } Self::Iceberg { config } => match config { StoredIcebergConfig::Supabase { project_ref, @@ -550,6 +598,12 @@ pub enum EncryptedStoredDestinationConfig { #[serde(default = "default_connection_pool_size")] connection_pool_size: usize, }, + ClickHouse { + url: Url, + user: String, + password: Option, + database: String, + }, Iceberg { #[serde(flatten)] config: EncryptedStoredIcebergConfig, @@ -667,6 +721,14 @@ impl Decrypt for EncryptedStoredDestinationConfig { }) } }, + EncryptedStoredDestinationConfig::ClickHouse { url, user, password, database } => { + let password = password + .map(|p| decrypt_text(p, encryption_key)) + .transpose()? + .map(SerializableSecretString::from); + + Ok(StoredDestinationConfig::ClickHouse { url, user, password, database }) + } Self::Ducklake { catalog_url, data_path, @@ -976,6 +1038,126 @@ mod tests { } } + #[test] + fn stored_destination_config_encryption_decryption_clickhouse() { + let config = StoredDestinationConfig::ClickHouse { + url: Url::parse("https://example.com:8443").unwrap(), + user: "etl".to_owned(), + password: Some(SerializableSecretString::from("secret".to_owned())), + database: "analytics".to_owned(), + }; + + let key = EncryptionKey { id: 1, key: generate_random_key::<32>().unwrap() }; + + let encrypted = config.clone().encrypt(&key).unwrap(); + let decrypted = encrypted.decrypt(&key).unwrap(); + + match (config, decrypted) { + ( + StoredDestinationConfig::ClickHouse { + url: u1, + user: user1, + password: p1, + database: d1, + }, + StoredDestinationConfig::ClickHouse { + url: u2, + user: user2, + password: p2, + database: d2, + }, + ) => { + assert_eq!(u1, u2); + assert_eq!(user1, user2); + assert_eq!(d1, d2); + assert_eq!( + p1.as_ref().map(|value| value.expose_secret()), + p2.as_ref().map(|value| value.expose_secret()) + ); + } + _ => panic!("Config types don't match"), + } + } + + #[test] + fn full_api_destination_config_conversion_clickhouse() { + let full_config = FullApiDestinationConfig::ClickHouse { + url: Url::parse("https://example.com:8443").unwrap(), + user: "etl".to_owned(), + password: Some(SerializableSecretString::from("secret".to_owned())), + database: "analytics".to_owned(), + }; + + let stored: StoredDestinationConfig = full_config.clone().into(); + let back_to_full: FullApiDestinationConfig = stored.into(); + + match (full_config, back_to_full) { + ( + FullApiDestinationConfig::ClickHouse { + url: u1, + user: user1, + password: p1, + database: d1, + }, + FullApiDestinationConfig::ClickHouse { + url: u2, + user: user2, + password: p2, + database: d2, + }, + ) => { + assert_eq!(u1, u2); + assert_eq!(user1, user2); + assert_eq!(d1, d2); + assert_eq!( + p1.as_ref().map(|value| value.expose_secret()), + p2.as_ref().map(|value| value.expose_secret()) + ); + } + _ => panic!("Config types don't match"), + } + } + + #[test] + fn full_api_destination_config_deserializes_clickhouse_url() { + let json = r#" + { + "clickhouse": { + "url": " https://example.com:8443 ", + "user": "etl", + "database": "analytics" + } + } + "#; + + let deserialized: FullApiDestinationConfig = serde_json::from_str(json).unwrap(); + match deserialized { + FullApiDestinationConfig::ClickHouse { url, user, password, database } => { + assert_eq!(url.as_str(), "https://example.com:8443/"); + assert_eq!(user, "etl"); + assert!(password.is_none()); + assert_eq!(database, "analytics"); + } + _ => panic!("Deserialization failed or variant mismatch"), + } + } + + #[test] + fn full_api_destination_config_rejects_non_http_clickhouse_url() { + let json = r#" + { + "clickhouse": { + "url": "ftp://example.com/data", + "user": "etl", + "database": "analytics" + } + } + "#; + + let error = serde_json::from_str::(json).unwrap_err(); + assert!(error.to_string().contains("url must use http or https scheme")); + } + #[test] fn full_api_destination_config_conversion_bigquery() { let full_config = FullApiDestinationConfig::BigQuery { diff --git a/etl-api/src/k8s/base.rs b/etl-api/src/k8s/base.rs index 2763680b9..3bfaf4014 100644 --- a/etl-api/src/k8s/base.rs +++ b/etl-api/src/k8s/base.rs @@ -45,6 +45,12 @@ pub enum DestinationType { BigQuery, /// Apache Iceberg destination. Iceberg, + /// ClickHouse destination. + ClickHouse { + /// Whether the StatefulSet must reference the ClickHouse password + /// secret. + password_secret_required: bool, + }, /// DuckLake destination. Ducklake, } @@ -55,6 +61,9 @@ impl From<&StoredDestinationConfig> for DestinationType { match value { StoredDestinationConfig::BigQuery { .. } => DestinationType::BigQuery, StoredDestinationConfig::Iceberg { .. } => DestinationType::Iceberg, + StoredDestinationConfig::ClickHouse { password, .. } => { + DestinationType::ClickHouse { password_secret_required: password.is_some() } + } StoredDestinationConfig::Ducklake { .. } => DestinationType::Ducklake, } } @@ -140,6 +149,13 @@ pub trait K8sClient: Send + Sync { bq_service_account_key: &str, ) -> Result<(), K8sError>; + /// Creates or updates the ClickHouse password secret for a replicator. + async fn create_or_update_clickhouse_secret( + &self, + prefix: &str, + password: Option<&str>, + ) -> Result<(), K8sError>; + /// Creates or updates the Iceberg credentials secret for a replicator. /// /// The secret contains the catalog token, S3 access key ID, and S3 secret @@ -168,6 +184,11 @@ pub trait K8sClient: Send + Sync { /// Does nothing if the secret does not exist. async fn delete_postgres_secret(&self, prefix: &str) -> Result<(), K8sError>; + /// Deletes the ClickHouse credentials for a replicator. + /// + /// Does nothing if the secret does not exist. + async fn delete_clickhouse_secret(&self, prefix: &str) -> Result<(), K8sError>; + /// Deletes the BigQuery service account secret for a replicator. /// /// Does nothing if the secret does not exist. diff --git a/etl-api/src/k8s/cache.rs b/etl-api/src/k8s/cache.rs index 10dfe95b8..f20a4e4a6 100644 --- a/etl-api/src/k8s/cache.rs +++ b/etl-api/src/k8s/cache.rs @@ -167,6 +167,14 @@ mod tests { Ok(()) } + async fn create_or_update_clickhouse_secret( + &self, + _prefix: &str, + _password: Option<&str>, + ) -> Result<(), K8sError> { + Ok(()) + } + async fn create_or_update_ducklake_secret( &self, _prefix: &str, @@ -180,6 +188,10 @@ mod tests { Ok(()) } + async fn delete_clickhouse_secret(&self, _prefix: &str) -> Result<(), K8sError> { + Ok(()) + } + async fn delete_bigquery_secret(&self, _prefix: &str) -> Result<(), K8sError> { Ok(()) } diff --git a/etl-api/src/k8s/core.rs b/etl-api/src/k8s/core.rs index 33f717476..603f464ce 100644 --- a/etl-api/src/k8s/core.rs +++ b/etl-api/src/k8s/core.rs @@ -50,6 +50,13 @@ pub enum Secrets { /// Google Cloud service account key JSON for BigQuery authentication. big_query_service_account_key: String, }, + /// Credentials for ClickHouse destinations. + ClickHouse { + /// PostgreSQL source database password. + postgres_password: String, + /// Clickhouse password + password: Option, + }, /// Credentials for Iceberg destinations. Iceberg { /// PostgreSQL source database password. @@ -234,6 +241,10 @@ fn build_secrets_from_configs( s3_access_key_id: s3_access_key_id.expose_secret().to_owned(), s3_secret_access_key: s3_secret_access_key.expose_secret().to_owned(), }, + StoredDestinationConfig::ClickHouse { password, .. } => Secrets::ClickHouse { + postgres_password, + password: password.as_ref().map(|p| p.expose_secret().to_owned()), + }, StoredDestinationConfig::Ducklake { s3_access_key_id, s3_secret_access_key, .. } => { Secrets::Ducklake { postgres_password, @@ -314,6 +325,14 @@ async fn create_or_update_dynamic_replicator_secrets( ) .await?; } + Secrets::ClickHouse { postgres_password, password } => { + k8s_client.create_or_update_postgres_secret(prefix, &postgres_password).await?; + if let Some(password) = password.as_deref() { + k8s_client.create_or_update_clickhouse_secret(prefix, Some(password)).await?; + } else { + k8s_client.delete_clickhouse_secret(prefix).await?; + } + } Secrets::Ducklake { postgres_password, s3_access_key_id, s3_secret_access_key } => { k8s_client.create_or_update_postgres_secret(prefix, &postgres_password).await?; if let (Some(s3_access_key_id), Some(s3_secret_access_key)) = @@ -405,14 +424,12 @@ async fn delete_dynamic_replicator_secrets( ) -> Result<(), K8sCoreError> { k8s_client.delete_postgres_secret(prefix).await?; - // Although it won't happen that there are both bq and iceberg secrets at the - // same time we delete them both here because the state in the db might not - // be the same as that running in the k8s cluster. E.g. if a pipeline is - // updated from bq to iceberg or vice-versa then there's a risk of wrong - // secret type being attempted for deletion which might leave the actual - // secret behind. So for simplicty we just delete both kinds of secrets. The - // one which doesn't exist will be safely ignored. + // Delete all destination-specific secret types unconditionally. Only one will + // exist at a time, but if a pipeline's destination was changed (e.g. BigQuery → + // ClickHouse) the old secret type might still be present. Deleting a + // non-existent secret is a safe no-op. k8s_client.delete_bigquery_secret(prefix).await?; + k8s_client.delete_clickhouse_secret(prefix).await?; k8s_client.delete_iceberg_secret(prefix).await?; k8s_client.delete_ducklake_secret(prefix).await?; @@ -473,6 +490,25 @@ mod tests { } } + fn source_config_with_password() -> StoredSourceConfig { + StoredSourceConfig { + host: "localhost".to_owned(), + port: 5432, + name: "postgres".to_owned(), + username: "postgres".to_owned(), + password: Some(SerializableSecretString::from("password".to_owned())), + } + } + + fn clickhouse_destination_config(password: Option<&str>) -> StoredDestinationConfig { + StoredDestinationConfig::ClickHouse { + url: "http://localhost:8123".parse().unwrap(), + user: "default".to_owned(), + password: password.map(ToOwned::to_owned).map(SerializableSecretString::from), + database: "default".to_owned(), + } + } + #[async_trait] impl K8sClient for RecordingK8sClient { async fn create_or_update_postgres_secret( @@ -502,6 +538,18 @@ mod tests { Ok(()) } + async fn create_or_update_clickhouse_secret( + &self, + prefix: &str, + password: Option<&str>, + ) -> Result<(), K8sError> { + self.calls + .lock() + .unwrap() + .push(format!("clickhouse:{prefix}:{}", password.unwrap_or(""))); + Ok(()) + } + async fn create_or_update_ducklake_secret( &self, prefix: &str, @@ -527,6 +575,11 @@ mod tests { Ok(()) } + async fn delete_clickhouse_secret(&self, prefix: &str) -> Result<(), K8sError> { + self.calls.lock().unwrap().push(format!("delete-clickhouse:{prefix}")); + Ok(()) + } + async fn delete_ducklake_secret(&self, prefix: &str) -> Result<(), K8sError> { self.calls.lock().unwrap().push(format!("delete-ducklake:{prefix}")); Ok(()) @@ -582,6 +635,44 @@ mod tests { assert!(is_active); } + #[tokio::test] + async fn clickhouse_with_password_creates_password_secret() { + let source_config = source_config_with_password(); + let destination_config = clickhouse_destination_config(Some("clickhouse-password")); + + let secrets = build_secrets_from_configs(&source_config, &destination_config); + let client = RecordingK8sClient::default(); + + create_or_update_dynamic_replicator_secrets(&client, "tenant-42", secrets).await.unwrap(); + + assert_eq!( + client.calls(), + vec![ + "postgres:tenant-42:password".to_owned(), + "clickhouse:tenant-42:clickhouse-password".to_owned(), + ] + ); + } + + #[tokio::test] + async fn passwordless_clickhouse_deletes_any_stale_password_secret() { + let source_config = source_config_with_password(); + let destination_config = clickhouse_destination_config(None); + + let secrets = build_secrets_from_configs(&source_config, &destination_config); + let client = RecordingK8sClient::default(); + + create_or_update_dynamic_replicator_secrets(&client, "tenant-42", secrets).await.unwrap(); + + assert_eq!( + client.calls(), + vec![ + "postgres:tenant-42:password".to_owned(), + "delete-clickhouse:tenant-42".to_owned(), + ] + ); + } + #[tokio::test] async fn ducklake_creates_postgres_and_s3_secrets() { let source_config = StoredSourceConfig { diff --git a/etl-api/src/k8s/http.rs b/etl-api/src/k8s/http.rs index 6f1015258..dbb5e2fbb 100644 --- a/etl-api/src/k8s/http.rs +++ b/etl-api/src/k8s/http.rs @@ -1,10 +1,15 @@ +use std::collections::BTreeMap; + use async_trait::async_trait; use base64::{Engine, prelude::BASE64_STANDARD}; use chrono::Utc; use etl_config::Environment; -use k8s_openapi::api::{ - apps::v1::StatefulSet, - core::v1::{ConfigMap, Pod, Secret}, +use k8s_openapi::{ + api::{ + apps::v1::StatefulSet, + core::v1::{ConfigMap, Pod, Secret}, + }, + apimachinery::pkg::apis::meta::v1::ObjectMeta, }; use kube::{ Client, @@ -21,6 +26,10 @@ use crate::{ /// Secret name suffix for the BigQuery service account key. const BQ_SECRET_NAME_SUFFIX: &str = "bq-service-account-key"; +/// Secret name suffix for the ClickHouse password. +const CLICKHOUSE_SECRET_NAME_SUFFIX: &str = "clickhouse-password"; +/// Name of the password in the ClickHouse secret and its reference. +const CLICKHOUSE_PASSWORD_NAME: &str = "clickhouse-password"; /// Name of the service account key in the BigQuery secret and its reference. const BQ_SERVICE_ACCOUNT_KEY_NAME: &str = "service-account-key"; /// Secret name suffix for iceberg secrets (includes catalog token, @@ -333,6 +342,33 @@ impl K8sClient for HttpK8sClient { Ok(()) } + async fn create_or_update_clickhouse_secret( + &self, + prefix: &str, + password: Option<&str>, + ) -> Result<(), K8sError> { + debug!("patching clickhouse secret"); + + if let Some(password) = password { + let clickhouse_secret_name = create_clickhouse_secret_name(prefix); + let replicator_app_name = create_replicator_app_name(prefix); + let secret = create_clickhouse_password_secret( + &clickhouse_secret_name, + &replicator_app_name, + password, + ); + + // We are forcing the update since we are the field manager that should own the + // fields. If there is an override (likely during an incident or + // SREs intervention), we want to override their changes. The API + // database is the source of truth for credentials. + let pp = PatchParams::apply(&clickhouse_secret_name).force(); + self.secrets_api.patch(&clickhouse_secret_name, &pp, &Patch::Apply(secret)).await?; + } + + Ok(()) + } + async fn create_or_update_iceberg_secret( &self, prefix: &str, @@ -406,6 +442,18 @@ impl K8sClient for HttpK8sClient { Ok(()) } + async fn delete_clickhouse_secret(&self, prefix: &str) -> Result<(), K8sError> { + debug!("deleting clickhouse secret"); + + let clickhouse_secret_name = create_clickhouse_secret_name(prefix); + let dp = DeleteParams::default(); + Self::handle_delete_with_404_ignore( + self.secrets_api.delete(&clickhouse_secret_name, &dp).await, + )?; + + Ok(()) + } + async fn delete_bigquery_secret(&self, prefix: &str) -> Result<(), K8sError> { debug!("deleting bq secret"); @@ -611,6 +659,10 @@ fn create_iceberg_secret_name(prefix: &str) -> String { format!("{prefix}-{ICEBERG_SECRET_NAME_SUFFIX}") } +fn create_clickhouse_secret_name(prefix: &str) -> String { + format!("{prefix}-{CLICKHOUSE_SECRET_NAME_SUFFIX}") +} + fn create_ducklake_secret_name(prefix: &str) -> String { format!("{prefix}-{DUCKLAKE_SECRET_NAME_SUFFIX}") } @@ -662,6 +714,30 @@ fn create_postgres_secret_json( }) } +fn create_clickhouse_password_secret( + secret_name: &str, + replicator_app_name: &str, + clickhouse_password: &str, +) -> Secret { + Secret { + metadata: ObjectMeta { + name: Some(secret_name.to_owned()), + namespace: Some(DATA_PLANE_NAMESPACE.to_owned()), + labels: Some(BTreeMap::from([ + ("etl.supabase.com/app-name".to_owned(), replicator_app_name.to_owned()), + ("etl.supabase.com/app-type".to_owned(), REPLICATOR_APP_LABEL.to_owned()), + ])), + ..ObjectMeta::default() + }, + type_: Some("Opaque".to_owned()), + string_data: Some(BTreeMap::from([( + CLICKHOUSE_PASSWORD_NAME.to_owned(), + clickhouse_password.to_owned(), + )])), + ..Secret::default() + } +} + fn create_bq_service_account_key_secret_json( secret_name: &str, replicator_app_name: &str, @@ -841,6 +917,19 @@ fn create_container_environment_json( let bq_secret_env_var_json = create_bq_secret_env_var_json(&bq_secret_name); container_environment.push(bq_secret_env_var_json); } + DestinationType::ClickHouse { password_secret_required } => { + let postgres_secret_name = create_postgres_secret_name(prefix); + let postgres_secret_env_var_json = + create_postgres_secret_env_var_json(&postgres_secret_name); + container_environment.push(postgres_secret_env_var_json); + + if password_secret_required { + let clickhouse_secret_name = create_clickhouse_secret_name(prefix); + let clickhouse_secret_env_var_json = + create_clickhouse_secret_env_var_json(&clickhouse_secret_name); + container_environment.push(clickhouse_secret_env_var_json); + } + } DestinationType::Iceberg => { let postgres_secret_name = create_postgres_secret_name(prefix); let postgres_secret_env_var_json = @@ -1022,6 +1111,18 @@ fn create_bq_secret_env_var_json(bq_secret_name: &str) -> serde_json::Value { }) } +fn create_clickhouse_secret_env_var_json(clickhouse_secret_name: &str) -> serde_json::Value { + json!({ + "name": "APP_DESTINATION__CLICKHOUSE__PASSWORD", + "valueFrom": { + "secretKeyRef": { + "name": clickhouse_secret_name, + "key": CLICKHOUSE_PASSWORD_NAME + } + } + }) +} + fn create_iceberg_catlog_token_env_var_json(iceberg_secret_name: &str) -> serde_json::Value { json!({ "name": "APP_DESTINATION__ICEBERG__SUPABASE__CATALOG_TOKEN", @@ -1207,6 +1308,15 @@ mod tests { format!("{tenant_id}-{replicator_id}") } + fn container_environment_has_var( + container_environment: &[serde_json::Value], + name: &str, + ) -> bool { + container_environment + .iter() + .any(|entry| entry.get("name").and_then(serde_json::Value::as_str) == Some(name)) + } + #[test] fn test_replicator_resource_config_uses_environment_defaults() { let prod = ReplicatorResourceConfig::load(&Environment::Prod).unwrap(); @@ -1546,6 +1656,52 @@ mod tests { assert_json_snapshot!(container_environment); } + #[test] + fn clickhouse_with_password_references_password_secret() { + let prefix = create_k8s_object_prefix(TENANT_ID, 42); + let replicator_image = "ramsup/etl-replicator:2a41356af735f891de37d71c0e1a62864fe4630e"; + + let container_environment = create_container_environment_json( + &prefix, + &Environment::Dev, + replicator_image, + DestinationType::ClickHouse { password_secret_required: true }, + LogLevel::Info, + ); + + assert!(container_environment_has_var( + &container_environment, + "APP_PIPELINE__PG_CONNECTION__PASSWORD", + )); + assert!(container_environment_has_var( + &container_environment, + "APP_DESTINATION__CLICKHOUSE__PASSWORD", + )); + } + + #[test] + fn passwordless_clickhouse_does_not_reference_missing_password_secret() { + let prefix = create_k8s_object_prefix(TENANT_ID, 42); + let replicator_image = "ramsup/etl-replicator:2a41356af735f891de37d71c0e1a62864fe4630e"; + + let container_environment = create_container_environment_json( + &prefix, + &Environment::Dev, + replicator_image, + DestinationType::ClickHouse { password_secret_required: false }, + LogLevel::Info, + ); + + assert!(container_environment_has_var( + &container_environment, + "APP_PIPELINE__PG_CONNECTION__PASSWORD", + )); + assert!(!container_environment_has_var( + &container_environment, + "APP_DESTINATION__CLICKHOUSE__PASSWORD", + )); + } + #[test] fn test_create_node_selector() { let node_selector = create_node_selector_json(&Environment::Dev); diff --git a/etl-api/src/utils.rs b/etl-api/src/utils.rs index 4b84baa8c..2f9ac0aa0 100644 --- a/etl-api/src/utils.rs +++ b/etl-api/src/utils.rs @@ -1,6 +1,7 @@ use etl_config::SerializableSecretString; use rand::Rng; -use serde::{Deserialize, Deserializer}; +use serde::{Deserialize, Deserializer, de::Error as _}; +use url::Url; /// Deserializes a string and trims leading and trailing whitespace. pub fn trim_string<'de, D>(deserializer: D) -> Result @@ -33,6 +34,20 @@ where Ok(opt.map(|s| SerializableSecretString::from(s.trim().to_owned()))) } +/// Deserializes an HTTP(S) URL string, trimming whitespace. +pub fn trim_http_url<'de, D>(deserializer: D) -> Result +where + D: Deserializer<'de>, +{ + let s = String::deserialize(deserializer)?; + let url = Url::parse(s.trim()).map_err(D::Error::custom)?; + + match url.scheme() { + "http" | "https" => Ok(url), + scheme => Err(D::Error::custom(format!("url must use http or https scheme, got {scheme}"))), + } +} + /// Generates a random alphabetic string of length `len`. pub fn generate_random_alpha_str(len: usize) -> String { let chars = [ @@ -290,4 +305,30 @@ mod tests { let result: TestStruct = serde_json::from_str(json).unwrap(); assert_eq!(result.value, Some("".to_owned())); } + + #[test] + fn trim_http_url_trims_and_parses() { + #[derive(Debug, Deserialize)] + struct TestStruct { + #[serde(rename = "value", deserialize_with = "trim_http_url")] + _value: Url, + } + + let json = r#"{"value": " https://example.com:8443/path "}"#; + let result: TestStruct = serde_json::from_str(json).unwrap(); + assert_eq!(result._value.as_str(), "https://example.com:8443/path"); + } + + #[test] + fn trim_http_url_rejects_non_http_scheme() { + #[derive(Debug, Deserialize)] + struct TestStruct { + #[serde(rename = "value", deserialize_with = "trim_http_url")] + _value: Url, + } + + let json = r#"{"value": "ftp://example.com/data"}"#; + let error = serde_json::from_str::(json).unwrap_err(); + assert!(error.to_string().contains("url must use http or https scheme")); + } } diff --git a/etl-api/src/validation/validators.rs b/etl-api/src/validation/validators.rs index d9406119c..eb82a980f 100644 --- a/etl-api/src/validation/validators.rs +++ b/etl-api/src/validation/validators.rs @@ -4,14 +4,16 @@ use std::collections::HashMap; use async_trait::async_trait; use etl::store::both::memory::MemoryStore; -use etl_config::parse_ducklake_url; +use etl_config::{SerializableSecretString, parse_ducklake_url}; use etl_destinations::{ bigquery::BigQueryClient, + clickhouse::ClickHouseClient, ducklake::{DuckLakeDestination, S3Config as DucklakeS3Config}, iceberg::{IcebergClient, S3_ACCESS_KEY_ID, S3_ENDPOINT, S3_SECRET_ACCESS_KEY}, }; use secrecy::ExposeSecret; use sqlx::FromRow; +use url::Url; use super::{ValidationContext, ValidationError, ValidationFailure, Validator}; use crate::configs::{ @@ -630,6 +632,51 @@ impl Validator for BigQueryValidator { } } +/// Validates Clickhouse destination connectivity and dataset accessibility. +#[derive(Debug)] +struct ClickHouseValidator { + url: Url, + user: String, + password: Option, + database: String, +} + +impl ClickHouseValidator { + fn new( + url: Url, + user: String, + password: Option, + database: String, + ) -> Self { + Self { url, user, password, database } + } +} + +#[async_trait] +impl Validator for ClickHouseValidator { + async fn validate( + &self, + _ctx: &ValidationContext, + ) -> Result, ValidationError> { + let client = ClickHouseClient::new( + self.url.clone(), + self.user.clone(), + self.password.as_ref().map(|password| password.expose_secret().to_owned()), + self.database.clone(), + ); + match client.validate_connectivity().await { + Ok(_) => Ok(Vec::new()), + Err(_) => Ok(vec![ValidationFailure::critical( + "ClickHouse Connection Failed", + "Unable to create clickhouse client.\n\nPlease verify:\n(1) The url is valid and \ + accessible\n(2) The username is correct\n(3) You set the right password\n(4) You \ + set the right database name + ", + )]), + } + } +} + /// Validates Iceberg destination connectivity. #[derive(Debug)] struct IcebergValidator { @@ -877,6 +924,15 @@ impl Validator for DestinationValidator { ); validator.validate(ctx).await } + FullApiDestinationConfig::ClickHouse { url, user, password, database } => { + let validator = ClickHouseValidator::new( + url.clone(), + user.clone(), + password.clone(), + database.clone(), + ); + validator.validate(ctx).await + } FullApiDestinationConfig::Iceberg { config } => { let validator = IcebergValidator::new(config.clone()); validator.validate(ctx).await diff --git a/etl-api/tests/support/k8s_client.rs b/etl-api/tests/support/k8s_client.rs index 8cb4db26a..fe87647a4 100644 --- a/etl-api/tests/support/k8s_client.rs +++ b/etl-api/tests/support/k8s_client.rs @@ -92,6 +92,14 @@ impl K8sClient for MockK8sClient { Ok(()) } + async fn create_or_update_clickhouse_secret( + &self, + _prefix: &str, + _password: Option<&str>, + ) -> Result<(), K8sError> { + Ok(()) + } + async fn create_or_update_iceberg_secret( &self, _prefix: &str, @@ -116,6 +124,10 @@ impl K8sClient for MockK8sClient { Ok(()) } + async fn delete_clickhouse_secret(&self, _prefix: &str) -> Result<(), K8sError> { + Ok(()) + } + async fn delete_bigquery_secret(&self, _prefix: &str) -> Result<(), K8sError> { Ok(()) } diff --git a/etl-config/src/shared/destination.rs b/etl-config/src/shared/destination.rs index 1e8255fad..b3cd5d3c0 100644 --- a/etl-config/src/shared/destination.rs +++ b/etl-config/src/shared/destination.rs @@ -1,5 +1,6 @@ use secrecy::SecretString; use serde::{Deserialize, Serialize}; +use url::Url; const fn default_connection_pool_size() -> usize { DestinationConfig::DEFAULT_CONNECTION_POOL_SIZE @@ -47,6 +48,17 @@ pub enum DestinationConfig { #[serde(default = "default_connection_pool_size")] connection_pool_size: usize, }, + #[serde(rename = "clickhouse")] + ClickHouse { + /// ClickHouse HTTP(S) endpoint URL. + url: Url, + /// ClickHouse user name + user: String, + /// ClickHouse password (omit for passwordless access) + password: Option, + /// ClickHouse target database + database: String, + }, Iceberg { #[serde(flatten)] config: IcebergConfig, @@ -232,6 +244,15 @@ pub enum DestinationConfigWithoutSecrets { #[serde(default = "default_connection_pool_size")] connection_pool_size: usize, }, + #[serde(rename = "clickhouse")] + ClickHouse { + /// ClickHouse HTTP(S) endpoint URL. + url: Url, + /// ClickHouse user name + user: String, + /// ClickHouse target database + database: String, + }, Iceberg { #[serde(flatten)] config: IcebergConfigWithoutSecrets, @@ -278,6 +299,9 @@ impl From for DestinationConfigWithoutSecrets { max_staleness_mins, connection_pool_size, }, + DestinationConfig::ClickHouse { url, user, database, .. } => { + DestinationConfigWithoutSecrets::ClickHouse { url, user, database } + } DestinationConfig::Iceberg { config } => { DestinationConfigWithoutSecrets::Iceberg { config: config.into() } } diff --git a/etl-destinations/Cargo.toml b/etl-destinations/Cargo.toml index 6b1425332..2bd32b9d6 100644 --- a/etl-destinations/Cargo.toml +++ b/etl-destinations/Cargo.toml @@ -49,6 +49,16 @@ iceberg = [ "dep:serde", "dep:serde_json", ] +clickhouse = [ + "dep:clickhouse", + "dep:metrics", + "dep:tracing", + "dep:tokio", + "dep:serde", + "dep:futures", + "dep:parking_lot", + "dep:url", +] egress = ["etl/egress"] # We assume that `test-utils` is always used in conjunction with `bigquery` or `iceberg` thus we only # put here the extra dependencies needed. @@ -59,8 +69,10 @@ test-utils = ["dep:uuid"] arrow = { workspace = true, optional = true } async-trait = { workspace = true, optional = true } chrono = { workspace = true } +clickhouse = { workspace = true, optional = true, features = ["inserter", "lz4", "rustls-tls"] } duckdb = { workspace = true, optional = true, features = ["bundled", "json", "parquet", "r2d2"] } etl = { workspace = true } +futures = { workspace = true, optional = true } gcp-bigquery-client = { workspace = true, optional = true, features = ["rust-tls", "aws-lc-rs"] } iceberg = { workspace = true, optional = true } iceberg-catalog-rest = { workspace = true, optional = true } @@ -94,7 +106,7 @@ etl-telemetry = { workspace = true } futures = { workspace = true } rand = { workspace = true, features = ["thread_rng"] } rustls = { workspace = true, features = ["aws-lc-rs", "logging"] } -serde = { workspace = true } +serde = { workspace = true, features = ["derive"] } serde_json = { workspace = true } tempfile = { workspace = true } tokio = { workspace = true, features = ["full"] } diff --git a/etl-destinations/src/clickhouse/client.rs b/etl-destinations/src/clickhouse/client.rs new file mode 100644 index 000000000..1dfdd3328 --- /dev/null +++ b/etl-destinations/src/clickhouse/client.rs @@ -0,0 +1,380 @@ +use std::{sync::Arc, time::Instant}; + +use clickhouse::Client; +use etl::{ + error::{ErrorKind, EtlResult}, + etl_error, +}; +use url::Url; + +use crate::clickhouse::{ + encoding::{ClickHouseValue, encode_to_row_binary}, + metrics::{ETL_CLICKHOUSE_DDL_DURATION_SECONDS, ETL_CLICKHOUSE_INSERT_DURATION_SECONDS}, + schema::{clickhouse_column_type, quote_identifier}, +}; + +/// Capacity of the internal write buffer used per INSERT statement. +/// +/// When this many bytes have been written to the buffer it is flushed to the +/// network (but the INSERT statement itself is not closed — that only happens +/// when `end()` is called or the `max_bytes_per_insert` limit is reached). +const BUFFERED_CAPACITY: usize = 256 * 1024; + +/// A ClickHouse table column returned from `system.columns`. +#[derive(Debug, Clone, PartialEq, Eq, clickhouse::Row, serde::Deserialize)] +pub(crate) struct ClickHouseTableColumn { + /// Column name. + pub(crate) name: String, + /// ClickHouse type string, for example `Int32` or `Nullable(String)`. + pub(crate) type_name: String, +} + +/// Returns the placement clause for an `ADD COLUMN` statement. +/// +/// `None` means the destination table has no user columns to anchor on, so +/// the new column goes at the front via `FIRST` (which still places it +/// before the trailing CDC columns). +fn add_column_placement_clause(after_column: Option<&str>) -> String { + match after_column { + Some(anchor) => format!("AFTER {}", quote_identifier(anchor)), + None => "FIRST".to_owned(), + } +} + +/// Builds the SQL used to add a column to a ClickHouse table. +fn build_add_column_sql( + table_name: &str, + column: &etl::types::ColumnSchema, + after_column: Option<&str>, +) -> String { + let col_type = clickhouse_column_type(column, true); + let table_name = quote_identifier(table_name); + let column_name = quote_identifier(&column.name); + let placement = add_column_placement_clause(after_column); + + format!( + "ALTER TABLE {table_name} ADD COLUMN IF NOT EXISTS {column_name} {col_type} {placement}" + ) +} + +/// Builds the SQL used to drop a column from a ClickHouse table. +fn build_drop_column_sql(table_name: &str, column_name: &str) -> String { + let table_name = quote_identifier(table_name); + let column_name = quote_identifier(column_name); + format!("ALTER TABLE {table_name} DROP COLUMN IF EXISTS {column_name}") +} + +/// Builds the SQL used to rename a column in a ClickHouse table. +fn build_rename_column_sql(table_name: &str, old_name: &str, new_name: &str) -> String { + let table_name = quote_identifier(table_name); + let old_name = quote_identifier(old_name); + let new_name = quote_identifier(new_name); + format!("ALTER TABLE {table_name} RENAME COLUMN IF EXISTS {old_name} TO {new_name}") +} + +/// Builds the SQL used to truncate a ClickHouse table. +fn build_truncate_table_sql(table_name: &str) -> String { + let table_name = quote_identifier(table_name); + format!("TRUNCATE TABLE IF EXISTS {table_name}") +} + +/// Builds the SQL used to insert RowBinary rows into a ClickHouse table. +fn build_insert_rows_sql(table_name: &str) -> String { + let table_name = quote_identifier(table_name); + format!("INSERT INTO {table_name} FORMAT RowBinary") +} + +/// Kind of DDL being executed; surfaces as a `kind` label on the +/// `etl_clickhouse_ddl_duration_seconds` histogram so per-operation latencies +/// can be distinguished (one-shot CREATE vs. online ALTER, etc.). +#[derive(Copy, Clone)] +pub(crate) enum DdlKind { + CreateTable, + AddColumn, + DropColumn, + RenameColumn, +} + +impl DdlKind { + fn as_label(self) -> &'static str { + match self { + DdlKind::CreateTable => "create_table", + DdlKind::AddColumn => "add_column", + DdlKind::DropColumn => "drop_column", + DdlKind::RenameColumn => "rename_column", + } + } +} + +/// High-level ClickHouse client used by [`super::core::ClickHouseDestination`]. +/// +/// Wraps a [`clickhouse::Client`] and exposes typed methods for DDL, +/// truncation, and RowBinary bulk inserts. +#[derive(Clone)] +pub struct ClickHouseClient { + inner: Arc, +} + +impl ClickHouseClient { + /// Creates a new [`ClickHouseClient`]. + /// + /// When `url` starts with `https://`, TLS is handled automatically by the + /// `rustls-tls` feature using webpki root certificates. + pub fn new( + url: Url, + user: impl Into, + password: Option, + database: impl Into, + ) -> Self { + let mut client = + Client::default().with_url(url.to_string()).with_user(user).with_database(database); + + if let Some(pw) = password { + client = client.with_password(pw); + } + + Self { inner: Arc::new(client) } + } + + /// Verifies that the ClickHouse server is reachable. + /// + /// Issues a `SELECT 1` round-trip; cheaper than any DDL or metadata + /// query and exercises the auth/transport path. Mirrors the Iceberg + /// destination's `validate_connectivity` so callers (notably the + /// `etl-api` validators) can treat the two destinations uniformly. + pub async fn validate_connectivity(&self) -> EtlResult<()> { + self.inner + .query("SELECT 1") + .fetch_one::() + .await + .map(|_| ()) + .map_err(|e| etl_error!(ErrorKind::Unknown, "ClickHouse connectivity check failed", e)) + } + + /// Executes a DDL statement (e.g. `CREATE TABLE IF NOT EXISTS …`) and + /// records its duration in the `etl_clickhouse_ddl_duration_seconds` + /// histogram labelled with the DDL `kind` and `table_name`. + pub(crate) async fn execute_ddl(&self, kind: DdlKind, sql: &str) -> EtlResult<()> { + let ddl_start = Instant::now(); + let result = self.inner.query(sql).execute().await.map_err(|e| { + etl_error!( + ErrorKind::Unknown, + "ClickHouse DDL failed", + format!("DDL execution failed: {e}") + ) + }); + metrics::histogram!( + ETL_CLICKHOUSE_DDL_DURATION_SECONDS, + "kind" => kind.as_label(), + ) + .record(ddl_start.elapsed().as_secs_f64()); + result + } + + /// Returns ClickHouse columns for a table in position order. + pub(crate) async fn table_columns( + &self, + table_name: &str, + ) -> EtlResult> { + self.inner + .query( + "SELECT name, type AS type_name FROM system.columns WHERE database = \ + currentDatabase() AND table = ? ORDER BY position", + ) + .bind(table_name) + .fetch_all::() + .await + .map_err(|e| { + etl_error!( + ErrorKind::Unknown, + "ClickHouse schema query failed", + format!("Failed to query columns for table '{table_name}': {e}") + ) + }) + } + + /// Adds a column to an existing ClickHouse table. + /// + /// New columns are always Nullable since ClickHouse cannot backfill + /// existing rows with a NOT NULL default. + /// + /// `after_column` controls placement: `Some(name)` inserts the new column + /// immediately AFTER `name`, `None` inserts it FIRST (used when the table + /// has no user columns yet). Either way the new column lands before the + /// trailing CDC columns (`cdc_operation`, `cdc_lsn`), which is required + /// because RowBinary encoding is positional. + pub(crate) async fn add_column( + &self, + table_name: &str, + column: &etl::types::ColumnSchema, + after_column: Option<&str>, + ) -> EtlResult<()> { + let sql = build_add_column_sql(table_name, column, after_column); + self.execute_ddl(DdlKind::AddColumn, &sql).await + } + + /// Drops a column from an existing ClickHouse table (idempotent). + pub(crate) async fn drop_column(&self, table_name: &str, column_name: &str) -> EtlResult<()> { + let sql = build_drop_column_sql(table_name, column_name); + self.execute_ddl(DdlKind::DropColumn, &sql).await + } + + /// Renames a column in an existing ClickHouse table (idempotent). + /// + /// `RENAME COLUMN IF EXISTS` makes the ALTER a server-side noop when the + /// old column is already absent, so the check and the rename happen in + /// one statement without a racy read-then-write. + pub(crate) async fn rename_column( + &self, + table_name: &str, + old_name: &str, + new_name: &str, + ) -> EtlResult<()> { + let sql = build_rename_column_sql(table_name, old_name, new_name); + self.execute_ddl(DdlKind::RenameColumn, &sql).await + } + + /// Executes `TRUNCATE TABLE IF EXISTS` for the supplied table. + pub(crate) async fn truncate_table(&self, table_name: &str) -> EtlResult<()> { + self.inner.query(&build_truncate_table_sql(table_name)).execute().await.map_err(|e| { + etl_error!( + ErrorKind::Unknown, + "ClickHouse truncate failed", + format!("Failed to truncate table '{table_name}': {e}") + ) + }) + } + + /// Inserts `rows` into `table_name` using the RowBinary format. + /// + /// Each element of `rows` is a complete, already-encoded row of + /// [`ClickHouseValue`]s in column order (user columns + CDC columns). + /// `nullable_flags` must have the same length as each row. + /// + /// When the accumulated uncompressed byte count reaches + /// `max_bytes_per_insert` the current INSERT statement is committed and + /// a new one is opened, keeping peak memory usage bounded for large + /// initial copies. + /// + /// The `source` label (`"copy"` or `"streaming"`) is attached to the + /// `etl_clickhouse_insert_duration_seconds` histogram recorded after each + /// committed INSERT statement. + pub(crate) async fn insert_rows( + &self, + table_name: &str, + rows: Vec>, + nullable_flags: &[bool], + max_bytes_per_insert: u64, + source: &'static str, + ) -> EtlResult<()> { + let sql = build_insert_rows_sql(table_name); + let mut rows = rows.into_iter().peekable(); + let mut row_buf = Vec::new(); + + while rows.peek().is_some() { + let mut insert = self + .inner + .insert_formatted_with(sql.clone()) + .buffered_with_capacity(BUFFERED_CAPACITY); + let mut bytes = 0u64; + let insert_start = Instant::now(); + + while bytes < max_bytes_per_insert { + let Some(row) = rows.next() else { break }; + row_buf.clear(); + encode_to_row_binary(row, nullable_flags, &mut row_buf)?; + insert.write_buffered(&row_buf); + bytes += row_buf.len() as u64; + } + + insert.end().await.map_err(|e| { + etl_error!( + ErrorKind::Unknown, + "ClickHouse insert flush failed", + format!("Failed to flush INSERT for '{table_name}': {e}") + ) + })?; + metrics::histogram!( + ETL_CLICKHOUSE_INSERT_DURATION_SECONDS, + "source" => source, + ) + .record(insert_start.elapsed().as_secs_f64()); + } + + Ok(()) + } +} + +#[cfg(test)] +mod tests { + use etl::types::{ColumnSchema, Type}; + + use super::*; + + fn column_schema(name: &str) -> ColumnSchema { + ColumnSchema { + name: name.to_owned(), + typ: Type::INT4, + modifier: -1, + ordinal_position: 1, + primary_key_ordinal_position: Some(1), + nullable: false, + } + } + + #[test] + fn add_column_sql_quotes_identifiers() { + let column = column_schema("new\"column"); + let sql = build_add_column_sql("table\"name", &column, Some("old\"column")); + + assert_eq!( + sql, + "ALTER TABLE \"table\"\"name\" ADD COLUMN IF NOT EXISTS \"new\"\"column\" \ + Nullable(Int32) AFTER \"old\"\"column\"" + ); + } + + #[test] + fn add_column_sql_uses_first_when_anchor_is_none() { + let column = column_schema("only_col"); + let sql = build_add_column_sql("test_table", &column, None); + + assert_eq!( + sql, + "ALTER TABLE \"test_table\" ADD COLUMN IF NOT EXISTS \"only_col\" Nullable(Int32) \ + FIRST" + ); + } + + #[test] + fn drop_column_sql_quotes_identifiers() { + let sql = build_drop_column_sql("table\"name", "old\"column"); + + assert_eq!(sql, "ALTER TABLE \"table\"\"name\" DROP COLUMN IF EXISTS \"old\"\"column\""); + } + + #[test] + fn rename_column_sql_quotes_identifiers() { + let sql = build_rename_column_sql("table\"name", "old\"column", "new\"column"); + + assert_eq!( + sql, + "ALTER TABLE \"table\"\"name\" RENAME COLUMN IF EXISTS \"old\"\"column\" TO \ + \"new\"\"column\"" + ); + } + + #[test] + fn truncate_table_sql_quotes_identifiers() { + let sql = build_truncate_table_sql("table\"name"); + + assert_eq!(sql, "TRUNCATE TABLE IF EXISTS \"table\"\"name\""); + } + + #[test] + fn insert_rows_sql_quotes_identifiers() { + let sql = build_insert_rows_sql("table\"name"); + + assert_eq!(sql, "INSERT INTO \"table\"\"name\" FORMAT RowBinary"); + } +} diff --git a/etl-destinations/src/clickhouse/core.rs b/etl-destinations/src/clickhouse/core.rs new file mode 100644 index 000000000..e9353f524 --- /dev/null +++ b/etl-destinations/src/clickhouse/core.rs @@ -0,0 +1,1032 @@ +use std::{collections::HashMap, sync::Arc}; + +use etl::{ + destination::{ + Destination, + async_result::{TruncateTableResult, WriteEventsResult, WriteTableRowsResult}, + }, + error::{ErrorKind, EtlResult}, + etl_error, + state::destination_metadata::{DestinationTableMetadata, DestinationTableSchemaStatus}, + store::{schema::SchemaStore, state::StateStore}, + types::{ + Cell, Event, IdentityType, OldTableRow, PgLsn, ReplicatedTableSchema, SchemaDiff, TableId, + TableRow, Type, UpdatedTableRow, is_array_type, + }, +}; +use parking_lot::RwLock; +use tokio::task::JoinSet; +use tracing::{debug, info, warn}; +use url::Url; + +use crate::{ + clickhouse::{ + client::{ClickHouseClient, ClickHouseTableColumn, DdlKind}, + encoding::{ClickHouseValue, cell_to_clickhouse_value}, + metrics::register_metrics, + schema::{CDC_LSN_COLUMN_NAME, CDC_OPERATION_COLUMN_NAME, build_create_table_sql}, + }, + table_name::try_stringify_table_name, +}; + +/// Postgres CDC operation kind. Written to the `cdc_operation` column as the +/// matching uppercase string (`"INSERT"`, `"UPDATE"`, `"DELETE"`) so downstream +/// consumers (ReplacingMergeTree dedup, materialized views, etc.) can filter +/// or branch on operation type. +#[derive(Copy, Clone)] +enum CdcOperation { + /// New row inserted on the source. + Insert, + /// Existing row updated on the source. Carries the post-update values. + Update, + /// Row deleted on the source. Carries pre-delete values for the PK + /// columns; non-PK columns are filled in by `expand_key_row` (NULL for + /// nullable columns, type-appropriate zero for non-nullable). + Delete, +} + +impl std::fmt::Display for CdcOperation { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + match self { + CdcOperation::Insert => write!(f, "INSERT"), + CdcOperation::Update => write!(f, "UPDATE"), + CdcOperation::Delete => write!(f, "DELETE"), + } + } +} + +/// A row pending insertion with its CDC metadata. +struct PendingRow { + /// CDC op kind, written into the `cdc_operation` column. + operation: CdcOperation, + /// Commit LSN of the source transaction, written into `cdc_lsn`. + lsn: PgLsn, + /// User column values in source schema order. The two CDC columns + /// (`cdc_operation`, `cdc_lsn`) are appended at encode time and are + /// not present here. + cells: Vec, +} + +/// Converts a Postgres LSN into the ClickHouse CDC LSN value. +fn cdc_lsn_to_clickhouse_value(lsn: PgLsn) -> ClickHouseValue { + ClickHouseValue::UInt64(u64::from(lsn)) +} + +/// Returns true if the ClickHouse type has an outer Nullable wrapper. +fn clickhouse_type_expects_nullable_marker(type_name: &str) -> bool { + type_name.starts_with("Nullable(") +} + +/// Returns expected ClickHouse column names for a replicated schema. +fn expected_clickhouse_column_names(schema: &ReplicatedTableSchema) -> Vec { + let mut names: Vec = + schema.column_schemas().map(|column| column.name.clone()).collect(); + names.push(CDC_OPERATION_COLUMN_NAME.to_owned()); + names.push(CDC_LSN_COLUMN_NAME.to_owned()); + names +} + +/// Derives RowBinary nullable flags from the actual ClickHouse table schema. +/// +/// RowBinary requires a leading null-marker byte before each `Nullable(T)` +/// column. The actual nullability of a ClickHouse column can drift from the +/// source Postgres column: `ALTER TABLE ADD COLUMN` forces the new column to +/// `Nullable(T)` regardless of the upstream `NOT NULL` constraint, because +/// ClickHouse cannot backfill a non-null default for existing rows. Deriving +/// flags from the destination schema therefore matches what ClickHouse expects +/// on the wire even after schema evolution. +/// +/// The column-count and column-order checks are an integrity guard: if the +/// destination has otherwise drifted from `ReplicatedTableSchema`, we surface +/// a `CorruptedTableSchema` error rather than emit misaligned RowBinary bytes. +fn nullable_flags_from_clickhouse_columns( + clickhouse_table_name: &str, + expected_column_names: &[String], + actual_columns: &[ClickHouseTableColumn], +) -> EtlResult> { + if actual_columns.len() != expected_column_names.len() { + return Err(etl_error!( + ErrorKind::CorruptedTableSchema, + "ClickHouse table schema does not match replicated schema", + format!( + "table '{}' has {} columns, but {} were expected", + clickhouse_table_name, + actual_columns.len(), + expected_column_names.len() + ) + )); + } + + let mut nullable_flags = Vec::with_capacity(actual_columns.len()); + for (index, (actual_column, expected_name)) in + actual_columns.iter().zip(expected_column_names).enumerate() + { + if actual_column.name != *expected_name { + return Err(etl_error!( + ErrorKind::CorruptedTableSchema, + "ClickHouse table schema does not match replicated schema", + format!( + "table '{}' column {} is named '{}', but '{}' was expected", + clickhouse_table_name, + index + 1, + actual_column.name, + expected_name + ) + )); + } + + nullable_flags.push(clickhouse_type_expects_nullable_marker(&actual_column.type_name)); + } + + Ok(nullable_flags.into()) +} + +/// Controls intermediate flushing inside a single `write_table_rows` / +/// `write_events` call. +/// +/// The upstream `BatchConfig::max_fill_ms` controls when `write_events` is +/// called; this limit prevents unbounded memory use for very large batches +/// (e.g. initial copy). +#[derive(Copy, Clone)] +pub struct ClickHouseInserterConfig { + /// Start a new INSERT after this many uncompressed bytes. Fixed cap + /// because incoming and outgoing buffers can both be near-full at once; + /// could be made tunable later if needed. + pub max_bytes_per_insert: u64, +} + +impl ClickHouseInserterConfig { + /// Default per-INSERT byte cap. 64 MiB lands in the upper end of + /// ClickHouse's recommended bulk-insert range (10k - 100k rows per + /// INSERT) for typical CDC payload widths. + /// + /// See . + pub const DEFAULT_MAX_BYTES_PER_INSERT: u64 = 64 * 1024 * 1024; +} + +impl Default for ClickHouseInserterConfig { + fn default() -> Self { + Self { max_bytes_per_insert: Self::DEFAULT_MAX_BYTES_PER_INSERT } + } +} + +/// CDC-capable ClickHouse destination that replicates Postgres tables. +/// +/// Uses append-only MergeTree tables with two CDC columns (`cdc_operation`, +/// `cdc_lsn`) appended to each row. Rows are encoded as RowBinary and sent via +/// `INSERT INTO "table" FORMAT RowBinary` -- no column-name header required. +#[derive(Clone)] +pub struct ClickHouseDestination { + /// HTTP client used for all DDL and RowBinary INSERT traffic. + client: ClickHouseClient, + /// Per-INSERT byte budget; gates intermediate flushes within a single + /// `write_table_rows` / `write_events` call. + inserter_config: ClickHouseInserterConfig, + /// Schema/state store used to persist destination table metadata + /// (Applying / Applied) and to look up replicated schemas. + store: Arc, + /// ClickHouse table name -> per-column nullable flags (in column order, + /// including the two trailing CDC columns which are always `false`). + /// + /// Populated lazily on first encounter of a table and consulted on the + /// hot insert path. `std::sync::RwLock` is sufficient: every critical + /// section is a brief in-memory map op with no `.await` inside, so the + /// async `tokio::sync::RwLock` would be needless overhead. + table_cache: Arc>>>, +} + +impl ClickHouseDestination +where + S: StateStore + SchemaStore + Send + Sync, +{ + /// Creates a new `ClickHouseDestination`. + /// + /// When using an `https://` URL, TLS is handled automatically by the `rustls-tls` + /// feature using webpki root certificates. + pub fn new( + url: Url, + user: impl Into, + password: Option, + database: impl Into, + inserter_config: ClickHouseInserterConfig, + store: S, + ) -> EtlResult { + register_metrics(); + Ok(Self { + client: ClickHouseClient::new(url, user, password, database), + inserter_config, + store: Arc::new(store), + table_cache: Arc::new(RwLock::new(HashMap::new())), + }) + } + + /// Creates a ClickHouse table for a never-before-seen `table_id`, + /// bracketing the DDL with `DestinationTableMetadata` writes so the + /// operation is crash-recoverable. + /// + /// Sequence: + /// 1. Persist `Applying` metadata (so a crash between this write and step 3 + /// leaves a marker that lets restart logic detect the interrupted + /// operation). + /// 2. Execute `CREATE TABLE IF NOT EXISTS` against ClickHouse. + /// 3. Persist `Applied` metadata. + /// + /// Recovery is handled by `ensure_table_exists`: on restart, an + /// `Applying` row signals that the previous run died mid-creation, so + /// it re-runs the idempotent DDL and transitions the metadata to + /// `Applied` itself. + async fn create_table_with_metadata( + &self, + table_id: TableId, + clickhouse_table_name: &str, + schema: &ReplicatedTableSchema, + snapshot_id: etl::types::SnapshotId, + replication_mask: etl::types::ReplicationMask, + ) -> EtlResult<()> { + let metadata = DestinationTableMetadata::new_applying( + clickhouse_table_name.to_owned(), + snapshot_id, + replication_mask, + ); + self.store.store_destination_table_metadata(table_id, metadata.clone()).await?; + + let ddl = build_create_table_sql(clickhouse_table_name, schema.column_schemas()); + self.client.execute_ddl(DdlKind::CreateTable, &ddl).await?; + + self.store.store_destination_table_metadata(table_id, metadata.to_applied()).await?; + + Ok(()) + } + + /// Ensures the ClickHouse table for the given schema exists, returning + /// `(clickhouse_table_name, nullable_flags)`. + /// + /// On first encounter, executes `CREATE TABLE IF NOT EXISTS` and stores + /// destination metadata with `Applied` status. Subsequent calls return + /// the cached result. + async fn ensure_table_exists( + &self, + schema: &ReplicatedTableSchema, + ) -> EtlResult<(String, Arc<[bool]>)> { + validate_replica_identity_for_clickhouse(schema)?; + let clickhouse_table_name = try_stringify_table_name(schema.name())?; + + if let Some(flags) = self.table_cache.read().get(&clickhouse_table_name).cloned() { + return Ok((clickhouse_table_name, flags)); + } + + let table_id = schema.id(); + match self.store.get_destination_table_metadata(table_id).await? { + None => { + self.create_table_with_metadata( + table_id, + &clickhouse_table_name, + schema, + schema.inner().snapshot_id, + schema.replication_mask().clone(), + ) + .await?; + } + Some(metadata) => { + if metadata.is_applying() { + self.recover_applying_metadata( + table_id, + &clickhouse_table_name, + schema, + metadata, + ) + .await?; + } + // Otherwise the metadata is already `Applied`: this branch + // runs after `handle_relation_event` invalidated the cache, + // so no DDL is needed and we just fall through to recompute + // nullable flags below. + } + } + + // Compute nullable flags from the actual ClickHouse schema. This matters after + // `ALTER TABLE ADD COLUMN`: ClickHouse scalar columns are forced to + // `Nullable(T)` even when the Postgres column is `NOT NULL`, so RowBinary must + // include the nullable marker byte ClickHouse expects. + let actual_columns = self.client.table_columns(&clickhouse_table_name).await?; + let expected_column_names = expected_clickhouse_column_names(schema); + let nullable_flags = nullable_flags_from_clickhouse_columns( + &clickhouse_table_name, + &expected_column_names, + &actual_columns, + )?; + + // `or_insert_with` handles the race where a concurrent caller populated + // the entry between our read-miss and this write. + let flags = { + let mut guard = self.table_cache.write(); + Arc::clone( + guard + .entry(clickhouse_table_name.clone()) + .or_insert_with(|| Arc::clone(&nullable_flags)), + ) + }; + + Ok((clickhouse_table_name, flags)) + } + + /// Re-runs an interrupted DDL idempotently and transitions metadata to + /// `Applied`. Distinguishes between an interrupted schema change (replays + /// the diff against the previous snapshot) and an interrupted initial + /// creation (re-issues `CREATE TABLE IF NOT EXISTS`). + async fn recover_applying_metadata( + &self, + table_id: TableId, + clickhouse_table_name: &str, + schema: &ReplicatedTableSchema, + metadata: DestinationTableMetadata, + ) -> EtlResult<()> { + warn!("table {} has Applying metadata, recovering interrupted operation", table_id); + + match metadata.previous_snapshot_id { + Some(prev_snapshot_id) => { + let old_table_schema = + self.store.get_table_schema(&table_id, prev_snapshot_id).await?.ok_or_else( + || { + etl_error!( + ErrorKind::InvalidState, + "Old schema not found for recovery", + format!( + "Cannot find schema for table {} at snapshot_id {}", + table_id, prev_snapshot_id + ) + ) + }, + )?; + let old_schema = ReplicatedTableSchema::from_mask( + old_table_schema, + metadata.replication_mask.clone(), + ); + let diff = old_schema.diff(schema); + self.apply_schema_diff(clickhouse_table_name, &diff, &old_schema).await?; + } + None => { + let ddl = build_create_table_sql(clickhouse_table_name, schema.column_schemas()); + self.client.execute_ddl(DdlKind::CreateTable, &ddl).await?; + } + } + + self.store.store_destination_table_metadata(table_id, metadata.to_applied()).await?; + Ok(()) + } + + async fn truncate_table_inner(&self, schema: &ReplicatedTableSchema) -> EtlResult<()> { + let (clickhouse_table_name, _) = self.ensure_table_exists(schema).await?; + self.client.truncate_table(&clickhouse_table_name).await + } + + async fn write_table_rows_inner( + &self, + schema: &ReplicatedTableSchema, + table_rows: Vec, + ) -> EtlResult<()> { + let (clickhouse_table_name, nullable_flags) = self.ensure_table_exists(schema).await?; + + let rows: Vec> = table_rows + .into_iter() + .map(|table_row| { + let mut values: Vec = table_row + .into_values() + .into_iter() + .map(cell_to_clickhouse_value) + .collect::>()?; + // CDC columns: initial-copy rows are tagged as INSERT with LSN 0 + // (sentinel meaning "this row pre-dates the streaming cursor"). + values.push(ClickHouseValue::String(CdcOperation::Insert.to_string())); + values.push(cdc_lsn_to_clickhouse_value(PgLsn::from(0))); + Ok(values) + }) + .collect::>()?; + + self.client + .insert_rows( + &clickhouse_table_name, + rows, + &nullable_flags, + self.inserter_config.max_bytes_per_insert, + "copy", + ) + .await + } + + /// Handles a schema change event (Relation) by computing the diff and + /// applying ALTER TABLE statements. + async fn handle_relation_event(&self, new_schema: &ReplicatedTableSchema) -> EtlResult<()> { + validate_replica_identity_for_clickhouse(new_schema)?; + + let table_id = new_schema.id(); + let new_snapshot_id = new_schema.inner().snapshot_id; + let new_replication_mask = new_schema.replication_mask().clone(); + + let metadata = + self.store.get_applied_destination_table_metadata(table_id).await?.ok_or_else( + || { + etl_error!( + ErrorKind::CorruptedTableSchema, + "Missing destination table metadata", + format!( + "No destination table metadata found for table {} when processing \ + schema change. The metadata should have been recorded during initial \ + table synchronization.", + table_id + ) + ) + }, + )?; + + let current_snapshot_id = metadata.snapshot_id; + let current_replication_mask = metadata.replication_mask.clone(); + + if current_snapshot_id == new_snapshot_id + && current_replication_mask == new_replication_mask + { + info!("schema for table {} unchanged (snapshot_id: {})", table_id, new_snapshot_id); + return Ok(()); + } + + info!( + "schema change detected for table {}: snapshot_id {} -> {}", + table_id, current_snapshot_id, new_snapshot_id + ); + + // Retrieve the old schema to compute the diff. + let current_table_schema = + self.store.get_table_schema(&table_id, current_snapshot_id).await?.ok_or_else( + || { + etl_error!( + ErrorKind::InvalidState, + "Old schema not found", + format!( + "Could not find schema for table {} at snapshot_id {}", + table_id, current_snapshot_id + ) + ) + }, + )?; + + let current_schema = ReplicatedTableSchema::from_mask( + current_table_schema, + current_replication_mask.clone(), + ); + + let clickhouse_table_name = &metadata.destination_table_id; + + // Mark as Applying before DDL changes. + let updated_metadata = DestinationTableMetadata::new_applied( + clickhouse_table_name.clone(), + current_snapshot_id, + current_replication_mask, + ) + .with_schema_change( + new_snapshot_id, + new_replication_mask, + DestinationTableSchemaStatus::Applying, + ); + self.store.store_destination_table_metadata(table_id, updated_metadata.clone()).await?; + + // Compute and apply the diff. + let diff = current_schema.diff(new_schema); + if let Err(err) = + self.apply_schema_diff(clickhouse_table_name, &diff, ¤t_schema).await + { + warn!( + "schema change failed for table {}: {}. Manual intervention may be required.", + table_id, err + ); + return Err(err); + } + + // Mark as Applied. + self.store + .store_destination_table_metadata(table_id, updated_metadata.to_applied()) + .await?; + + // Invalidate cached nullable flags so the next write recomputes them. + { + let mut guard = self.table_cache.write(); + guard.remove(clickhouse_table_name); + } + + info!( + "schema change completed for table {}: snapshot_id {} applied", + table_id, new_snapshot_id + ); + + Ok(()) + } + + /// Applies a schema diff to a ClickHouse table: add columns, rename + /// columns, then drop columns (in that order for safety). + /// + /// New columns are placed AFTER the last existing user column (before the + /// CDC columns) using ClickHouse's `AFTER` clause. This is critical because + /// RowBinary encoding is positional -- without explicit placement, ADD + /// COLUMN appends after `cdc_lsn`, misaligning the encoding. + /// + /// Schema changes create an inherently inconsistent window: rows written + /// before the ALTER were encoded with the old column set, while rows + /// after use the new one. Specifically: + /// + /// - ADD COLUMN: existing rows get NULL/default for the new column. + /// - DROP COLUMN: data in the dropped column is lost for all rows. + /// - RENAME COLUMN: existing data is preserved under the new name. + /// + /// ClickHouse does not support transactional DDL, so if the replicator is + /// killed between individual ALTER statements the table may be left in a + /// partially altered state. The `DestinationTableMetadata` Applying/Applied + /// status tracks this for diagnostic purposes. + async fn apply_schema_diff( + &self, + clickhouse_table_name: &str, + diff: &SchemaDiff, + current_schema: &ReplicatedTableSchema, + ) -> EtlResult<()> { + if diff.is_empty() { + return Ok(()); + } + + // Track the last user column name for AFTER placement. New columns + // are inserted after this column, and each added column becomes the + // new anchor for the next. `None` (no user columns in the current + // schema) falls through to `FIRST` placement inside `add_column`, + // which still keeps the new column before the trailing CDC columns. + let mut last_user_column: Option = + current_schema.column_schemas().last().map(|c| c.name.clone()); + + for column in &diff.columns_to_add { + self.client + .add_column(clickhouse_table_name, column, last_user_column.as_deref()) + .await?; + last_user_column = Some(column.name.clone()); + } + + for rename in &diff.columns_to_rename { + self.client + .rename_column(clickhouse_table_name, &rename.old_name, &rename.new_name) + .await?; + } + + for column in &diff.columns_to_remove { + self.client.drop_column(clickhouse_table_name, &column.name).await?; + } + + Ok(()) + } + + /// Processes events in passes driven by an outer loop that runs until the + /// iterator is exhausted. Each pass: + /// 1. Accumulates Insert/Update/Delete rows per table until a Truncate, + /// Relation, or end of events. + /// 2. Writes those rows concurrently. + /// 3. Processes any Relation events (schema changes) sequentially. + /// 4. Drains consecutive Truncate events (deduplicated) and executes them. + async fn write_events_inner(&self, events: Vec) -> EtlResult<()> { + let mut event_iter = events.into_iter().peekable(); + + while event_iter.peek().is_some() { + let mut pending: HashMap)> = + HashMap::new(); + + // Accumulate data events until we hit a Truncate or Relation boundary. + while let Some(event) = event_iter.peek() { + if matches!(event, Event::Truncate(_) | Event::Relation(_)) { + break; + } + + let event = event_iter.next().expect("peeked event must be present; qed"); + match event { + Event::Insert(insert) => { + let table_id = insert.replicated_table_schema.id(); + let entry = pending + .entry(table_id) + .or_insert_with(|| (insert.replicated_table_schema, Vec::new())); + entry.1.push(PendingRow { + operation: CdcOperation::Insert, + lsn: insert.commit_lsn, + cells: insert.table_row.into_values(), + }); + } + Event::Update(update) => { + let UpdatedTableRow::Full(table_row) = update.updated_table_row else { + return Err(etl_error!( + ErrorKind::InvalidState, + "ClickHouse update requires a full new row image", + format!( + "Table '{}' emitted a partial update row: some column values \ + could not be reconstructed. Writing it would record NULL for \ + the missing columns and misrepresent the source. Configuring \ + the source so that all column values are available in the \ + new- or old-row image (e.g. REPLICA IDENTITY FULL) prevents \ + this.", + update.replicated_table_schema.name() + ) + )); + }; + let table_id = update.replicated_table_schema.id(); + let entry = pending + .entry(table_id) + .or_insert_with(|| (update.replicated_table_schema, Vec::new())); + entry.1.push(PendingRow { + operation: CdcOperation::Update, + lsn: update.commit_lsn, + cells: table_row.into_values(), + }); + } + Event::Delete(delete) => { + let Some(old_table_row) = delete.old_table_row else { + debug!("delete event has no row data, skipping"); + continue; + }; + let old_row = match old_table_row { + OldTableRow::Full(row) => row, + OldTableRow::Key(key_row) => { + expand_key_row(key_row, &delete.replicated_table_schema) + } + }; + let table_id = delete.replicated_table_schema.id(); + let entry = pending + .entry(table_id) + .or_insert_with(|| (delete.replicated_table_schema, Vec::new())); + entry.1.push(PendingRow { + operation: CdcOperation::Delete, + lsn: delete.commit_lsn, + cells: old_row.into_values(), + }); + } + event => { + debug!( + event_type = %event.event_type(), + "skipping unsupported event type" + ); + } + } + } + + self.flush_pending_rows(pending).await?; + + // Process Relation events (schema changes) sequentially. + while let Some(Event::Relation(_)) = event_iter.peek() { + if let Some(Event::Relation(relation)) = event_iter.next() { + self.handle_relation_event(&relation.replicated_table_schema).await?; + } + } + + // Collect and deduplicate truncate events. + let mut truncate_schemas: HashMap = HashMap::new(); + while let Some(Event::Truncate(_)) = event_iter.peek() { + if let Some(Event::Truncate(truncate_event)) = event_iter.next() { + for schema in truncate_event.truncated_tables { + truncate_schemas.entry(schema.id()).or_insert(schema); + } + } + } + + futures::future::try_join_all( + truncate_schemas.values().map(|schema| self.truncate_table_inner(schema)), + ) + .await?; + } + + Ok(()) + } + + /// Encodes the accumulated `PendingRow` batches and inserts them into + /// ClickHouse, one `JoinSet` task per table. No-op if `pending` is empty. + /// + /// All `ensure_table_exists` calls run sequentially before any insert is + /// spawned, so a schema-resolution failure aborts the whole pass without + /// any partial-write side effects. + async fn flush_pending_rows( + &self, + pending: HashMap)>, + ) -> EtlResult<()> { + if pending.is_empty() { + return Ok(()); + } + + let mut prepared: Vec<(String, Arc<[bool]>, Vec)> = + Vec::with_capacity(pending.len()); + for (_, (schema, rows)) in pending { + let (clickhouse_table_name, nullable_flags) = self.ensure_table_exists(&schema).await?; + prepared.push((clickhouse_table_name, nullable_flags, rows)); + } + + let mut join_set: JoinSet> = JoinSet::new(); + for (clickhouse_table_name, nullable_flags, rows) in prepared { + let client = self.client.clone(); + let max_bytes = self.inserter_config.max_bytes_per_insert; + + join_set.spawn(async move { + let rows: Vec> = rows + .into_iter() + .map(|PendingRow { operation, lsn, cells }| { + let mut values: Vec = cells + .into_iter() + .map(cell_to_clickhouse_value) + .collect::>()?; + values.push(ClickHouseValue::String(operation.to_string())); + values.push(cdc_lsn_to_clickhouse_value(lsn)); + Ok(values) + }) + .collect::>()?; + + client + .insert_rows( + &clickhouse_table_name, + rows, + &nullable_flags, + max_bytes, + "streaming", + ) + .await + }); + } + + while let Some(result) = join_set.join_next().await { + result.map_err(|e| { + etl_error!(ErrorKind::ApplyWorkerPanic, "insert task failed", e.to_string()) + })??; + } + + Ok(()) + } +} + +/// Rejects replica identities the ClickHouse destination cannot represent. +/// +/// `expand_key_row` assumes the key-only old-row image carries primary-key +/// values, so the row identity must match the primary key. `Full` is also +/// fine because it bypasses `expand_key_row` entirely. `AlternativeKey` +/// (a non-PK unique index) and `Missing` would either land identity values +/// in the wrong PK slots or leave us without enough data to write a +/// well-formed tombstone. +fn validate_replica_identity_for_clickhouse( + replicated_table_schema: &ReplicatedTableSchema, +) -> EtlResult<()> { + match replicated_table_schema.identity_type() { + IdentityType::PrimaryKey | IdentityType::Full => Ok(()), + identity_type => Err(etl_error!( + ErrorKind::SourceSchemaError, + "ClickHouse requires primary-key or full replica identity", + format!( + "Table '{}' uses replica identity {:?}. ClickHouse needs the source row identity \ + to match the primary key (so DELETE tombstones land in the right PK slots) or to \ + carry the full row image. Configure REPLICA IDENTITY DEFAULT (when the PK is the \ + natural identity) or REPLICA IDENTITY FULL.", + replicated_table_schema.name(), + identity_type + ) + )), + } +} + +/// Expands a key-only delete row to full column width for RowBinary encoding. +/// +/// PK columns keep their real values. Non-PK columns get `Cell::Null` if +/// nullable, or a type-appropriate zero value if non-nullable (since RowBinary +/// rejects NULL for non-nullable columns). +/// +/// Caller must ensure the source replica identity is `PrimaryKey` (or `Full`, +/// in which case this function isn't invoked) -- see +/// [`validate_replica_identity_for_clickhouse`]. +fn expand_key_row(key_row: TableRow, schema: &ReplicatedTableSchema) -> TableRow { + let key_cells = key_row.into_values(); + let mut key_iter = key_cells.into_iter(); + let cells: Vec = schema + .column_schemas() + .map(|col| { + if col.primary_key_ordinal_position.is_some() { + key_iter.next().unwrap_or(Cell::Null) + } else if col.nullable && !is_array_type(&col.typ) { + // Nullable scalars -> NULL. Array columns are never nullable + // in ClickHouse (Array(Nullable(T)) without outer Nullable), + // so they must use an empty array default instead. + Cell::Null + } else { + default_cell(&col.typ) + } + }) + .collect(); + TableRow::new(cells) +} + +/// Returns a zero-value Cell for a Postgres type, used to fill non-PK columns +/// in key-only DELETE tombstones. Array types produce empty arrays. All other +/// non-primitive types fall through to an empty String, which is a valid zero +/// value for every ClickHouse String-mapped type (numeric, time, json, bytea). +/// Date, Timestamp, and UUID use typed zero values because their ClickHouse +/// wire format is not String. +fn default_cell(typ: &Type) -> Cell { + use etl::types::ArrayCell; + + match *typ { + Type::BOOL => Cell::Bool(false), + Type::INT2 => Cell::I16(0), + Type::INT4 => Cell::I32(0), + Type::INT8 => Cell::I64(0), + Type::OID => Cell::U32(0), + Type::FLOAT4 => Cell::F32(0.0), + Type::FLOAT8 => Cell::F64(0.0), + Type::DATE => Cell::Date(chrono::NaiveDate::from_ymd_opt(1970, 1, 1).unwrap()), + Type::TIMESTAMP => Cell::Timestamp(chrono::DateTime::UNIX_EPOCH.naive_utc()), + Type::TIMESTAMPTZ => Cell::TimestampTz(chrono::DateTime::UNIX_EPOCH), + Type::UUID => Cell::Uuid(uuid::Uuid::nil()), + Type::BOOL_ARRAY => Cell::Array(ArrayCell::Bool(Vec::new())), + Type::INT2_ARRAY => Cell::Array(ArrayCell::I16(Vec::new())), + Type::INT4_ARRAY => Cell::Array(ArrayCell::I32(Vec::new())), + Type::INT8_ARRAY => Cell::Array(ArrayCell::I64(Vec::new())), + Type::OID_ARRAY => Cell::Array(ArrayCell::U32(Vec::new())), + Type::FLOAT4_ARRAY => Cell::Array(ArrayCell::F32(Vec::new())), + Type::FLOAT8_ARRAY => Cell::Array(ArrayCell::F64(Vec::new())), + Type::TEXT_ARRAY + | Type::VARCHAR_ARRAY + | Type::CHAR_ARRAY + | Type::BPCHAR_ARRAY + | Type::NAME_ARRAY => Cell::Array(ArrayCell::String(Vec::new())), + Type::NUMERIC_ARRAY => Cell::Array(ArrayCell::Numeric(Vec::new())), + Type::DATE_ARRAY => Cell::Array(ArrayCell::Date(Vec::new())), + Type::TIME_ARRAY => Cell::Array(ArrayCell::Time(Vec::new())), + Type::TIMESTAMP_ARRAY => Cell::Array(ArrayCell::Timestamp(Vec::new())), + Type::TIMESTAMPTZ_ARRAY => Cell::Array(ArrayCell::TimestampTz(Vec::new())), + Type::UUID_ARRAY => Cell::Array(ArrayCell::Uuid(Vec::new())), + Type::JSON_ARRAY | Type::JSONB_ARRAY => Cell::Array(ArrayCell::Json(Vec::new())), + Type::BYTEA_ARRAY => Cell::Array(ArrayCell::Bytes(Vec::new())), + _ => Cell::String(String::new()), + } +} + +impl Destination for ClickHouseDestination +where + S: StateStore + SchemaStore + Send + Sync, +{ + fn name() -> &'static str { + "clickhouse" + } + + // The trait methods below intentionally do not use `?` on the inner work. + // Errors must reach the caller via `async_result.send(result)`, not via the + // outer `EtlResult<()>`; using `?` would short-circuit before `send` runs + // and leave the receiver waiting. The outer return value just signals + // "work accepted, watch the channel for completion". `AsyncResult::send` + // itself returns `()`, and its `Drop` impl synthesizes a "dropped without + // sending" error if the path ever skips `send`, so the receiver is never + // silently abandoned. + + async fn truncate_table( + &self, + replicated_table_schema: &ReplicatedTableSchema, + async_result: TruncateTableResult<()>, + ) -> EtlResult<()> { + let result = self.truncate_table_inner(replicated_table_schema).await; + async_result.send(result); + Ok(()) + } + + async fn write_table_rows( + &self, + replicated_table_schema: &ReplicatedTableSchema, + table_rows: Vec, + async_result: WriteTableRowsResult<()>, + ) -> EtlResult<()> { + let result = self.write_table_rows_inner(replicated_table_schema, table_rows).await; + async_result.send(result); + Ok(()) + } + + async fn write_events( + &self, + events: Vec, + async_result: WriteEventsResult<()>, + ) -> EtlResult<()> { + let result = self.write_events_inner(events).await; + async_result.send(result); + Ok(()) + } +} + +#[cfg(test)] +mod tests { + use etl::types::{ColumnSchema, IdentityMask, ReplicationMask, TableName, TableSchema}; + + use super::*; + + fn clickhouse_column(name: &str, type_name: &str) -> ClickHouseTableColumn { + ClickHouseTableColumn { name: name.to_owned(), type_name: type_name.to_owned() } + } + + fn replicated_schema(identity_type: IdentityType) -> ReplicatedTableSchema { + let table_schema = Arc::new(TableSchema::new( + TableId::new(1), + TableName::new("public".to_owned(), "users".to_owned()), + vec![ + ColumnSchema::new("id".to_owned(), Type::INT4, -1, 1, Some(1), false), + ColumnSchema::new("name".to_owned(), Type::TEXT, -1, 2, None, true), + ], + )); + let replication_mask = ReplicationMask::all(&table_schema); + let identity_mask = match identity_type { + IdentityType::Full => IdentityMask::from_bytes(vec![1, 1]), + IdentityType::PrimaryKey => IdentityMask::from_bytes(vec![1, 0]), + IdentityType::AlternativeKey => IdentityMask::from_bytes(vec![0, 1]), + IdentityType::Missing => IdentityMask::from_bytes(vec![0, 0]), + }; + ReplicatedTableSchema::from_masks(table_schema, replication_mask, identity_mask) + } + + #[test] + fn validate_replica_identity_for_clickhouse_accepts_primary_key() { + validate_replica_identity_for_clickhouse(&replicated_schema(IdentityType::PrimaryKey)) + .unwrap(); + } + + #[test] + fn validate_replica_identity_for_clickhouse_accepts_full() { + validate_replica_identity_for_clickhouse(&replicated_schema(IdentityType::Full)).unwrap(); + } + + #[test] + fn validate_replica_identity_for_clickhouse_rejects_alternative_key() { + let err = validate_replica_identity_for_clickhouse(&replicated_schema( + IdentityType::AlternativeKey, + )) + .unwrap_err(); + assert_eq!(err.kind(), ErrorKind::SourceSchemaError); + } + + #[test] + fn validate_replica_identity_for_clickhouse_rejects_missing() { + let err = + validate_replica_identity_for_clickhouse(&replicated_schema(IdentityType::Missing)) + .unwrap_err(); + assert_eq!(err.kind(), ErrorKind::SourceSchemaError); + } + + #[test] + fn cdc_lsn_value_preserves_full_u64_range() { + let value = cdc_lsn_to_clickhouse_value(PgLsn::from(u64::MAX)); + + match value { + ClickHouseValue::UInt64(lsn) => assert_eq!(lsn, u64::MAX), + _ => panic!("expected UInt64 CDC LSN value"), + } + } + + #[test] + fn nullable_flags_use_clickhouse_destination_nullability() { + let expected_names = vec![ + "id".to_owned(), + "score".to_owned(), + "tags".to_owned(), + CDC_OPERATION_COLUMN_NAME.to_owned(), + CDC_LSN_COLUMN_NAME.to_owned(), + ]; + let actual_columns = vec![ + clickhouse_column("id", "Int64"), + clickhouse_column("score", "Nullable(Int32)"), + clickhouse_column("tags", "Array(Nullable(String))"), + clickhouse_column(CDC_OPERATION_COLUMN_NAME, "String"), + clickhouse_column(CDC_LSN_COLUMN_NAME, "UInt64"), + ]; + + let flags = + nullable_flags_from_clickhouse_columns("test_table", &expected_names, &actual_columns) + .unwrap(); + + assert_eq!(flags.as_ref(), [false, true, false, false, false]); + } + + #[test] + fn nullable_flags_reject_clickhouse_column_count_mismatch() { + let expected_names = vec!["id".to_owned(), CDC_OPERATION_COLUMN_NAME.to_owned()]; + let actual_columns = vec![clickhouse_column("id", "Int64")]; + + let err = + nullable_flags_from_clickhouse_columns("test_table", &expected_names, &actual_columns) + .unwrap_err(); + + assert_eq!(err.kind(), ErrorKind::CorruptedTableSchema); + assert_eq!(err.detail(), Some("table 'test_table' has 1 columns, but 2 were expected")); + } + + #[test] + fn nullable_flags_reject_clickhouse_column_order_mismatch() { + let expected_names = vec!["id".to_owned(), "name".to_owned()]; + let actual_columns = + vec![clickhouse_column("name", "String"), clickhouse_column("id", "Int64")]; + + let err = + nullable_flags_from_clickhouse_columns("test_table", &expected_names, &actual_columns) + .unwrap_err(); + + assert_eq!(err.kind(), ErrorKind::CorruptedTableSchema); + assert_eq!( + err.detail(), + Some("table 'test_table' column 1 is named 'name', but 'id' was expected") + ); + } +} diff --git a/etl-destinations/src/clickhouse/encoding.rs b/etl-destinations/src/clickhouse/encoding.rs new file mode 100644 index 000000000..92689dca7 --- /dev/null +++ b/etl-destinations/src/clickhouse/encoding.rs @@ -0,0 +1,512 @@ +use chrono::NaiveDate; +use etl::{ + error::{ErrorKind, EtlResult}, + etl_error, + types::{ArrayCell, Cell}, +}; + +// RowBinary bytes are written directly via `Client::insert_formatted_with`, +// bypassing the typed `Inserter` / serde path because: +// - `Insert::new` panics on empty `COLUMN_NAMES` (via `join_column_names`) even +// with validation disabled. +// - The RowBinary serde serializer wraps `BufMut` with a fresh `&mut` on every +// `serialize_some`, telescoping `&mut &mut ... BytesMut` on nullable array +// elements and overflowing the compiler recursion limit. +// +// Direct byte-writing has no generics and no type-level recursion. + +/// Owned ClickHouse-compatible value, moved (not cloned) from a [`Cell`]. +pub(crate) enum ClickHouseValue { + Null, + Bool(bool), + Int16(i16), + Int32(i32), + Int64(i64), + UInt32(u32), + /// Unsigned 64-bit integer, used for CDC LSN metadata. + UInt64(u64), + Float32(f32), + Float64(f64), + /// TEXT, NUMERIC (string), TIME (string), JSON, BYTEA (hex-encoded) + String(String), + /// Days from Unix epoch (ClickHouse `Date32` on wire = Int32 LE). The + /// signed offset lets us represent pre-1970 dates that ClickHouse `Date` + /// (UInt16) would have silently clamped to 1970-01-01. + Date32(i32), + /// Microseconds since Unix epoch (ClickHouse `DateTime64(6)` on wire = + /// Int64 LE) + DateTime64(i64), + /// UUID in standard 16-byte big-endian order (converted to ClickHouse wire + /// format on encode) + Uuid([u8; 16]), + Array(Vec), +} + +/// Converts a [`Cell`] to a [`ClickHouseValue`], consuming it (no clone). +/// +/// Returns [`ErrorKind::ConversionError`] when a value cannot be represented +/// in ClickHouse without loss, e.g. a Postgres `date` outside ClickHouse +/// `Date32`'s `1900-01-01..=2299-12-31` range. +pub(crate) fn cell_to_clickhouse_value(cell: Cell) -> EtlResult { + Ok(match cell { + Cell::Null => ClickHouseValue::Null, + Cell::Bool(b) => ClickHouseValue::Bool(b), + Cell::I16(v) => ClickHouseValue::Int16(v), + Cell::I32(v) => ClickHouseValue::Int32(v), + Cell::I64(v) => ClickHouseValue::Int64(v), + Cell::U32(v) => ClickHouseValue::UInt32(v), + Cell::F32(v) => ClickHouseValue::Float32(v), + Cell::F64(v) => ClickHouseValue::Float64(v), + Cell::Numeric(n) => ClickHouseValue::String(n.to_string()), + Cell::Date(d) => ClickHouseValue::Date32(date_to_date32_days(d)?), + Cell::Time(t) => ClickHouseValue::String(t.to_string()), + Cell::Timestamp(dt) => ClickHouseValue::DateTime64(dt.and_utc().timestamp_micros()), + Cell::TimestampTz(dt) => ClickHouseValue::DateTime64(dt.timestamp_micros()), + Cell::Uuid(u) => ClickHouseValue::Uuid(*u.as_bytes()), + Cell::Json(j) => ClickHouseValue::String(j.to_string()), + Cell::Bytes(b) => ClickHouseValue::String(bytes_to_hex(&b)), + Cell::String(s) => ClickHouseValue::String(s), + Cell::Array(array_cell) => { + ClickHouseValue::Array(array_cell_to_clickhouse_values(array_cell)?) + } + }) +} + +/// Converts an [`ArrayCell`] to a flat `Vec`, mapping each +/// `Some(x)` to the matching scalar variant and each `None` to +/// [`ClickHouseValue::Null`]. Per-element conversions mirror +/// [`cell_to_clickhouse_value`]. +fn array_cell_to_clickhouse_values(array_cell: ArrayCell) -> EtlResult> { + Ok(match array_cell { + ArrayCell::Bool(v) => map_array(v, ClickHouseValue::Bool), + ArrayCell::String(v) => map_array(v, ClickHouseValue::String), + ArrayCell::I16(v) => map_array(v, ClickHouseValue::Int16), + ArrayCell::I32(v) => map_array(v, ClickHouseValue::Int32), + ArrayCell::I64(v) => map_array(v, ClickHouseValue::Int64), + ArrayCell::U32(v) => map_array(v, ClickHouseValue::UInt32), + ArrayCell::F32(v) => map_array(v, ClickHouseValue::Float32), + ArrayCell::F64(v) => map_array(v, ClickHouseValue::Float64), + ArrayCell::Numeric(v) => map_array(v, |n| ClickHouseValue::String(n.to_string())), + ArrayCell::Date(v) => { + try_map_array(v, |d| Ok(ClickHouseValue::Date32(date_to_date32_days(d)?)))? + } + ArrayCell::Time(v) => map_array(v, |t| ClickHouseValue::String(t.to_string())), + ArrayCell::Timestamp(v) => { + map_array(v, |dt| ClickHouseValue::DateTime64(dt.and_utc().timestamp_micros())) + } + ArrayCell::TimestampTz(v) => { + map_array(v, |dt| ClickHouseValue::DateTime64(dt.timestamp_micros())) + } + ArrayCell::Uuid(v) => map_array(v, |u| ClickHouseValue::Uuid(*u.as_bytes())), + ArrayCell::Json(v) => map_array(v, |j| ClickHouseValue::String(j.to_string())), + ArrayCell::Bytes(v) => map_array(v, |b| ClickHouseValue::String(bytes_to_hex(&b))), + }) +} + +/// Maps a `Vec>` to `Vec`, applying `f` to each +/// `Some` and substituting [`ClickHouseValue::Null`] for each `None`. +fn map_array(v: Vec>, mut f: F) -> Vec +where + F: FnMut(T) -> ClickHouseValue, +{ + v.into_iter() + .map(|o| match o { + Some(t) => f(t), + None => ClickHouseValue::Null, + }) + .collect() +} + +/// Fallible variant of [`map_array`] for element converters that can fail. +fn try_map_array(v: Vec>, mut f: F) -> EtlResult> +where + F: FnMut(T) -> EtlResult, +{ + v.into_iter() + .map(|o| match o { + Some(t) => f(t), + None => Ok(ClickHouseValue::Null), + }) + .collect() +} + +/// Converts a [`NaiveDate`] to a ClickHouse `Date32` day offset (signed days +/// from 1970-01-01). +/// +/// Returns [`ErrorKind::ConversionError`] when the date falls outside +/// ClickHouse `Date32`'s `1900-01-01..=2299-12-31` range. Silent clamping +/// would corrupt historical or far-future values, so we fail the batch +/// instead. +fn date_to_date32_days(d: NaiveDate) -> EtlResult { + if d < date32_min() || d > date32_max() { + return Err(etl_error!( + ErrorKind::ConversionError, + "date out of ClickHouse Date32 range", + format!("{d} is outside the supported range {}..={}", date32_min(), date32_max()) + )); + } + // The bounds check above guarantees the day count fits in i32. + Ok(d.signed_duration_since(unix_epoch()).num_days() as i32) +} + +fn unix_epoch() -> NaiveDate { + NaiveDate::from_ymd_opt(1970, 1, 1).expect("valid date") +} + +fn date32_min() -> NaiveDate { + NaiveDate::from_ymd_opt(1900, 1, 1).expect("valid date") +} + +fn date32_max() -> NaiveDate { + NaiveDate::from_ymd_opt(2299, 12, 31).expect("valid date") +} + +/// Lowercase hex-encodes `bytes` into a fresh `String`. +fn bytes_to_hex(bytes: &[u8]) -> String { + const HEX: &[u8; 16] = b"0123456789abcdef"; + let mut s = String::with_capacity(bytes.len() * 2); + for &b in bytes { + s.push(HEX[(b >> 4) as usize] as char); + s.push(HEX[(b & 0x0f) as usize] as char); + } + s +} + +/// Encodes `v` as LEB128 for ClickHouse string/array lengths. +pub(crate) fn rb_varint(mut v: usize, buf: &mut Vec) { + loop { + // LEB128: bottom 7 bits per byte, MSB set on all but the last. + let byte = (v & 0x7f) as u8; + v >>= 7; + if v == 0 { + buf.push(byte); + return; + } + buf.push(byte | 0x80); + } +} + +/// Encodes a value for a `Nullable(T)` column (1-byte null indicator + value). +pub(crate) fn rb_encode_nullable(val: ClickHouseValue, buf: &mut Vec) -> EtlResult<()> { + match val { + ClickHouseValue::Null => buf.push(1), + v => { + buf.push(0); + rb_encode_value(v, buf)?; + } + } + Ok(()) +} + +/// Encodes a value for a non-nullable column (no null indicator byte). +pub(crate) fn rb_encode_value(val: ClickHouseValue, buf: &mut Vec) -> EtlResult<()> { + match val { + ClickHouseValue::Null => { + // The Postgres schema says this column is NOT NULL, but a NULL arrived. + // If this proves too strict (e.g. transient schema mismatches), we could + // downgrade to writing a zero-length string as a silent fallback. + return Err(etl_error!( + ErrorKind::ConversionError, + "NULL value for non-nullable ClickHouse column" + )); + } + ClickHouseValue::Bool(b) => buf.push(b as u8), + ClickHouseValue::Int16(v) => buf.extend_from_slice(&v.to_le_bytes()), + ClickHouseValue::Int32(v) => buf.extend_from_slice(&v.to_le_bytes()), + ClickHouseValue::Int64(v) => buf.extend_from_slice(&v.to_le_bytes()), + ClickHouseValue::UInt32(v) => buf.extend_from_slice(&v.to_le_bytes()), + ClickHouseValue::UInt64(v) => buf.extend_from_slice(&v.to_le_bytes()), + ClickHouseValue::Float32(v) => buf.extend_from_slice(&v.to_le_bytes()), + ClickHouseValue::Float64(v) => buf.extend_from_slice(&v.to_le_bytes()), + ClickHouseValue::String(s) => { + rb_varint(s.len(), buf); + buf.extend_from_slice(s.as_bytes()); + } + ClickHouseValue::Date32(days) => buf.extend_from_slice(&days.to_le_bytes()), + ClickHouseValue::DateTime64(micros) => buf.extend_from_slice(µs.to_le_bytes()), + ClickHouseValue::Uuid(bytes) => { + // ClickHouse RowBinary UUID = high u64 (LE) then low u64 (LE). Our + // bytes are in standard UUID big-endian order; reinterpret as a + // u128 to split halves cleanly. + let n = u128::from_be_bytes(bytes); + buf.extend_from_slice(&((n >> 64) as u64).to_le_bytes()); + buf.extend_from_slice(&(n as u64).to_le_bytes()); + } + // Array elements are always Nullable in ClickHouse: Array(Nullable(T)). + ClickHouseValue::Array(items) => { + rb_varint(items.len(), buf); + for item in items { + rb_encode_nullable(item, buf)?; + } + } + } + Ok(()) +} + +/// Encodes a complete row into `buf`, selecting nullable vs non-nullable +/// encoding per column. +pub(crate) fn encode_to_row_binary( + values: Vec, + nullable_flags: &[bool], + buf: &mut Vec, +) -> EtlResult<()> { + if values.len() != nullable_flags.len() { + return Err(etl_error!( + ErrorKind::ConversionError, + "ClickHouse RowBinary row width mismatch", + format!( + "values length {} does not match nullable flags length {}", + values.len(), + nullable_flags.len() + ) + )); + } + + for (val, &is_nullable) in values.into_iter().zip(nullable_flags.iter()) { + if is_nullable { + rb_encode_nullable(val, buf)?; + } else { + rb_encode_value(val, buf)?; + } + } + Ok(()) +} + +#[cfg(test)] +mod tests { + use chrono::NaiveDate; + use etl::types::Cell; + use uuid::Uuid; + + use super::*; + + #[test] + fn cell_to_clickhouse_value_null() { + assert!(matches!(cell_to_clickhouse_value(Cell::Null).unwrap(), ClickHouseValue::Null)); + } + + #[test] + fn cell_to_clickhouse_value_bool() { + assert!(matches!( + cell_to_clickhouse_value(Cell::Bool(true)).unwrap(), + ClickHouseValue::Bool(true) + )); + } + + #[test] + fn cell_to_clickhouse_value_i32() { + assert!(matches!( + cell_to_clickhouse_value(Cell::I32(42)).unwrap(), + ClickHouseValue::Int32(42) + )); + } + + #[test] + fn cell_to_clickhouse_value_string() { + if let ClickHouseValue::String(s) = + cell_to_clickhouse_value(Cell::String("hello".to_owned())).unwrap() + { + assert_eq!(s, "hello"); + } else { + panic!("expected String variant"); + } + } + + #[test] + fn cell_to_clickhouse_value_date() { + let epoch = NaiveDate::from_ymd_opt(1970, 1, 1).unwrap(); + assert!(matches!( + cell_to_clickhouse_value(Cell::Date(epoch)).unwrap(), + ClickHouseValue::Date32(0) + )); + + let day1 = NaiveDate::from_ymd_opt(1970, 1, 2).unwrap(); + assert!(matches!( + cell_to_clickhouse_value(Cell::Date(day1)).unwrap(), + ClickHouseValue::Date32(1) + )); + + // Pre-1970 dates round-trip through Date32 as a negative offset rather + // than being silently clamped to the epoch. + let pre_epoch = NaiveDate::from_ymd_opt(1969, 12, 31).unwrap(); + assert!(matches!( + cell_to_clickhouse_value(Cell::Date(pre_epoch)).unwrap(), + ClickHouseValue::Date32(-1) + )); + } + + #[test] + fn cell_to_clickhouse_value_date_out_of_range_errors() { + let too_old = NaiveDate::from_ymd_opt(1899, 12, 31).unwrap(); + assert!(cell_to_clickhouse_value(Cell::Date(too_old)).is_err()); + + let too_new = NaiveDate::from_ymd_opt(2300, 1, 1).unwrap(); + assert!(cell_to_clickhouse_value(Cell::Date(too_new)).is_err()); + } + + #[test] + fn cell_to_clickhouse_value_timestamp() { + let epoch = chrono::DateTime::from_timestamp(0, 0).unwrap().naive_utc(); + assert!(matches!( + cell_to_clickhouse_value(Cell::Timestamp(epoch)).unwrap(), + ClickHouseValue::DateTime64(0) + )); + } + + #[test] + fn cell_to_clickhouse_value_uuid() { + let u = Uuid::parse_str("550e8400-e29b-41d4-a716-446655440000").unwrap(); + let expected_bytes = *u.as_bytes(); + if let ClickHouseValue::Uuid(bytes) = cell_to_clickhouse_value(Cell::Uuid(u)).unwrap() { + assert_eq!(bytes, expected_bytes); + } else { + panic!("expected Uuid variant"); + } + } + + #[test] + fn cell_to_clickhouse_value_bytes_hex() { + let bytes = vec![0xde, 0xad, 0xbe, 0xef]; + if let ClickHouseValue::String(s) = cell_to_clickhouse_value(Cell::Bytes(bytes)).unwrap() { + assert_eq!(s, "deadbeef"); + } else { + panic!("expected String variant"); + } + } + + #[test] + fn rb_encode_value_scalars() { + let mut buf = Vec::new(); + + buf.clear(); + rb_encode_value(ClickHouseValue::Bool(true), &mut buf).unwrap(); + assert_eq!(buf, [1u8]); + + buf.clear(); + rb_encode_value(ClickHouseValue::Int32(-1), &mut buf).unwrap(); + assert_eq!(buf, (-1i32).to_le_bytes()); + + buf.clear(); + rb_encode_value(ClickHouseValue::UInt64(u64::MAX), &mut buf).unwrap(); + assert_eq!(buf, u64::MAX.to_le_bytes()); + + buf.clear(); + rb_encode_value(ClickHouseValue::String("hi".to_owned()), &mut buf).unwrap(); + assert_eq!(buf, [2, b'h', b'i']); // varint(2) + bytes + + buf.clear(); + rb_encode_value(ClickHouseValue::Date32(1), &mut buf).unwrap(); + assert_eq!(buf, 1i32.to_le_bytes()); + + buf.clear(); + rb_encode_value(ClickHouseValue::Date32(-1), &mut buf).unwrap(); + assert_eq!(buf, (-1i32).to_le_bytes()); + } + + #[test] + fn rb_encode_uuid_wire_format() { + let u = Uuid::parse_str("550e8400-e29b-41d4-a716-446655440000").unwrap(); + let val = ClickHouseValue::Uuid(*u.as_bytes()); + let mut buf = Vec::new(); + rb_encode_value(val, &mut buf).unwrap(); + + assert_eq!(buf.len(), 16); + let bytes = u.as_bytes(); + let high = u64::from_be_bytes(bytes[0..8].try_into().unwrap()); + let low = u64::from_be_bytes(bytes[8..16].try_into().unwrap()); + let mut expected = high.to_le_bytes().to_vec(); + expected.extend_from_slice(&low.to_le_bytes()); + assert_eq!(buf, expected); + } + + #[test] + fn encode_nullable() { + let mut buf = Vec::new(); + + rb_encode_nullable(ClickHouseValue::Null, &mut buf).unwrap(); + assert_eq!(buf, [1u8]); + + buf.clear(); + rb_encode_nullable(ClickHouseValue::Int32(42), &mut buf).unwrap(); + let mut expected = vec![0u8]; + expected.extend_from_slice(&42i32.to_le_bytes()); + assert_eq!(buf, expected); + } + + #[test] + fn varint_encoding() { + let mut buf = Vec::new(); + rb_varint(0, &mut buf); + assert_eq!(buf, [0x00]); + + buf.clear(); + rb_varint(127, &mut buf); + assert_eq!(buf, [0x7f]); + + buf.clear(); + rb_varint(128, &mut buf); + assert_eq!(buf, [0x80, 0x01]); + + buf.clear(); + rb_varint(300, &mut buf); + assert_eq!(buf, [0xac, 0x02]); + } + + #[test] + fn hex_encoding() { + assert_eq!(bytes_to_hex(&[]), ""); + assert_eq!(bytes_to_hex(&[0x00]), "00"); + assert_eq!(bytes_to_hex(&[0xff]), "ff"); + assert_eq!(bytes_to_hex(&[0xde, 0xad, 0xbe, 0xef]), "deadbeef"); + } + + /// # GIVEN + /// A NULL ClickHouseValue passed to the non-nullable encoder. + /// + /// # WHEN + /// `rb_encode_value` is called. + /// + /// # THEN + /// It returns a ConversionError rather than writing invalid RowBinary. + #[test] + fn rb_encode_value_rejects_null_for_non_nullable_column() { + let mut buf = Vec::new(); + let result = rb_encode_value(ClickHouseValue::Null, &mut buf); + + assert!(result.is_err(), "NULL in non-nullable column must error"); + let err = result.unwrap_err(); + assert_eq!(err.kind(), ErrorKind::ConversionError); + assert!(buf.is_empty(), "no bytes should be written on error"); + } + + #[test] + fn encode_to_row_binary_rejects_fewer_values_than_nullable_flags() { + let mut buf = vec![0xaa]; + let result = + encode_to_row_binary(vec![ClickHouseValue::Int32(1)], &[false, false], &mut buf); + + assert!(result.is_err(), "row width mismatch must error"); + let err = result.unwrap_err(); + assert_eq!(err.kind(), ErrorKind::ConversionError); + assert_eq!(err.description(), Some("ClickHouse RowBinary row width mismatch")); + assert_eq!(err.detail(), Some("values length 1 does not match nullable flags length 2")); + assert_eq!(buf, vec![0xaa], "no bytes should be written on error"); + } + + #[test] + fn encode_to_row_binary_rejects_more_values_than_nullable_flags() { + let mut buf = vec![0xaa]; + let result = encode_to_row_binary( + vec![ClickHouseValue::Int32(1), ClickHouseValue::Int32(2)], + &[false], + &mut buf, + ); + + assert!(result.is_err(), "row width mismatch must error"); + let err = result.unwrap_err(); + assert_eq!(err.kind(), ErrorKind::ConversionError); + assert_eq!(err.description(), Some("ClickHouse RowBinary row width mismatch")); + assert_eq!(err.detail(), Some("values length 2 does not match nullable flags length 1")); + assert_eq!(buf, vec![0xaa], "no bytes should be written on error"); + } +} diff --git a/etl-destinations/src/clickhouse/metrics.rs b/etl-destinations/src/clickhouse/metrics.rs new file mode 100644 index 000000000..be287129e --- /dev/null +++ b/etl-destinations/src/clickhouse/metrics.rs @@ -0,0 +1,36 @@ +use std::sync::Once; + +use metrics::{Unit, describe_histogram}; + +static REGISTER_METRICS: Once = Once::new(); + +/// Duration of DDL operations sent to ClickHouse. +/// Labels: `kind` (`create_table`, `add_column`, `drop_column`, +/// `rename_column`). +pub(super) const ETL_CLICKHOUSE_DDL_DURATION_SECONDS: &str = "etl_clickhouse_ddl_duration_seconds"; + +/// Duration of a single RowBinary INSERT statement from first write to server +/// acknowledgement. Labels: `source` (`copy` = initial table sync, `streaming` +/// = CDC events). +pub(super) const ETL_CLICKHOUSE_INSERT_DURATION_SECONDS: &str = + "etl_clickhouse_insert_duration_seconds"; + +/// Register ClickHouse-specific metrics. +/// +/// Safe to call multiple times — registration happens only once. +pub(super) fn register_metrics() { + REGISTER_METRICS.call_once(|| { + describe_histogram!( + ETL_CLICKHOUSE_DDL_DURATION_SECONDS, + Unit::Seconds, + "Duration of DDL operations sent to ClickHouse, labeled by kind" + ); + + describe_histogram!( + ETL_CLICKHOUSE_INSERT_DURATION_SECONDS, + Unit::Seconds, + "Duration of RowBinary INSERT statements from first write to server acknowledgement, \ + labeled by source" + ); + }); +} diff --git a/etl-destinations/src/clickhouse/mod.rs b/etl-destinations/src/clickhouse/mod.rs new file mode 100644 index 000000000..87768db26 --- /dev/null +++ b/etl-destinations/src/clickhouse/mod.rs @@ -0,0 +1,11 @@ +pub mod client; +mod core; +mod encoding; +mod metrics; +mod schema; +#[cfg(feature = "test-utils")] +pub mod test_utils; + +pub use core::{ClickHouseDestination, ClickHouseInserterConfig}; + +pub use client::ClickHouseClient; diff --git a/etl-destinations/src/clickhouse/schema.rs b/etl-destinations/src/clickhouse/schema.rs new file mode 100644 index 000000000..cae5b3aa3 --- /dev/null +++ b/etl-destinations/src/clickhouse/schema.rs @@ -0,0 +1,242 @@ +use etl::types::{ColumnSchema, Type, is_array_type}; + +/// Name of the CDC operation metadata column appended to ClickHouse tables. +pub(crate) const CDC_OPERATION_COLUMN_NAME: &str = "cdc_operation"; +/// Name of the CDC LSN metadata column appended to ClickHouse tables. +pub(crate) const CDC_LSN_COLUMN_NAME: &str = "cdc_lsn"; + +/// Returns the base ClickHouse type string for a Postgres scalar type. +/// +/// The returned string does not include `Nullable(...)` wrapping — callers are +/// responsible for applying that when the column is nullable. Arrays always use +/// `Array(Nullable(T))` since Postgres array elements are nullable. +fn postgres_column_type_to_clickhouse_sql(typ: &Type) -> &'static str { + match typ { + &Type::BOOL => "Boolean", + &Type::CHAR | &Type::BPCHAR | &Type::VARCHAR | &Type::NAME | &Type::TEXT => "String", + &Type::INT2 => "Int16", + &Type::INT4 => "Int32", + &Type::INT8 => "Int64", + &Type::FLOAT4 => "Float32", + &Type::FLOAT8 => "Float64", + &Type::NUMERIC => "String", + &Type::DATE => "Date32", + &Type::TIME => "String", + &Type::TIMESTAMP => "DateTime64(6)", + &Type::TIMESTAMPTZ => "DateTime64(6, 'UTC')", + &Type::UUID => "UUID", + &Type::JSON | &Type::JSONB => "String", + &Type::BYTEA => "String", + &Type::OID => "UInt32", + _ => "String", + } +} + +/// Returns the ClickHouse array element type for a Postgres array type. +fn postgres_array_element_clickhouse_sql(typ: &Type) -> &'static str { + match typ { + &Type::BOOL_ARRAY => "Boolean", + &Type::CHAR_ARRAY + | &Type::BPCHAR_ARRAY + | &Type::VARCHAR_ARRAY + | &Type::NAME_ARRAY + | &Type::TEXT_ARRAY => "String", + &Type::INT2_ARRAY => "Int16", + &Type::INT4_ARRAY => "Int32", + &Type::INT8_ARRAY => "Int64", + &Type::FLOAT4_ARRAY => "Float32", + &Type::FLOAT8_ARRAY => "Float64", + &Type::NUMERIC_ARRAY => "String", + &Type::DATE_ARRAY => "Date32", + &Type::TIME_ARRAY => "String", + &Type::TIMESTAMP_ARRAY => "DateTime64(6)", + &Type::TIMESTAMPTZ_ARRAY => "DateTime64(6, 'UTC')", + &Type::UUID_ARRAY => "UUID", + &Type::JSON_ARRAY | &Type::JSONB_ARRAY => "String", + &Type::BYTEA_ARRAY => "String", + &Type::OID_ARRAY => "UInt32", + _ => "String", + } +} + +/// Quotes a ClickHouse identifier, escaping embedded double quotes. +pub(crate) fn quote_identifier(identifier: &str) -> String { + format!("\"{}\"", identifier.replace('"', "\"\"")) +} + +/// Returns the full ClickHouse type string for a column, with Nullable +/// wrapping. +/// +/// When `force_nullable` is true (ALTER TABLE ADD), all scalar columns become +/// Nullable since ClickHouse cannot backfill existing rows. +pub(super) fn clickhouse_column_type(col: &ColumnSchema, force_nullable: bool) -> String { + if is_array_type(&col.typ) { + let elem = postgres_array_element_clickhouse_sql(&col.typ); + format!("Array(Nullable({elem}))") + } else { + let base = postgres_column_type_to_clickhouse_sql(&col.typ); + if col.nullable || force_nullable { format!("Nullable({base})") } else { base.to_owned() } + } +} + +/// Generates a `CREATE TABLE IF NOT EXISTS` DDL for the given columns. +/// +/// Appends `cdc_operation String` and `cdc_lsn UInt64` as trailing non-nullable +/// columns. Uses `MergeTree()` with `ORDER BY tuple()`. +pub(super) fn build_create_table_sql<'a, I>(table_name: &str, column_schemas: I) -> String +where + I: IntoIterator, + I::IntoIter: ExactSizeIterator, +{ + let iter = column_schemas.into_iter(); + let mut cols = Vec::with_capacity(iter.len() + 2); + + for col in iter { + let col_type = clickhouse_column_type(col, false); + cols.push(format!(" {} {}", quote_identifier(&col.name), col_type)); + } + + // CDC columns — always non-nullable + cols.push(format!(" {} String", quote_identifier(CDC_OPERATION_COLUMN_NAME))); + cols.push(format!(" {} UInt64", quote_identifier(CDC_LSN_COLUMN_NAME))); + + let col_defs = cols.join(",\n"); + let quoted_table_name = quote_identifier(table_name); + format!( + "CREATE TABLE IF NOT EXISTS {quoted_table_name} (\n{col_defs}\n) ENGINE = \ + MergeTree()\nORDER BY tuple()" + ) +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn quote_identifier_escapes_embedded_quotes() { + assert_eq!(quote_identifier("plain"), "\"plain\""); + assert_eq!(quote_identifier("has\"quote"), "\"has\"\"quote\""); + } + + #[test] + fn build_create_table_sql_quotes_identifiers() { + let schemas = vec![ColumnSchema { + name: "id\"value".to_owned(), + typ: Type::INT4, + modifier: -1, + ordinal_position: 1, + primary_key_ordinal_position: Some(1), + nullable: false, + }]; + // Pre-encoded table name with embedded quotes to verify the SQL + // builder quotes/escapes the identifier itself. + let sql = build_create_table_sql("sche\"ma_ta\"ble", &schemas); + + assert!( + sql.contains("CREATE TABLE IF NOT EXISTS \"sche\"\"ma_ta\"\"ble\""), + "schema-derived table name should be quoted and escaped: {sql}" + ); + assert!( + sql.contains("\"id\"\"value\" Int32"), + "column name should be quoted and escaped: {sql}" + ); + } + + #[test] + fn scalar_type_mapping() { + assert_eq!(postgres_column_type_to_clickhouse_sql(&Type::BOOL), "Boolean"); + assert_eq!(postgres_column_type_to_clickhouse_sql(&Type::CHAR), "String"); + assert_eq!(postgres_column_type_to_clickhouse_sql(&Type::BPCHAR), "String"); + assert_eq!(postgres_column_type_to_clickhouse_sql(&Type::VARCHAR), "String"); + assert_eq!(postgres_column_type_to_clickhouse_sql(&Type::NAME), "String"); + assert_eq!(postgres_column_type_to_clickhouse_sql(&Type::TEXT), "String"); + assert_eq!(postgres_column_type_to_clickhouse_sql(&Type::INT2), "Int16"); + assert_eq!(postgres_column_type_to_clickhouse_sql(&Type::INT4), "Int32"); + assert_eq!(postgres_column_type_to_clickhouse_sql(&Type::INT8), "Int64"); + assert_eq!(postgres_column_type_to_clickhouse_sql(&Type::FLOAT4), "Float32"); + assert_eq!(postgres_column_type_to_clickhouse_sql(&Type::FLOAT8), "Float64"); + assert_eq!(postgres_column_type_to_clickhouse_sql(&Type::NUMERIC), "String"); + assert_eq!(postgres_column_type_to_clickhouse_sql(&Type::DATE), "Date32"); + assert_eq!(postgres_column_type_to_clickhouse_sql(&Type::TIME), "String"); + assert_eq!(postgres_column_type_to_clickhouse_sql(&Type::TIMESTAMP), "DateTime64(6)"); + assert_eq!( + postgres_column_type_to_clickhouse_sql(&Type::TIMESTAMPTZ), + "DateTime64(6, 'UTC')" + ); + assert_eq!(postgres_column_type_to_clickhouse_sql(&Type::UUID), "UUID"); + assert_eq!(postgres_column_type_to_clickhouse_sql(&Type::JSON), "String"); + assert_eq!(postgres_column_type_to_clickhouse_sql(&Type::JSONB), "String"); + assert_eq!(postgres_column_type_to_clickhouse_sql(&Type::BYTEA), "String"); + assert_eq!(postgres_column_type_to_clickhouse_sql(&Type::OID), "UInt32"); + } + + #[test] + fn array_type_mapping() { + assert_eq!(postgres_array_element_clickhouse_sql(&Type::BOOL_ARRAY), "Boolean"); + assert_eq!(postgres_array_element_clickhouse_sql(&Type::TEXT_ARRAY), "String"); + assert_eq!(postgres_array_element_clickhouse_sql(&Type::INT4_ARRAY), "Int32"); + assert_eq!(postgres_array_element_clickhouse_sql(&Type::INT8_ARRAY), "Int64"); + assert_eq!(postgres_array_element_clickhouse_sql(&Type::FLOAT8_ARRAY), "Float64"); + assert_eq!(postgres_array_element_clickhouse_sql(&Type::UUID_ARRAY), "UUID"); + assert_eq!(postgres_array_element_clickhouse_sql(&Type::JSONB_ARRAY), "String"); + } + + #[test] + fn build_create_table_sql_nullable() { + let schemas = vec![ + ColumnSchema { + name: "id".to_owned(), + typ: Type::INT4, + modifier: -1, + ordinal_position: 1, + primary_key_ordinal_position: Some(1), + nullable: false, + }, + ColumnSchema { + name: "name".to_owned(), + typ: Type::TEXT, + modifier: -1, + ordinal_position: 2, + primary_key_ordinal_position: None, + nullable: true, + }, + ]; + let sql = build_create_table_sql("public_users", &schemas); + assert!(sql.contains("\"id\" Int32"), "id should be non-nullable Int32"); + assert!(sql.contains("\"name\" Nullable(String)"), "name should be Nullable(String)"); + } + + #[test] + fn build_create_table_sql_cdc_columns() { + let schemas = vec![ColumnSchema { + name: "id".to_owned(), + typ: Type::INT4, + modifier: -1, + ordinal_position: 1, + primary_key_ordinal_position: Some(1), + nullable: false, + }]; + let sql = build_create_table_sql("public_t", &schemas); + assert!(sql.contains("\"cdc_operation\" String"), "cdc_operation should be non-nullable"); + assert!(sql.contains("\"cdc_lsn\" UInt64"), "cdc_lsn should be non-nullable UInt64"); + assert!(sql.contains("ENGINE = MergeTree()")); + assert!(sql.contains("ORDER BY tuple()")); + } + + #[test] + fn build_create_table_sql_array_columns() { + let schemas = vec![ColumnSchema { + name: "tags".to_owned(), + typ: Type::TEXT_ARRAY, + modifier: -1, + ordinal_position: 1, + primary_key_ordinal_position: None, + nullable: false, + }]; + let sql = build_create_table_sql("public_t", &schemas); + assert!( + sql.contains("\"tags\" Array(Nullable(String))"), + "array columns should always be Array(Nullable(T))" + ); + } +} diff --git a/etl-destinations/src/clickhouse/test_utils.rs b/etl-destinations/src/clickhouse/test_utils.rs new file mode 100644 index 000000000..cfb54a8af --- /dev/null +++ b/etl-destinations/src/clickhouse/test_utils.rs @@ -0,0 +1,232 @@ +//! Test utilities for ClickHouse destinations. + +use clickhouse::Client; +use etl::store::{schema::SchemaStore, state::StateStore}; +use tokio::runtime::Handle; +use url::Url; +use uuid::Uuid; + +use crate::clickhouse::{ClickHouseDestination, ClickHouseInserterConfig}; + +/// ClickHouse HTTP URL (e.g. `http://localhost:8123`). +pub const CLICKHOUSE_URL_ENV: &str = "TESTS_CLICKHOUSE_URL"; +/// ClickHouse user name (required). +pub const CLICKHOUSE_USER_ENV: &str = "TESTS_CLICKHOUSE_USER"; +/// ClickHouse password (optional -- omit or leave empty for passwordless +/// access). +pub const CLICKHOUSE_PASSWORD_ENV: &str = "TESTS_CLICKHOUSE_PASSWORD"; + +/// Returns the ClickHouse HTTP URL from the environment. +/// +/// # Panics +/// +/// Panics if [`CLICKHOUSE_URL_ENV`] is not set or is not a valid URL. +pub fn get_clickhouse_url() -> Url { + let value = std::env::var(CLICKHOUSE_URL_ENV) + .unwrap_or_else(|_| panic!("{CLICKHOUSE_URL_ENV} must be set")); + Url::parse(&value) + .unwrap_or_else(|error| panic!("{CLICKHOUSE_URL_ENV} must be a valid URL: {error}")) +} + +/// Returns the ClickHouse user name from the environment. +/// +/// # Panics +/// +/// Panics if [`CLICKHOUSE_USER_ENV`] is not set. +pub fn get_clickhouse_user() -> String { + std::env::var(CLICKHOUSE_USER_ENV) + .unwrap_or_else(|_| panic!("{CLICKHOUSE_USER_ENV} must be set")) +} + +/// Returns the ClickHouse password from the environment, or `None` if unset. +pub fn get_clickhouse_password() -> Option { + std::env::var(CLICKHOUSE_PASSWORD_ENV).ok().filter(|s| !s.is_empty()) +} + +/// Generates a unique database name for test isolation. +pub fn random_database_name() -> String { + format!("etl_tests_{}", Uuid::new_v4().simple()) +} + +/// ClickHouse connection for testing. +/// +/// Wraps a [`Client`] and automatically drops the test database on [`Drop`]. +pub struct ClickHouseTestDatabase { + /// Root client (no database selected) used for CREATE/DROP DATABASE. + root_client: Client, + /// Client scoped to the test database for queries. + db_client: Client, + url: Url, + user: String, + password: Option, + database: String, +} + +impl ClickHouseTestDatabase { + fn new(url: Url, user: String, password: Option, database: String) -> Self { + let build_client = |db: Option<&str>| { + let mut c = Client::default().with_url(url.as_str()).with_user(&user); + if let Some(db) = db { + c = c.with_database(db); + } + if let Some(pw) = &password { + c = c.with_password(pw); + } + c + }; + + Self { + root_client: build_client(None), + db_client: build_client(Some(&database)), + url, + user, + password, + database, + } + } + + /// Creates the test database in ClickHouse, retrying on transient errors. + pub async fn create_database(&self) { + let query = format!("CREATE DATABASE IF NOT EXISTS `{}`", self.database); + for attempt in 1..=5 { + match self.root_client.query(&query).execute().await { + Ok(()) => return, + Err(e) if attempt < 5 => { + eprintln!( + "warning: create_database attempt {attempt}/5 failed: {e}, retrying..." + ); + tokio::time::sleep(std::time::Duration::from_millis(200 * attempt)).await; + } + Err(e) => panic!("Failed to create test ClickHouse database after 5 attempts: {e}"), + } + } + } + + /// Drops the test database from ClickHouse. + pub async fn drop_database(&self) { + self.root_client + .query(&format!("DROP DATABASE IF EXISTS `{}`", self.database)) + .execute() + .await + .expect("Failed to drop test ClickHouse database"); + } + + /// Builds a [`ClickHouseDestination`] scoped to this test database with + /// default inserter config (100 MiB per INSERT -- large enough that tests + /// never hit an intermediate flush). + pub fn build_destination(&self, store: S) -> ClickHouseDestination + where + S: StateStore + SchemaStore + Send + Sync, + { + self.build_destination_with_config( + store, + ClickHouseInserterConfig { max_bytes_per_insert: 100 * 1024 * 1024 }, + ) + } + + /// Builds a [`ClickHouseDestination`] scoped to this test database with + /// a caller-supplied [`ClickHouseInserterConfig`]. + pub fn build_destination_with_config( + &self, + store: S, + config: ClickHouseInserterConfig, + ) -> ClickHouseDestination + where + S: StateStore + SchemaStore + Send + Sync, + { + ClickHouseDestination::new( + self.url.clone(), + &self.user, + self.password.clone(), + &self.database, + config, + store, + ) + .expect("Failed to create ClickHouseDestination for test") + } + + /// Fetches all rows from a ClickHouse table using the given SQL query. + /// + /// `T` must be an owned row type (i.e. `Value<'a> = Self`) and implement + /// [`serde::de::DeserializeOwned`]. The caller is responsible for writing a + /// SELECT whose columns match `T`'s fields in the correct order. + pub async fn query(&self, sql: &str) -> Vec + where + T: for<'a> clickhouse::Row = T> + serde::de::DeserializeOwned + 'static, + { + self.db_client.query(sql).fetch_all::().await.expect("ClickHouse query failed") + } + + /// Returns the underlying ClickHouse client for fallible queries. + pub fn db_client(&self) -> &Client { + &self.db_client + } + + /// Returns the column names of a ClickHouse table in position order, + /// excluding the CDC columns (`cdc_operation`, `cdc_lsn`). + pub async fn column_names(&self, table_name: &str) -> Vec { + self.column_types(table_name).await.into_iter().map(|(name, _)| name).collect() + } + + /// Returns the column names and ClickHouse type strings in position order, + /// excluding the CDC columns (`cdc_operation`, `cdc_lsn`). + pub async fn column_types(&self, table_name: &str) -> Vec<(String, String)> { + #[derive(clickhouse::Row, serde::Deserialize)] + struct Col { + name: String, + type_name: String, + } + self.db_client + .query( + "SELECT name, type AS type_name FROM system.columns WHERE database = ? AND table \ + = ? AND name NOT IN ('cdc_operation', 'cdc_lsn') ORDER BY position", + ) + .bind(&self.database) + .bind(table_name) + .fetch_all::() + .await + .expect("failed to query system.columns") + .into_iter() + .map(|c| (c.name, c.type_name)) + .collect() + } +} + +impl Drop for ClickHouseTestDatabase { + fn drop(&mut self) { + let root_client = self.root_client.clone(); + let database = self.database.clone(); + + let _ = std::panic::catch_unwind(std::panic::AssertUnwindSafe(|| { + tokio::task::block_in_place(move || { + Handle::current().block_on(async move { + if let Err(error) = root_client + .query(&format!("DROP DATABASE IF EXISTS `{database}`")) + .execute() + .await + { + eprintln!("warning: failed to drop test ClickHouse database: {error}"); + } + }); + }); + })); + } +} + +/// Creates a fresh, isolated ClickHouse database for a single test. +/// +/// Reads connection parameters from environment variables: +/// - [`CLICKHOUSE_URL_ENV`] — required +/// - [`CLICKHOUSE_USER_ENV`] — required +/// - [`CLICKHOUSE_PASSWORD_ENV`] — optional +/// +/// The database is dropped automatically when the returned handle is dropped. +pub async fn setup_clickhouse_database() -> ClickHouseTestDatabase { + let url = get_clickhouse_url(); + let user = get_clickhouse_user(); + let password = get_clickhouse_password(); + let database = random_database_name(); + let db = ClickHouseTestDatabase::new(url, user, password, database); + db.create_database().await; + db +} diff --git a/etl-destinations/src/lib.rs b/etl-destinations/src/lib.rs index aa1c15014..861f72d2f 100644 --- a/etl-destinations/src/lib.rs +++ b/etl-destinations/src/lib.rs @@ -10,6 +10,8 @@ mod table_name; #[cfg(feature = "bigquery")] pub mod bigquery; +#[cfg(feature = "clickhouse")] +pub mod clickhouse; #[cfg(feature = "ducklake")] pub mod ducklake; #[cfg(feature = "egress")] diff --git a/etl-destinations/tests/clickhouse_pipeline.rs b/etl-destinations/tests/clickhouse_pipeline.rs new file mode 100644 index 000000000..e5265236e --- /dev/null +++ b/etl-destinations/tests/clickhouse_pipeline.rs @@ -0,0 +1,2313 @@ +use std::{sync::Once, time::Duration}; + +use etl::{ + state::table::TableReplicationPhaseType, + store::state::StateStore, + test_utils::{ + database::{spawn_source_database, test_table_name}, + notifying_store::NotifyingStore, + pipeline::create_pipeline, + }, + types::PipelineId, +}; +use etl_destinations::clickhouse::{ + ClickHouseInserterConfig, + client::ClickHouseClient, + test_utils::{ + ClickHouseTestDatabase, get_clickhouse_password, get_clickhouse_url, get_clickhouse_user, + setup_clickhouse_database, + }, +}; +use etl_postgres::tokio::test_utils::TableModification; +use etl_telemetry::tracing::init_test_tracing; +use rand::random; +use tokio::time::sleep; +use url::Url; + +use crate::support::clickhouse::{AllTypesRow, BoundaryValuesRow, DateBoundariesRow}; + +/// Ensures the rustls crypto provider is only installed once across all tests. +static INIT_CRYPTO: Once = Once::new(); + +fn install_crypto_provider() { + INIT_CRYPTO.call_once(|| { + rustls::crypto::aws_lc_rs::default_provider() + .install_default() + .expect("failed to install default crypto provider"); + }); +} + +/// SELECT query that fetches all verified columns from the ClickHouse table. +/// +/// `uuid_col` is projected via `toString()` because the ClickHouse UUID +/// RowBinary wire format does not directly map to a Rust `String`; `toString()` +/// gives us the canonical `xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx` string form. +/// +/// All other columns are read with their native ClickHouse types: +/// - `Date` -> u16 (days since 1970-01-01) +/// - `DateTime64(6)` -> i64 (microseconds since epoch) +/// - `Array(Nullable(T))` -> `Vec>` +const ALL_TYPES_SELECT: &str = concat!( + "SELECT ", + "id, smallint_col, integer_col, bigint_col, real_col, double_col, ", + "numeric_col, boolean_col, text_col, varchar_col, ", + "date_col, timestamp_col, timestamptz_col, time_col, interval_col, ", + "jsonb_col, json_col, integer_array_col, text_array_col, ", + "bytea_col, inet_col, cidr_col, macaddr_col, ", + "toString(uuid_col) AS uuid_col, ", + "cdc_operation ", + "FROM \"test_all__types__encoding\" ", + "ORDER BY id", +); + +/// A row read back from the ClickHouse `update_flow` test table. +#[derive(clickhouse::Row, serde::Deserialize, Debug)] +struct UpdateFlowRow { + id: i64, + value: String, + cdc_operation: String, + cdc_lsn: u64, +} + +/// SELECT query used to verify the `update_flow` streaming test. +const UPDATE_FLOW_SELECT: &str = concat!( + "SELECT id, value, cdc_operation, cdc_lsn ", + "FROM \"test_update__flow\" ", + "ORDER BY id, cdc_lsn", +); + +/// SELECT query used to verify the `delete_flow` streaming test. +const DELETE_FLOW_SELECT: &str = concat!( + "SELECT id, value, cdc_operation, cdc_lsn ", + "FROM \"test_delete__flow\" ", + "ORDER BY id, cdc_lsn", +); + +/// SELECT query used to verify the `restart_flow` test. +const RESTART_FLOW_SELECT: &str = concat!( + "SELECT id, value, cdc_operation, cdc_lsn ", + "FROM \"test_restart__flow\" ", + "ORDER BY id, cdc_lsn", +); + +/// SELECT query used to verify the `truncate_flow` test. +const TRUNCATE_FLOW_SELECT: &str = concat!( + "SELECT id, value, cdc_operation, cdc_lsn ", + "FROM \"test_truncate__flow\" ", + "ORDER BY id, cdc_lsn", +); + +/// Days from 1970-01-01 to 2024-01-15 (used to verify the `date_col` +/// round-trip). +/// +/// Python: `(date(2024, 1, 15) - date(1970, 1, 1)).days` = 19737 +const DATE_2024_01_15_DAYS: i32 = 19737; + +/// Microseconds from epoch for `2024-01-15 12:00:00 UTC`. +const TS_2024_01_15_12_00_US: i64 = 1_705_320_000_000_000; + +/// Waits until ClickHouse returns at least `expected_rows` from +/// `UPDATE_FLOW_SELECT`. +async fn wait_for_update_flow_rows( + clickhouse_db: &ClickHouseTestDatabase, + expected_rows: usize, +) -> Vec { + let mut rows: Vec = Vec::with_capacity(expected_rows); + for _ in 0..50 { + rows = clickhouse_db.query(UPDATE_FLOW_SELECT).await; + if rows.len() >= expected_rows { + return rows; + } + sleep(Duration::from_millis(100)).await; + } + + panic!( + "timed out waiting for clickhouse update_flow rows: got {} of {}", + rows.len(), + expected_rows, + ); +} + +/// Waits until ClickHouse returns at least `expected_rows` from +/// `DELETE_FLOW_SELECT`. +async fn wait_for_delete_flow_rows( + clickhouse_db: &ClickHouseTestDatabase, + expected_rows: usize, +) -> Vec { + let mut rows: Vec = Vec::with_capacity(expected_rows); + for _ in 0..50 { + rows = clickhouse_db.query(DELETE_FLOW_SELECT).await; + if rows.len() >= expected_rows { + return rows; + } + sleep(Duration::from_millis(100)).await; + } + + panic!( + "timed out waiting for clickhouse delete_flow rows: got {} of {}", + rows.len(), + expected_rows, + ); +} + +/// Waits until ClickHouse returns at least `expected_rows` from +/// `RESTART_FLOW_SELECT`. +async fn wait_for_restart_flow_rows( + clickhouse_db: &ClickHouseTestDatabase, + expected_rows: usize, +) -> Vec { + let mut rows: Vec = Vec::with_capacity(expected_rows); + for _ in 0..50 { + rows = clickhouse_db.query(RESTART_FLOW_SELECT).await; + if rows.len() >= expected_rows { + return rows; + } + sleep(Duration::from_millis(100)).await; + } + + panic!( + "timed out waiting for clickhouse restart_flow rows: got {} of {}", + rows.len(), + expected_rows, + ); +} + +/// Waits until ClickHouse returns exactly zero rows from +/// `TRUNCATE_FLOW_SELECT`. +async fn wait_for_truncate_flow_empty(clickhouse_db: &ClickHouseTestDatabase) { + for _ in 0..50 { + let rows: Vec = clickhouse_db.query(TRUNCATE_FLOW_SELECT).await; + if rows.is_empty() { + return; + } + sleep(Duration::from_millis(100)).await; + } + + panic!("timed out waiting for clickhouse truncate_flow table to become empty"); +} + +/// Waits until ClickHouse returns at least `expected_rows` from +/// `TRUNCATE_FLOW_SELECT`. +async fn wait_for_truncate_flow_rows( + clickhouse_db: &ClickHouseTestDatabase, + expected_rows: usize, +) -> Vec { + let mut rows: Vec = Vec::with_capacity(expected_rows); + for _ in 0..50 { + rows = clickhouse_db.query(TRUNCATE_FLOW_SELECT).await; + if rows.len() >= expected_rows { + return rows; + } + sleep(Duration::from_millis(100)).await; + } + + panic!( + "timed out waiting for clickhouse truncate_flow rows: got {} of {}", + rows.len(), + expected_rows, + ); +} + +/// Tests that all Postgres column types (including nullable arrays) round-trip +/// correctly through the ClickHouse RowBinary encoding. +/// +/// # GIVEN +/// +/// A Postgres table covering every supported column type -- scalars (integers, +/// floats, numeric, boolean, text, varchar, date, timestamp, timestamptz, time, +/// interval, jsonb, json, bytea, inet, cidr, macaddr, uuid) and nullable array +/// columns (`integer[]`, `text[]`). Two rows are inserted before the pipeline +/// starts: +/// +/// 1. Positive/typical values with **empty** arrays. +/// 2. Boundary values (min-ints, negative floats) with **non-empty** arrays. +/// +/// # WHEN +/// +/// The pipeline runs initial table copy from Postgres to ClickHouse. +/// +/// # THEN +/// +/// Every column round-trips correctly: +/// - Scalars match their inserted values exactly (floats within epsilon). +/// - Empty arrays remain empty; non-empty arrays preserve elements. +/// - Both rows have `cdc_operation = "INSERT"`. +/// +/// # Regression +/// +/// Row 2's non-empty arrays specifically catch the nullable-array encoding bug +/// where `nullable_flags[i] = true` for array columns caused +/// `rb_encode_nullable` to prepend an extra null-indicator byte. ClickHouse +/// read that byte as `varint(0)` (empty array) and then parsed the actual +/// element bytes as subsequent column data, failing with "Cannot read all data" +/// at row 2. +#[tokio::test(flavor = "multi_thread")] +async fn all_types_table_copy() { + init_test_tracing(); + install_crypto_provider(); + + // --- GIVEN: Postgres source with all supported column types --- + let database = spawn_source_database().await; + let table_name = test_table_name("all_types_encoding"); + + let table_id = database + .create_table( + table_name.clone(), + true, // add serial primary key + &[ + // Scalar types + ("smallint_col", "smallint not null"), + ("integer_col", "integer not null"), + ("bigint_col", "bigint not null"), + ("real_col", "real not null"), + ("double_col", "double precision not null"), + ("numeric_col", "numeric(10,2) not null"), + ("boolean_col", "boolean not null"), + ("text_col", "text not null"), + ("varchar_col", "varchar(100) not null"), + ("date_col", "date not null"), + ("timestamp_col", "timestamp not null"), + ("timestamptz_col", "timestamptz not null"), + ("time_col", "time not null"), + ("interval_col", "interval not null"), + ("jsonb_col", "jsonb not null"), + ("json_col", "json not null"), + // Nullable array columns (key for the regression test). + // These are intentionally nullable so that nullable_flags[i] would + // have been set to `true` before the fix, triggering the bug. + ("integer_array_col", "integer[]"), + ("text_array_col", "text[]"), + // Other types + ("bytea_col", "bytea not null"), + ("inet_col", "inet not null"), + ("cidr_col", "cidr not null"), + ("macaddr_col", "macaddr not null"), + ("uuid_col", "uuid not null"), + ], + ) + .await + .expect("Failed to create test table"); + + let publication_name = "test_pub_clickhouse"; + database + .create_publication(publication_name, std::slice::from_ref(&table_name)) + .await + .expect("Failed to create publication"); + + // Row 1: empty arrays. With the old encoding bug, this accidentally + // produced valid RowBinary because `0x00` (null-indicator) == + // varint(0) == empty array. + database + .run_sql(&format!( + r#"INSERT INTO {table} ( + smallint_col, integer_col, bigint_col, + real_col, double_col, numeric_col, boolean_col, + text_col, varchar_col, + date_col, timestamp_col, timestamptz_col, + time_col, interval_col, jsonb_col, json_col, + integer_array_col, text_array_col, + bytea_col, inet_col, cidr_col, macaddr_col, uuid_col + ) VALUES ( + 42, 1000, 9999999, + 1.5, 2.5, 12345.67, true, + 'hello text', 'hello varchar', + '2024-01-15', '2024-01-15 12:00:00', '2024-01-15 12:00:00+00', + '14:30:00', '1 day', + '{{"key":"value"}}', '{{"simple":42}}', + ARRAY[]::integer[], ARRAY[]::text[], + '\xdeadbeef', + '192.168.1.1', '192.168.0.0/16', 'aa:bb:cc:dd:ee:ff', + 'f47ac10b-58cc-4372-a567-0e02b2c3d479' + )"#, + table = table_name.as_quoted_identifier(), + )) + .await + .expect("Failed to insert row 1"); + + // Row 2: NON-EMPTY arrays. With the old encoding bug, this row caused + // ClickHouse to fail with "Cannot read all data" because the extra + // null-indicator byte caused the entire RowBinary stream to be + // mis-aligned after the array column. + database + .run_sql(&format!( + r#"INSERT INTO {table} ( + smallint_col, integer_col, bigint_col, + real_col, double_col, numeric_col, boolean_col, + text_col, varchar_col, + date_col, timestamp_col, timestamptz_col, + time_col, interval_col, jsonb_col, json_col, + integer_array_col, text_array_col, + bytea_col, inet_col, cidr_col, macaddr_col, uuid_col + ) VALUES ( + -32768, -2147483648, -9223372036854775808, + -1.5, -2.5, -99999.99, false, + 'world text', 'world varchar', + '2024-01-15', '2024-01-15 12:00:00', '2024-01-15 12:00:00+00', + '00:00:01', '30 days 23 hours', + '{{"arr":[1,2,3]}}', '{{"n":0}}', + ARRAY[1, 2, 3]::integer[], ARRAY['alpha', 'beta']::text[], + '\xcafebabe', + '10.0.0.1', '10.0.0.0/8', 'ff:ee:dd:cc:bb:aa', + 'a1b2c3d4-e5f6-7890-abcd-ef1234567890' + )"#, + table = table_name.as_quoted_identifier(), + )) + .await + .expect("Failed to insert row 2"); + + // --- WHEN: pipeline copies data to ClickHouse --- + let clickhouse_db = setup_clickhouse_database().await; + let store = NotifyingStore::new(); + let pipeline_id: PipelineId = random(); + let destination = clickhouse_db.build_destination(store.clone()); + + let table_ready = + store.notify_on_table_state_type(table_id, TableReplicationPhaseType::Ready).await; + + let mut pipeline = create_pipeline( + &database.config, + pipeline_id, + publication_name.to_owned(), + store.clone(), + destination, + ); + + pipeline.start().await.unwrap(); + table_ready.notified().await; + pipeline.shutdown_and_wait().await.unwrap(); + + // --- THEN: every column round-trips correctly --- + let rows: Vec = clickhouse_db.query(ALL_TYPES_SELECT).await; + + assert_eq!(rows.len(), 2, "expected 2 rows in ClickHouse"); + + // Row 1: positive/typical values, empty arrays. + let r1 = &rows[0]; + assert_eq!(r1.id, 1); + assert_eq!(r1.smallint_col, 42); + assert_eq!(r1.integer_col, 1000); + assert_eq!(r1.bigint_col, 9_999_999); + assert!((r1.real_col - 1.5_f32).abs() < 1e-3, "real_col mismatch"); + assert!((r1.double_col - 2.5_f64).abs() < 1e-6, "double_col mismatch"); + assert_eq!(r1.numeric_col, "12345.67"); + assert!(r1.boolean_col); + assert_eq!(r1.text_col, "hello text"); + assert_eq!(r1.varchar_col, "hello varchar"); + assert_eq!(r1.date_col, DATE_2024_01_15_DAYS, "date round-trip failed"); + assert_eq!(r1.timestamp_col, TS_2024_01_15_12_00_US, "timestamp round-trip failed"); + assert_eq!(r1.timestamptz_col, TS_2024_01_15_12_00_US, "timestamptz round-trip failed"); + assert_eq!(r1.time_col, "14:30:00"); + assert_eq!(r1.bytea_col, "deadbeef"); + assert_eq!(r1.inet_col, "192.168.1.1"); + assert_eq!(r1.cidr_col, "192.168.0.0/16"); + assert_eq!(r1.macaddr_col, "aa:bb:cc:dd:ee:ff"); + assert_eq!(r1.uuid_col.to_lowercase(), "f47ac10b-58cc-4372-a567-0e02b2c3d479"); + assert_eq!(r1.cdc_operation, "INSERT"); + // Empty arrays -- the regression case that accidentally worked before the fix. + assert_eq!( + r1.integer_array_col, + Vec::>::new(), + "row 1 integer_array_col should be empty" + ); + assert_eq!( + r1.text_array_col, + Vec::>::new(), + "row 1 text_array_col should be empty" + ); + + // Row 2: boundary values, non-empty arrays (the regression case). + let r2 = &rows[1]; + assert_eq!(r2.id, 2); + assert_eq!(r2.smallint_col, -32768); + assert_eq!(r2.integer_col, -2_147_483_648); + assert_eq!(r2.bigint_col, i64::MIN); + assert!(!r2.boolean_col); + assert_eq!(r2.numeric_col, "-99999.99"); + assert_eq!(r2.bytea_col, "cafebabe"); + assert_eq!(r2.uuid_col.to_lowercase(), "a1b2c3d4-e5f6-7890-abcd-ef1234567890"); + assert_eq!(r2.cdc_operation, "INSERT"); + // Non-empty arrays -- the regression case that triggered the bug before the + // fix. + assert_eq!( + r2.integer_array_col, + vec![Some(1), Some(2), Some(3)], + "row 2 integer_array_col mismatch -- nullable-array encoding bug likely present" + ); + assert_eq!( + r2.text_array_col, + vec![Some("alpha".to_owned()), Some("beta".to_owned())], + "row 2 text_array_col mismatch -- nullable-array encoding bug likely present" + ); +} + +/// Tests that UPDATE events are streamed to ClickHouse after initial table +/// copy. +/// +/// # GIVEN +/// +/// A Postgres table with a single row (`id=1, value='before'`). +/// +/// # WHEN +/// +/// The pipeline copies the row, then an `UPDATE ... SET value = 'after'` is +/// issued against Postgres. +/// +/// # THEN +/// +/// ClickHouse contains two rows (append-only CDC): +/// - The original `INSERT` from table copy with `cdc_lsn = 0`. +/// - The streamed `UPDATE` with the new value and a positive LSN. +#[tokio::test(flavor = "multi_thread")] +async fn updates_are_streamed_to_clickhouse() { + init_test_tracing(); + install_crypto_provider(); + + // --- GIVEN: Postgres source with one row --- + let database = spawn_source_database().await; + let table_name = test_table_name("update_flow"); + + let table_id = database + .create_table(table_name.clone(), true, &[("value", "text not null")]) + .await + .expect("Failed to create update_flow test table"); + + let publication_name = "test_pub_clickhouse_updates"; + database + .create_publication(publication_name, std::slice::from_ref(&table_name)) + .await + .expect("Failed to create update_flow publication"); + + database + .run_sql(&format!( + "INSERT INTO {} (value) VALUES ('before')", + table_name.as_quoted_identifier(), + )) + .await + .expect("Failed to insert initial update_flow row"); + + // --- WHEN: pipeline copies data, then an UPDATE is streamed --- + let clickhouse_db = setup_clickhouse_database().await; + let store = NotifyingStore::new(); + let pipeline_id: PipelineId = random(); + let destination = clickhouse_db.build_destination(store.clone()); + + let table_ready = + store.notify_on_table_state_type(table_id, TableReplicationPhaseType::Ready).await; + + let mut pipeline = create_pipeline( + &database.config, + pipeline_id, + publication_name.to_owned(), + store, + destination, + ); + + pipeline.start().await.unwrap(); + table_ready.notified().await; + + database + .run_sql(&format!( + "UPDATE {} SET value = 'after' WHERE id = 1", + table_name.as_quoted_identifier(), + )) + .await + .expect("Failed to update update_flow row"); + + let rows = wait_for_update_flow_rows(&clickhouse_db, 2).await; + + pipeline.shutdown_and_wait().await.unwrap(); + + // --- THEN: one INSERT from table copy, one UPDATE from streaming --- + assert_eq!(rows.len(), 2, "expected copied row plus streamed update"); + + let insert_row = &rows[0]; + assert_eq!(insert_row.id, 1); + assert_eq!(insert_row.value, "before"); + assert_eq!(insert_row.cdc_operation, "INSERT"); + assert_eq!(insert_row.cdc_lsn, 0); + + let update_row = &rows[1]; + assert_eq!(update_row.id, 1); + assert_eq!(update_row.value, "after"); + assert_eq!(update_row.cdc_operation, "UPDATE"); + assert!(update_row.cdc_lsn > insert_row.cdc_lsn, "streamed update should have a positive LSN"); +} + +const BOUNDARY_VALUES_SELECT: &str = concat!( + "SELECT id, nullable_text, nullable_int, ", + "int_array_col, text_array_col, ", + "cdc_operation ", + "FROM \"test_boundary__values\" ", + "ORDER BY id", +); + +/// Tests that edge-case values survive the Postgres -> ClickHouse pipeline +/// without data loss or corruption. +/// +/// # GIVEN +/// +/// A Postgres table with nullable scalar columns and nullable array columns, +/// populated with four rows that exercise encoding boundary conditions: +/// +/// 1. **All NULLs** -- nullable scalars are NULL, arrays are empty. +/// 2. **NULL elements inside arrays** -- `{1, NULL, 3}`, `{'a', NULL, 'c'}`. +/// 3. **Empty strings** -- a present-but-empty text value next to a NULL +/// integer, plus single-element arrays (varint length = 1). +/// 4. **Multi-byte UTF-8** -- emoji and CJK characters, verifying that the +/// RowBinary varint encodes byte length (not character count) correctly. +/// +/// # WHEN +/// +/// The pipeline runs initial table copy from Postgres to ClickHouse. +/// +/// # THEN +/// +/// Every row in ClickHouse exactly matches what was inserted into Postgres: +/// - SQL NULLs remain NULL (not empty string, not zero). +/// - Empty strings remain empty strings (not NULL). +/// - Array elements preserve their position, including interior NULLs. +/// - Multi-byte text round-trips byte-for-byte. +#[tokio::test(flavor = "multi_thread")] +async fn boundary_values_table_copy() { + init_test_tracing(); + install_crypto_provider(); + + // --- GIVEN: Postgres source with boundary-value rows --- + + let database = spawn_source_database().await; + let table_name = test_table_name("boundary_values"); + + let table_id = database + .create_table( + table_name.clone(), + true, + &[ + ("nullable_text", "text"), // nullable + ("nullable_int", "integer"), // nullable + ("int_array_col", "integer[]"), // Array(Nullable(Int32)) + ("text_array_col", "text[]"), // Array(Nullable(String)) + ], + ) + .await + .expect("Failed to create boundary_values table"); + + let publication_name = "test_pub_ch_boundary"; + database + .create_publication(publication_name, std::slice::from_ref(&table_name)) + .await + .expect("Failed to create publication"); + + // Row 1: all nullable columns are NULL, arrays are empty. + database + .run_sql(&format!( + "INSERT INTO {} (nullable_text, nullable_int, int_array_col, text_array_col) VALUES \ + (NULL, NULL, ARRAY[]::integer[], ARRAY[]::text[])", + table_name.as_quoted_identifier(), + )) + .await + .expect("Failed to insert row 1 (all NULLs)"); + + // Row 2: arrays with interior NULL elements -- the element at index 1 is NULL + // while surrounding elements are present. + database + .run_sql(&format!( + "INSERT INTO {} (nullable_text, nullable_int, int_array_col, text_array_col) VALUES \ + ('present', 42, ARRAY[1, NULL, 3]::integer[], ARRAY['a', NULL, 'c']::text[])", + table_name.as_quoted_identifier(), + )) + .await + .expect("Failed to insert row 2 (NULL array elements)"); + + // Row 3: empty string (not NULL) for text, NULL for integer, and + // single-element arrays (varint length byte = 0x01). + database + .run_sql(&format!( + "INSERT INTO {} (nullable_text, nullable_int, int_array_col, text_array_col) VALUES \ + ('', NULL, ARRAY[99]::integer[], ARRAY['only']::text[])", + table_name.as_quoted_identifier(), + )) + .await + .expect("Failed to insert row 3 (empty string + single-element arrays)"); + + // Row 4: multi-byte UTF-8 -- emoji (4 bytes per char) and CJK (3 bytes per + // char). The RowBinary varint encodes byte length, not character count. + database + .run_sql(&format!( + "INSERT INTO {} (nullable_text, nullable_int, int_array_col, text_array_col) VALUES \ + ('hello 🌍🚀', 0, ARRAY[1, 2], ARRAY['日本語', '中文'])", + table_name.as_quoted_identifier(), + )) + .await + .expect("Failed to insert row 4 (multi-byte UTF-8)"); + + // --- WHEN: pipeline copies data to ClickHouse --- + let clickhouse_db = setup_clickhouse_database().await; + let store = NotifyingStore::new(); + let pipeline_id: PipelineId = random(); + let destination = clickhouse_db.build_destination(store.clone()); + + let table_ready = + store.notify_on_table_state_type(table_id, TableReplicationPhaseType::Ready).await; + + let mut pipeline = create_pipeline( + &database.config, + pipeline_id, + publication_name.to_owned(), + store.clone(), + destination, + ); + + pipeline.start().await.unwrap(); + table_ready.notified().await; + pipeline.shutdown_and_wait().await.unwrap(); + + // --- THEN: ClickHouse data matches Postgres exactly --- + let rows: Vec = clickhouse_db.query(BOUNDARY_VALUES_SELECT).await; + assert_eq!(rows.len(), 4, "expected 4 rows in ClickHouse"); + + // Row 1: NULL scalars stay NULL, empty arrays stay empty. + let r = &rows[0]; + assert_eq!(r.nullable_text, None, "NULL text must not become empty string"); + assert_eq!(r.nullable_int, None, "NULL int must not become zero"); + assert!(r.int_array_col.is_empty()); + assert!(r.text_array_col.is_empty()); + + // Row 2: interior NULLs preserved in position. + let r = &rows[1]; + assert_eq!(r.nullable_text.as_deref(), Some("present")); + assert_eq!(r.nullable_int, Some(42)); + assert_eq!( + r.int_array_col, + vec![Some(1), None, Some(3)], + "interior NULL in integer array must be preserved" + ); + assert_eq!( + r.text_array_col, + vec![Some("a".to_owned()), None, Some("c".to_owned())], + "interior NULL in text array must be preserved" + ); + + // Row 3: empty string is distinct from NULL. + let r = &rows[2]; + assert_eq!( + r.nullable_text.as_deref(), + Some(""), + "empty string must round-trip as empty string, not NULL" + ); + assert_eq!(r.nullable_int, None); + assert_eq!(r.int_array_col, vec![Some(99)], "single-element array"); + assert_eq!(r.text_array_col, vec![Some("only".to_owned())], "single-element array"); + + // Row 4: multi-byte UTF-8 preserved byte-for-byte. + let r = &rows[3]; + assert_eq!( + r.nullable_text.as_deref(), + Some("hello 🌍🚀"), + "multi-byte UTF-8 must round-trip exactly" + ); + assert_eq!(r.nullable_int, Some(0), "zero must not become NULL"); + assert_eq!( + r.text_array_col, + vec![Some("日本語".to_owned()), Some("中文".to_owned())], + "multi-byte UTF-8 in arrays must round-trip exactly" + ); +} + +/// Signed day offset from the Unix epoch to ClickHouse `Date32`'s minimum +/// representable date, `1900-01-01`. +/// +/// Python: `(date(1900, 1, 1) - date(1970, 1, 1)).days` = -25567. +const DATE32_MIN_DAYS_FROM_UNIX_EPOCH: i32 = -25567; + +/// Signed day offset from the Unix epoch to ClickHouse `Date32`'s maximum +/// representable date, `2299-12-31`. +/// +/// Python: `(date(2299, 12, 31) - date(1970, 1, 1)).days` = 120529. +const DATE32_MAX_DAYS_FROM_UNIX_EPOCH: i32 = 120529; + +const DATE_BOUNDARIES_SELECT: &str = concat!( + "SELECT id, date_col, cdc_operation ", + "FROM \"test_date__boundaries\" ", + "ORDER BY id", +); + +/// Tests that Postgres `date` values outside the Unix epoch (pre-1970 and +/// far-future) round-trip through ClickHouse `Date32` as signed day offsets, +/// rather than being silently clamped as the previous `Date` (UInt16) mapping +/// would have done. +/// +/// # GIVEN +/// +/// A Postgres table with a single non-null `date` column populated with the +/// `Date32` boundary values: `1900-01-01` (minimum), `1969-12-31` (just before +/// epoch), `1970-01-01` (epoch), `2024-01-15` (typical), and `2299-12-31` +/// (maximum). +/// +/// # WHEN +/// +/// The pipeline copies the rows to ClickHouse. +/// +/// # THEN +/// +/// Each `date_col` lands in ClickHouse as the matching signed day offset: +/// negative for pre-1970 dates, zero at the epoch, and a large positive value +/// for the far-future date. No value is clamped. +#[tokio::test(flavor = "multi_thread")] +async fn pre_1970_and_far_future_dates_round_trip() { + init_test_tracing(); + install_crypto_provider(); + + // --- GIVEN: Postgres source with date boundary rows --- + + let database = spawn_source_database().await; + let table_name = test_table_name("date_boundaries"); + + let table_id = database + .create_table(table_name.clone(), true, &[("date_col", "date not null")]) + .await + .expect("Failed to create date_boundaries table"); + + let publication_name = "test_pub_ch_date_boundaries"; + database + .create_publication(publication_name, std::slice::from_ref(&table_name)) + .await + .expect("Failed to create publication"); + + for date_literal in ["1900-01-01", "1969-12-31", "1970-01-01", "2024-01-15", "2299-12-31"] { + database + .run_sql(&format!( + "INSERT INTO {} (date_col) VALUES (DATE '{}')", + table_name.as_quoted_identifier(), + date_literal, + )) + .await + .unwrap_or_else(|_| panic!("Failed to insert {date_literal}")); + } + + // --- WHEN: pipeline copies data to ClickHouse --- + let clickhouse_db = setup_clickhouse_database().await; + let store = NotifyingStore::new(); + let pipeline_id: PipelineId = random(); + let destination = clickhouse_db.build_destination(store.clone()); + + let table_ready = + store.notify_on_table_state_type(table_id, TableReplicationPhaseType::Ready).await; + + let mut pipeline = create_pipeline( + &database.config, + pipeline_id, + publication_name.to_owned(), + store.clone(), + destination, + ); + + pipeline.start().await.unwrap(); + table_ready.notified().await; + pipeline.shutdown_and_wait().await.unwrap(); + + // --- THEN: each date encodes as the expected signed day offset --- + let rows: Vec = clickhouse_db.query(DATE_BOUNDARIES_SELECT).await; + assert_eq!(rows.len(), 5, "expected 5 rows in ClickHouse"); + + assert_eq!( + rows[0].date_col, DATE32_MIN_DAYS_FROM_UNIX_EPOCH, + "1900-01-01 must encode as the Date32 minimum offset, not be clamped" + ); + assert_eq!(rows[1].date_col, -1, "1969-12-31 must encode as one day before the Unix epoch"); + assert_eq!(rows[2].date_col, 0, "1970-01-01 must encode as 0 (the Unix epoch)"); + assert_eq!( + rows[3].date_col, DATE_2024_01_15_DAYS, + "2024-01-15 must encode as 19737 (typical post-epoch value)" + ); + assert_eq!( + rows[4].date_col, DATE32_MAX_DAYS_FROM_UNIX_EPOCH, + "2299-12-31 must encode as the Date32 maximum offset, not be clamped" + ); +} + +/// Tests that DELETE events are streamed to ClickHouse after initial table +/// copy. +/// +/// # GIVEN +/// +/// A Postgres table with `REPLICA IDENTITY FULL` (so Postgres sends all column +/// values in DELETE events, not just the primary key), populated with two rows: +/// +/// 1. `id=1, value='keep_me'` -- will remain untouched. +/// 2. `id=2, value='delete_me'` -- will be deleted after table copy. +/// +/// # WHEN +/// +/// The pipeline copies both rows, then a `DELETE ... WHERE id = 2` is issued +/// against Postgres. +/// +/// # THEN +/// +/// ClickHouse contains three rows (append-only CDC): +/// - Two `INSERT` rows from the initial table copy (`cdc_lsn = 0`). +/// - One `DELETE` row for `id=2` with the old row data preserved and a positive +/// LSN. +/// - The `id=1` row has no corresponding `DELETE`. +#[tokio::test(flavor = "multi_thread")] +async fn deletes_are_streamed_to_clickhouse() { + init_test_tracing(); + install_crypto_provider(); + + // --- GIVEN: Postgres source with two rows, REPLICA IDENTITY FULL --- + + let database = spawn_source_database().await; + let table_name = test_table_name("delete_flow"); + + let table_id = database + .create_table(table_name.clone(), true, &[("value", "text not null")]) + .await + .expect("Failed to create delete_flow test table"); + + database + .run_sql(&format!( + "ALTER TABLE {} REPLICA IDENTITY FULL", + table_name.as_quoted_identifier(), + )) + .await + .expect("Failed to set replica identity full"); + + let publication_name = "test_pub_clickhouse_deletes"; + database + .create_publication(publication_name, std::slice::from_ref(&table_name)) + .await + .expect("Failed to create delete_flow publication"); + + database + .run_sql(&format!( + "INSERT INTO {} (value) VALUES ('keep_me'), ('delete_me')", + table_name.as_quoted_identifier(), + )) + .await + .expect("Failed to insert delete_flow rows"); + + // --- WHEN: pipeline copies data, then a DELETE is streamed --- + + let clickhouse_db = setup_clickhouse_database().await; + let store = NotifyingStore::new(); + let pipeline_id: PipelineId = random(); + let destination = clickhouse_db.build_destination(store.clone()); + + let table_ready = + store.notify_on_table_state_type(table_id, TableReplicationPhaseType::Ready).await; + + let mut pipeline = create_pipeline( + &database.config, + pipeline_id, + publication_name.to_owned(), + store, + destination, + ); + + pipeline.start().await.unwrap(); + table_ready.notified().await; + + database + .run_sql(&format!("DELETE FROM {} WHERE id = 2", table_name.as_quoted_identifier(),)) + .await + .expect("Failed to delete delete_flow row"); + + let rows = wait_for_delete_flow_rows(&clickhouse_db, 3).await; + + pipeline.shutdown_and_wait().await.unwrap(); + + // --- THEN: two INSERTs from table copy, one DELETE from streaming --- + + assert_eq!(rows.len(), 3, "expected 2 copied rows plus 1 streamed delete"); + + // Row 1: copied, untouched. + let r = &rows[0]; + assert_eq!(r.id, 1); + assert_eq!(r.value, "keep_me"); + assert_eq!(r.cdc_operation, "INSERT"); + assert_eq!(r.cdc_lsn, 0); + + // Row 2: copied, then deleted. + let r = &rows[1]; + assert_eq!(r.id, 2); + assert_eq!(r.value, "delete_me"); + assert_eq!(r.cdc_operation, "INSERT"); + assert_eq!(r.cdc_lsn, 0); + + // Row 3: the streamed DELETE for id=2, preserving old row data. + let r = &rows[2]; + assert_eq!(r.id, 2, "delete must target the correct row"); + assert_eq!(r.value, "delete_me", "old row data must be preserved in DELETE"); + assert_eq!(r.cdc_operation, "DELETE"); + assert!(r.cdc_lsn > 0, "streamed delete should have a positive LSN"); +} + +/// Tests that a pipeline restart resumes CDC streaming without re-running +/// the initial table copy. +/// +/// # GIVEN +/// +/// A Postgres table with one row (`id=1, value='before_restart'`), copied +/// to ClickHouse by a first pipeline run that then shuts down cleanly. +/// +/// # WHEN +/// +/// A new `ClickHouseDestination` and `Pipeline` are built with the same +/// store and pipeline_id (simulating process restart), the pipeline is +/// started, and a second row (`id=2, value='after_restart'`) is inserted +/// into Postgres. +/// +/// # THEN +/// +/// ClickHouse contains exactly two rows: +/// - `id=1` from the initial table copy (`cdc_lsn = 0`). +/// - `id=2` from CDC streaming in the second run (`cdc_lsn > 0`). +/// No duplicate `id=1` row exists -- table copy must not re-run. +#[tokio::test(flavor = "multi_thread")] +async fn pipeline_restart_resumes_streaming() { + init_test_tracing(); + install_crypto_provider(); + + // --- GIVEN: first pipeline run copies one row --- + let database = spawn_source_database().await; + let table_name = test_table_name("restart_flow"); + + let table_id = database + .create_table(table_name.clone(), true, &[("value", "text not null")]) + .await + .expect("Failed to create restart_flow test table"); + + let publication_name = "test_pub_clickhouse_restart"; + database + .create_publication(publication_name, std::slice::from_ref(&table_name)) + .await + .expect("Failed to create restart_flow publication"); + + database + .run_sql(&format!( + "INSERT INTO {} (value) VALUES ('before_restart')", + table_name.as_quoted_identifier(), + )) + .await + .expect("Failed to insert initial restart_flow row"); + + let clickhouse_db = setup_clickhouse_database().await; + let store = NotifyingStore::new(); + let pipeline_id: PipelineId = random(); + let destination = clickhouse_db.build_destination(store.clone()); + + let table_ready = + store.notify_on_table_state_type(table_id, TableReplicationPhaseType::Ready).await; + + let mut pipeline = create_pipeline( + &database.config, + pipeline_id, + publication_name.to_owned(), + store.clone(), + destination, + ); + + pipeline.start().await.unwrap(); + table_ready.notified().await; + pipeline.shutdown_and_wait().await.unwrap(); + + // Verify first run produced exactly one row. + let rows: Vec = clickhouse_db.query(RESTART_FLOW_SELECT).await; + assert_eq!(rows.len(), 1, "first run should copy exactly one row"); + assert_eq!(rows[0].id, 1); + assert_eq!(rows[0].value, "before_restart"); + + // --- WHEN: rebuild destination and pipeline, then stream a new insert --- + let destination = clickhouse_db.build_destination(store.clone()); + + let mut pipeline = create_pipeline( + &database.config, + pipeline_id, + publication_name.to_owned(), + store, + destination, + ); + + pipeline.start().await.unwrap(); + + database + .run_sql(&format!( + "INSERT INTO {} (value) VALUES ('after_restart')", + table_name.as_quoted_identifier(), + )) + .await + .expect("Failed to insert post-restart row"); + + let rows = wait_for_restart_flow_rows(&clickhouse_db, 2).await; + + pipeline.shutdown_and_wait().await.unwrap(); + + // --- THEN: exactly two rows, no duplicate from re-running table copy --- + assert_eq!( + rows.len(), + 2, + "expected original copied row plus one streamed insert, no duplicates" + ); + + let r = &rows[0]; + assert_eq!(r.id, 1); + assert_eq!(r.value, "before_restart"); + assert_eq!(r.cdc_operation, "INSERT"); + assert_eq!(r.cdc_lsn, 0, "first row should be from table copy"); + + let r = &rows[1]; + assert_eq!(r.id, 2); + assert_eq!(r.value, "after_restart"); + assert_eq!(r.cdc_operation, "INSERT"); + assert!(r.cdc_lsn > 0, "second row should be from CDC streaming after restart"); +} + +/// Tests that TRUNCATE clears the ClickHouse table and that subsequent inserts +/// produce a clean slate with only post-truncate data. +/// +/// # GIVEN +/// +/// A Postgres table with two rows (`id=1, value='alpha'` and `id=2, +/// value='beta'`), copied to ClickHouse by the initial table copy. +/// +/// # WHEN +/// +/// 1. Postgres issues `TRUNCATE` on the table. +/// 2. After the table becomes empty in ClickHouse, a new row (`id=3, +/// value='gamma'`) is inserted into Postgres. +/// +/// # THEN +/// +/// After truncate, ClickHouse contains zero rows. +/// After the post-truncate insert, ClickHouse contains exactly one row: +/// - `id=3, value='gamma', cdc_operation='INSERT', cdc_lsn > 0`. +/// No pre-truncate rows survive. +#[tokio::test(flavor = "multi_thread")] +async fn truncate_clears_table_and_accepts_new_inserts() { + init_test_tracing(); + install_crypto_provider(); + + // --- GIVEN: two rows copied to ClickHouse --- + let database = spawn_source_database().await; + let table_name = test_table_name("truncate_flow"); + + let table_id = database + .create_table(table_name.clone(), true, &[("value", "text not null")]) + .await + .expect("Failed to create truncate_flow test table"); + + let publication_name = "test_pub_clickhouse_truncate"; + database + .create_publication(publication_name, std::slice::from_ref(&table_name)) + .await + .expect("Failed to create truncate_flow publication"); + + database + .run_sql(&format!( + "INSERT INTO {} (value) VALUES ('alpha'), ('beta')", + table_name.as_quoted_identifier(), + )) + .await + .expect("Failed to insert truncate_flow rows"); + + let clickhouse_db = setup_clickhouse_database().await; + let store = NotifyingStore::new(); + let pipeline_id: PipelineId = random(); + let destination = clickhouse_db.build_destination(store.clone()); + + let table_ready = + store.notify_on_table_state_type(table_id, TableReplicationPhaseType::Ready).await; + + let mut pipeline = create_pipeline( + &database.config, + pipeline_id, + publication_name.to_owned(), + store, + destination, + ); + + pipeline.start().await.unwrap(); + table_ready.notified().await; + + // Verify both rows arrived from table copy. + let rows: Vec = clickhouse_db.query(TRUNCATE_FLOW_SELECT).await; + assert_eq!(rows.len(), 2, "table copy should produce two rows"); + + // --- WHEN: truncate, then insert a new row --- + database + .truncate_table(table_name.clone()) + .await + .expect("Failed to truncate table in Postgres"); + + wait_for_truncate_flow_empty(&clickhouse_db).await; + + database + .run_sql(&format!( + "INSERT INTO {} (value) VALUES ('gamma')", + table_name.as_quoted_identifier(), + )) + .await + .expect("Failed to insert post-truncate row"); + + let rows = wait_for_truncate_flow_rows(&clickhouse_db, 1).await; + + pipeline.shutdown_and_wait().await.unwrap(); + + // --- THEN: only the post-truncate row exists --- + assert_eq!(rows.len(), 1, "only post-truncate row should exist"); + + let r = &rows[0]; + assert_eq!(r.id, 3, "post-truncate row should have id=3 (serial continues)"); + assert_eq!(r.value, "gamma"); + assert_eq!(r.cdc_operation, "INSERT"); + assert!(r.cdc_lsn > 0, "post-truncate insert should come from CDC streaming"); +} + +/// SELECT query used to verify the `flush_split` test. +const FLUSH_SPLIT_SELECT: &str = concat!( + "SELECT id, value, cdc_operation, cdc_lsn ", + "FROM \"test_flush__split\" ", + "ORDER BY id, cdc_lsn", +); + +/// Tests that the intermediate INSERT flush (`max_bytes_per_insert`) does not +/// lose rows when a batch is split across multiple INSERT statements. +/// +/// # GIVEN +/// +/// A Postgres table with 10 rows, and a ClickHouse destination configured with +/// `max_bytes_per_insert = 1` (forcing a new INSERT after every single row). +/// +/// # WHEN +/// +/// The pipeline runs initial table copy from Postgres to ClickHouse. +/// +/// # THEN +/// +/// All 10 rows arrive in ClickHouse despite being split across many INSERT +/// statements. No rows are lost at flush boundaries. +#[tokio::test(flavor = "multi_thread")] +async fn intermediate_flush_preserves_all_rows() { + init_test_tracing(); + install_crypto_provider(); + + // --- GIVEN: 10 rows and a tiny max_bytes_per_insert --- + let database = spawn_source_database().await; + let table_name = test_table_name("flush_split"); + + let table_id = database + .create_table(table_name.clone(), true, &[("value", "text not null")]) + .await + .expect("Failed to create flush_split test table"); + + let publication_name = "test_pub_clickhouse_flush"; + database + .create_publication(publication_name, std::slice::from_ref(&table_name)) + .await + .expect("Failed to create flush_split publication"); + + let row_count = 10; + let values: Vec = (1..=row_count).map(|i| format!("('row_{i}')")).collect(); + database + .run_sql(&format!( + "INSERT INTO {} (value) VALUES {}", + table_name.as_quoted_identifier(), + values.join(", "), + )) + .await + .expect("Failed to insert flush_split rows"); + + let clickhouse_db = setup_clickhouse_database().await; + let store = NotifyingStore::new(); + let pipeline_id: PipelineId = random(); + let destination = clickhouse_db.build_destination_with_config( + store.clone(), + ClickHouseInserterConfig { + // 1 byte -- forces a new INSERT after every row. + max_bytes_per_insert: 1, + }, + ); + + let table_ready = + store.notify_on_table_state_type(table_id, TableReplicationPhaseType::Ready).await; + + // --- WHEN: pipeline copies data with aggressive flush splitting --- + let mut pipeline = create_pipeline( + &database.config, + pipeline_id, + publication_name.to_owned(), + store, + destination, + ); + + pipeline.start().await.unwrap(); + table_ready.notified().await; + pipeline.shutdown_and_wait().await.unwrap(); + + // --- THEN: all rows arrive despite being split across many INSERTs --- + let rows: Vec = clickhouse_db.query(FLUSH_SPLIT_SELECT).await; + assert_eq!(rows.len(), row_count, "all rows must survive intermediate flush splits"); + + for (i, r) in rows.iter().enumerate() { + let expected_id = (i + 1) as i64; + let expected_value = format!("row_{}", i + 1); + assert_eq!(r.id, expected_id, "row {} id mismatch", i + 1); + assert_eq!(r.value, expected_value, "row {} value mismatch", i + 1); + assert_eq!(r.cdc_operation, "INSERT"); + assert_eq!(r.cdc_lsn, 0, "all rows should be from table copy"); + } +} + +/// Tests that CDC events for multiple tables in the same publication are +/// written correctly to separate ClickHouse tables. +/// +/// # GIVEN +/// +/// Two Postgres tables in the same publication, each with one pre-existing +/// row: +/// - `multi_a` with `(id=1, value='init_a')` +/// - `multi_b` with `(id=1, value='init_b')` +/// +/// # WHEN +/// +/// The pipeline copies both tables, then one new row is inserted into each +/// Postgres table (`'streamed_a'` and `'streamed_b'`). +/// +/// # THEN +/// +/// Each ClickHouse table contains exactly two rows: +/// - One from table copy (`cdc_lsn = 0`) +/// - One from CDC streaming (`cdc_lsn > 0`) +#[tokio::test(flavor = "multi_thread")] +async fn multiple_tables_receive_independent_writes() { + init_test_tracing(); + install_crypto_provider(); + + // --- GIVEN: two tables in one publication, each with one row --- + let database = spawn_source_database().await; + let table_a = test_table_name("multi_a"); + let table_b = test_table_name("multi_b"); + + let table_a_id = database + .create_table(table_a.clone(), true, &[("value", "text not null")]) + .await + .expect("Failed to create multi_a table"); + + let table_b_id = database + .create_table(table_b.clone(), true, &[("value", "text not null")]) + .await + .expect("Failed to create multi_b table"); + + let publication_name = "test_pub_clickhouse_multi"; + database + .create_publication(publication_name, &[table_a.clone(), table_b.clone()]) + .await + .expect("Failed to create multi-table publication"); + + database + .run_sql(&format!( + "INSERT INTO {} (value) VALUES ('init_a')", + table_a.as_quoted_identifier(), + )) + .await + .expect("Failed to insert into multi_a"); + + database + .run_sql(&format!( + "INSERT INTO {} (value) VALUES ('init_b')", + table_b.as_quoted_identifier(), + )) + .await + .expect("Failed to insert into multi_b"); + + let clickhouse_db = setup_clickhouse_database().await; + let store = NotifyingStore::new(); + let pipeline_id: PipelineId = random(); + let destination = clickhouse_db.build_destination(store.clone()); + + let table_a_ready = + store.notify_on_table_state_type(table_a_id, TableReplicationPhaseType::Ready).await; + let table_b_ready = + store.notify_on_table_state_type(table_b_id, TableReplicationPhaseType::Ready).await; + + let mut pipeline = create_pipeline( + &database.config, + pipeline_id, + publication_name.to_owned(), + store, + destination, + ); + + pipeline.start().await.unwrap(); + tokio::join!(table_a_ready.notified(), table_b_ready.notified()); + + // --- WHEN: insert one row into each table --- + database + .run_sql(&format!( + "INSERT INTO {} (value) VALUES ('streamed_a')", + table_a.as_quoted_identifier(), + )) + .await + .expect("Failed to insert streamed row into multi_a"); + + database + .run_sql(&format!( + "INSERT INTO {} (value) VALUES ('streamed_b')", + table_b.as_quoted_identifier(), + )) + .await + .expect("Failed to insert streamed row into multi_b"); + + let select_a = concat!( + "SELECT id, value, cdc_operation, cdc_lsn ", + "FROM \"test_multi__a\" ", + "ORDER BY id, cdc_lsn", + ); + let select_b = concat!( + "SELECT id, value, cdc_operation, cdc_lsn ", + "FROM \"test_multi__b\" ", + "ORDER BY id, cdc_lsn", + ); + + // Poll until both tables have 2 rows. + let mut rows_a: Vec = Vec::with_capacity(2); + let mut rows_b: Vec = Vec::with_capacity(2); + for _ in 0..50 { + rows_a = clickhouse_db.query(select_a).await; + rows_b = clickhouse_db.query(select_b).await; + if rows_a.len() >= 2 && rows_b.len() >= 2 { + break; + } + sleep(Duration::from_millis(100)).await; + } + assert!( + rows_a.len() >= 2 && rows_b.len() >= 2, + "timed out: multi_a has {} rows, multi_b has {} rows", + rows_a.len(), + rows_b.len() + ); + + pipeline.shutdown_and_wait().await.unwrap(); + + // --- THEN: each table has one copied row and one streamed row --- + assert_eq!(rows_a.len(), 2, "multi_a should have 2 rows"); + assert_eq!(rows_b.len(), 2, "multi_b should have 2 rows"); + + assert_eq!(rows_a[0].id, 1); + assert_eq!(rows_a[0].value, "init_a"); + assert_eq!(rows_a[0].cdc_operation, "INSERT"); + assert_eq!(rows_a[0].cdc_lsn, 0); + + assert_eq!(rows_a[1].id, 2); + assert_eq!(rows_a[1].value, "streamed_a"); + assert_eq!(rows_a[1].cdc_operation, "INSERT"); + assert!(rows_a[1].cdc_lsn > 0); + + assert_eq!(rows_b[0].id, 1); + assert_eq!(rows_b[0].value, "init_b"); + assert_eq!(rows_b[0].cdc_operation, "INSERT"); + assert_eq!(rows_b[0].cdc_lsn, 0); + + assert_eq!(rows_b[1].id, 2); + assert_eq!(rows_b[1].value, "streamed_b"); + assert_eq!(rows_b[1].cdc_operation, "INSERT"); + assert!(rows_b[1].cdc_lsn > 0); +} + +/// SELECT query used to verify the `tx_order` test. +const TX_ORDER_SELECT: &str = concat!( + "SELECT id, value, cdc_operation, cdc_lsn ", + "FROM \"test_tx__order\" ", + "ORDER BY id, cdc_lsn", +); + +/// Tests that updates from separately committed transactions arrive in +/// ClickHouse with LSNs reflecting Postgres commit order. +/// +/// # GIVEN +/// +/// A Postgres table with one row (`id=1, value='original'`), copied to +/// ClickHouse. Two database connections to the same source. +/// +/// # WHEN +/// +/// Transaction A (on connection 1) updates the row to `'update_a'` and +/// commits. Then transaction B (on connection 2) updates the row to +/// `'update_b'` and commits. +/// +/// # THEN +/// +/// ClickHouse contains three rows (append-only CDC) with strictly +/// increasing `cdc_lsn`: +/// - `value='original'`, `cdc_operation='INSERT'`, `cdc_lsn=0` +/// - `value='update_a'`, `cdc_operation='UPDATE'`, `cdc_lsn > 0` +/// - `value='update_b'`, `cdc_operation='UPDATE'`, `cdc_lsn > update_a's lsn` +#[tokio::test(flavor = "multi_thread")] +async fn sequential_transactions_preserve_commit_order() { + init_test_tracing(); + install_crypto_provider(); + + // --- GIVEN: one row, two database connections --- + let mut database_1 = spawn_source_database().await; + let mut database_2 = database_1.duplicate().await; + let table_name = test_table_name("tx_order"); + + let table_id = database_1 + .create_table(table_name.clone(), true, &[("value", "text not null")]) + .await + .expect("Failed to create tx_order test table"); + + let publication_name = "test_pub_clickhouse_tx_order"; + database_1 + .create_publication(publication_name, std::slice::from_ref(&table_name)) + .await + .expect("Failed to create tx_order publication"); + + database_1 + .run_sql(&format!( + "INSERT INTO {} (value) VALUES ('original')", + table_name.as_quoted_identifier(), + )) + .await + .expect("Failed to insert initial tx_order row"); + + let clickhouse_db = setup_clickhouse_database().await; + let store = NotifyingStore::new(); + let pipeline_id: PipelineId = random(); + let destination = clickhouse_db.build_destination(store.clone()); + + let table_ready = + store.notify_on_table_state_type(table_id, TableReplicationPhaseType::Ready).await; + + let mut pipeline = create_pipeline( + &database_1.config, + pipeline_id, + publication_name.to_owned(), + store, + destination, + ); + + pipeline.start().await.unwrap(); + table_ready.notified().await; + + // --- WHEN: two transactions commit sequentially on separate connections --- + let tx_a = database_1.begin_transaction().await; + tx_a.run_sql(&format!( + "UPDATE {} SET value = 'update_a' WHERE id = 1", + table_name.as_quoted_identifier(), + )) + .await + .expect("Failed to execute update_a"); + tx_a.commit_transaction().await; + + let tx_b = database_2.begin_transaction().await; + tx_b.run_sql(&format!( + "UPDATE {} SET value = 'update_b' WHERE id = 1", + table_name.as_quoted_identifier(), + )) + .await + .expect("Failed to execute update_b"); + tx_b.commit_transaction().await; + + // Poll until all three rows arrive. + let mut rows: Vec = Vec::with_capacity(3); + for _ in 0..50 { + rows = clickhouse_db.query(TX_ORDER_SELECT).await; + if rows.len() >= 3 { + break; + } + sleep(Duration::from_millis(100)).await; + } + assert!(rows.len() >= 3, "timed out waiting for tx_order rows: got {} of 3", rows.len()); + + pipeline.shutdown_and_wait().await.unwrap(); + + // --- THEN: three rows with strictly increasing LSNs --- + assert_eq!(rows.len(), 3, "expected INSERT + two UPDATEs"); + + let r = &rows[0]; + assert_eq!(r.value, "original"); + assert_eq!(r.cdc_operation, "INSERT"); + assert_eq!(r.cdc_lsn, 0); + + let r = &rows[1]; + assert_eq!(r.value, "update_a"); + assert_eq!(r.cdc_operation, "UPDATE"); + assert!(r.cdc_lsn > 0); + + let r = &rows[2]; + assert_eq!(r.value, "update_b"); + assert_eq!(r.cdc_operation, "UPDATE"); + assert!(r.cdc_lsn > rows[1].cdc_lsn, "update_b must have a higher LSN than update_a"); +} + +/// Row struct for the wide default-identity delete test. +#[derive(clickhouse::Row, serde::Deserialize, Debug)] +struct DefaultIdentityDeleteRow { + id: i64, + smallint_col: i16, + integer_col: i32, + bigint_col: i64, + real_col: f32, + double_col: f64, + numeric_col: String, + boolean_col: bool, + text_col: String, + varchar_col: String, + date_col: i32, + timestamp_col: i64, + timestamptz_col: i64, + time_col: String, + jsonb_col: String, + bytea_col: String, + uuid_col: String, + nullable_text: Option, + nullable_int: Option, + int_array_col: Vec>, + text_array_col: Vec>, + cdc_operation: String, + cdc_lsn: u64, +} + +const DEFAULT_IDENTITY_DELETE_SELECT: &str = concat!( + "SELECT id, ", + "smallint_col, integer_col, bigint_col, real_col, double_col, ", + "numeric_col, boolean_col, text_col, varchar_col, ", + "date_col, timestamp_col, timestamptz_col, time_col, ", + "jsonb_col, bytea_col, toString(uuid_col) AS uuid_col, ", + "nullable_text, nullable_int, ", + "int_array_col, text_array_col, ", + "cdc_operation, cdc_lsn ", + "FROM \"test_default__identity__delete\" ", + "ORDER BY id, cdc_lsn", +); + +/// Tests that a DELETE under default replica identity (PK only) produces a +/// tombstone row with the correct PK and zero-value defaults for every +/// supported column type. +/// +/// # GIVEN +/// +/// A wide Postgres table with **default replica identity** (not FULL) covering: +/// - Non-nullable scalars: smallint, integer, bigint, real, double, numeric, +/// boolean, text, varchar, date, timestamp, timestamptz, time, jsonb, bytea, +/// uuid +/// - Nullable scalars: text, integer +/// - Nullable arrays: integer[], text[] +/// Two rows inserted and copied to ClickHouse. +/// +/// # WHEN +/// +/// Row `id=2` is deleted in Postgres, and a new row (`id=3`) is inserted. +/// +/// # THEN +/// +/// The DELETE tombstone has: +/// - Correct PK (`id=2`) +/// - Non-nullable scalars filled with type-appropriate zero values +/// - Nullable scalars filled with NULL +/// - Arrays filled with empty arrays +#[tokio::test(flavor = "multi_thread")] +async fn delete_with_default_replica_identity() { + init_test_tracing(); + install_crypto_provider(); + + // --- GIVEN: wide table with all types, default replica identity --- + let database = spawn_source_database().await; + let table_name = test_table_name("default_identity_delete"); + + let table_id = database + .create_table( + table_name.clone(), + true, + &[ + ("smallint_col", "smallint not null"), + ("integer_col", "integer not null"), + ("bigint_col", "bigint not null"), + ("real_col", "real not null"), + ("double_col", "double precision not null"), + ("numeric_col", "numeric(10,2) not null"), + ("boolean_col", "boolean not null"), + ("text_col", "text not null"), + ("varchar_col", "varchar(100) not null"), + ("date_col", "date not null"), + ("timestamp_col", "timestamp not null"), + ("timestamptz_col", "timestamptz not null"), + ("time_col", "time not null"), + ("jsonb_col", "jsonb not null"), + ("bytea_col", "bytea not null"), + ("uuid_col", "uuid not null"), + ("nullable_text", "text"), + ("nullable_int", "integer"), + ("int_array_col", "integer[]"), + ("text_array_col", "text[]"), + ], + ) + .await + .expect("Failed to create test table"); + + // Deliberately NOT setting REPLICA IDENTITY FULL -- default (PK only). + + let publication_name = "test_pub_ch_default_ident"; + database + .create_publication(publication_name, std::slice::from_ref(&table_name)) + .await + .expect("Failed to create publication"); + + database + .run_sql(&format!( + r#"INSERT INTO {table} ( + smallint_col, integer_col, bigint_col, real_col, double_col, + numeric_col, boolean_col, text_col, varchar_col, + date_col, timestamp_col, timestamptz_col, time_col, + jsonb_col, bytea_col, uuid_col, + nullable_text, nullable_int, int_array_col, text_array_col + ) VALUES + (1, 10, 100, 1.5, 2.5, 123.45, true, + 'keep', 'keeper', '2024-01-15', '2024-01-15 12:00:00', + '2024-01-15 12:00:00+00', '14:30:00', + '{{"key":"value"}}', '\xdeadbeef', + 'f47ac10b-58cc-4372-a567-0e02b2c3d479', + 'present', 42, ARRAY[1,2,3]::integer[], ARRAY['a','b']::text[]), + (2, 20, 200, 3.5, 4.5, 678.90, false, + 'delete_me', 'doomed', '2024-06-01', '2024-06-01 08:00:00', + '2024-06-01 08:00:00+00', '09:00:00', + '{{"x":1}}', '\xcafebabe', + 'a1b2c3d4-e5f6-7890-abcd-ef1234567890', + 'also_present', 99, ARRAY[4,5]::integer[], ARRAY['c']::text[])"#, + table = table_name.as_quoted_identifier(), + )) + .await + .expect("Failed to insert rows"); + + let clickhouse_db = setup_clickhouse_database().await; + let store = NotifyingStore::new(); + let pipeline_id: PipelineId = random(); + let destination = clickhouse_db.build_destination(store.clone()); + + let table_ready = + store.notify_on_table_state_type(table_id, TableReplicationPhaseType::Ready).await; + + let mut pipeline = create_pipeline( + &database.config, + pipeline_id, + publication_name.to_owned(), + store, + destination, + ); + + pipeline.start().await.unwrap(); + table_ready.notified().await; + + // --- WHEN: delete id=2, insert id=3 --- + database + .run_sql(&format!("DELETE FROM {} WHERE id = 2", table_name.as_quoted_identifier())) + .await + .expect("Failed to delete row"); + + database + .run_sql(&format!( + r#"INSERT INTO {table} ( + smallint_col, integer_col, bigint_col, real_col, double_col, + numeric_col, boolean_col, text_col, varchar_col, + date_col, timestamp_col, timestamptz_col, time_col, + jsonb_col, bytea_col, uuid_col, + int_array_col, text_array_col + ) VALUES ( + 3, 30, 300, 5.5, 6.5, 111.11, true, + 'after', 'survivor', '2025-01-01', '2025-01-01 00:00:00', + '2025-01-01 00:00:00+00', '00:00:00', + '{{"new":true}}', '\x00', + 'b2c3d4e5-f6a7-8901-bcde-f12345678901', + ARRAY[7]::integer[], ARRAY['d']::text[])"#, + table = table_name.as_quoted_identifier(), + )) + .await + .expect("Failed to insert post-delete row"); + + // Poll for 4 rows: 2 copied INSERTs + DELETE tombstone + new INSERT. + let mut rows: Vec = Vec::new(); + for _ in 0..50 { + rows = clickhouse_db.query(DEFAULT_IDENTITY_DELETE_SELECT).await; + if rows.len() >= 4 { + break; + } + sleep(Duration::from_millis(100)).await; + } + assert!( + rows.len() >= 4, + "timed out waiting for default_identity_delete rows: got {} of 4", + rows.len() + ); + + pipeline.shutdown_and_wait().await.unwrap(); + + // --- THEN: DELETE tombstone has zero-value defaults for all types --- + assert_eq!(rows.len(), 4, "expected 2 copied INSERTs + DELETE + new INSERT"); + + // Row 1: copied, untouched -- spot check. + let r = &rows[0]; + assert_eq!(r.id, 1); + assert_eq!(r.text_col, "keep"); + assert_eq!(r.integer_col, 10); + assert!(r.boolean_col); + assert_eq!(r.nullable_text, Some("present".to_owned())); + assert_eq!(r.int_array_col, vec![Some(1), Some(2), Some(3)]); + assert_eq!(r.cdc_operation, "INSERT"); + + // Row 2: copied, will be deleted. + let r = &rows[1]; + assert_eq!(r.id, 2); + assert_eq!(r.text_col, "delete_me"); + assert_eq!(r.cdc_operation, "INSERT"); + + // Row 3: DELETE tombstone -- every non-PK column type verified. + let r = &rows[2]; + assert_eq!(r.id, 2, "DELETE must target the correct row"); + assert_eq!(r.cdc_operation, "DELETE"); + assert!(r.cdc_lsn > 0); + // Non-nullable scalars -> zero values. + assert_eq!(r.smallint_col, 0, "smallint -> 0"); + assert_eq!(r.integer_col, 0, "integer -> 0"); + assert_eq!(r.bigint_col, 0, "bigint -> 0"); + assert!(r.real_col.abs() < 1e-6, "real -> 0.0"); + assert!(r.double_col.abs() < 1e-9, "double -> 0.0"); + assert_eq!(r.numeric_col, "", "numeric -> empty string"); + assert!(!r.boolean_col, "boolean -> false"); + assert_eq!(r.text_col, "", "text -> empty string"); + assert_eq!(r.varchar_col, "", "varchar -> empty string"); + assert_eq!(r.date_col, 0, "date -> 1970-01-01 (day 0)"); + assert_eq!(r.timestamp_col, 0, "timestamp -> unix epoch"); + assert_eq!(r.timestamptz_col, 0, "timestamptz -> unix epoch"); + assert_eq!(r.time_col, "", "time -> empty string (String-mapped)"); + assert_eq!(r.jsonb_col, "", "jsonb -> empty string (String-mapped)"); + assert_eq!(r.bytea_col, "", "bytea -> empty string"); + assert_eq!(r.uuid_col, "00000000-0000-0000-0000-000000000000", "uuid -> nil UUID"); + // Nullable scalars -> NULL. + assert_eq!(r.nullable_text, None, "nullable text -> NULL"); + assert_eq!(r.nullable_int, None, "nullable int -> NULL"); + // Arrays -> empty. + assert!(r.int_array_col.is_empty(), "int array -> empty"); + assert!(r.text_array_col.is_empty(), "text array -> empty"); + + // Row 4: post-delete INSERT proves pipeline continued. + let r = &rows[3]; + assert_eq!(r.id, 3); + assert_eq!(r.text_col, "after"); + assert_eq!(r.cdc_operation, "INSERT"); + assert!(r.cdc_lsn > 0); +} + +/// SELECT query used to verify the `large_batch` test. +const LARGE_BATCH_SELECT: &str = concat!( + "SELECT id, value, cdc_operation, cdc_lsn ", + "FROM \"test_large__batch\" ", + "ORDER BY id, cdc_lsn", +); + +/// Tests that a large table copy (1024 rows) completes without data loss or +/// corruption. +/// +/// # GIVEN +/// +/// A Postgres table with 1024 rows, each with a unique value derived from its +/// row number. +/// +/// # WHEN +/// +/// The pipeline runs initial table copy from Postgres to ClickHouse. +/// +/// # THEN +/// +/// All 1024 rows arrive in ClickHouse. A sample of rows at known positions +/// (first, last, powers of two, and a few interior points) are spot-checked +/// for correct id and value. +#[tokio::test(flavor = "multi_thread")] +async fn exclusive_large_batch_table_copy() { + init_test_tracing(); + install_crypto_provider(); + + // --- GIVEN: 1024 rows with unique values --- + let database = spawn_source_database().await; + let table_name = test_table_name("large_batch"); + + let table_id = database + .create_table(table_name.clone(), true, &[("value", "text not null")]) + .await + .expect("Failed to create large_batch test table"); + + let publication_name = "test_pub_clickhouse_large_batch"; + database + .create_publication(publication_name, std::slice::from_ref(&table_name)) + .await + .expect("Failed to create large_batch publication"); + + let row_count: usize = 1024; + let values: Vec = (1..=row_count).map(|i| format!("('val_{i:04}')")).collect(); + // Insert in chunks to avoid exceeding Postgres query size limits. + for chunk in values.chunks(256) { + database + .run_sql(&format!( + "INSERT INTO {} (value) VALUES {}", + table_name.as_quoted_identifier(), + chunk.join(", "), + )) + .await + .expect("Failed to insert large_batch rows"); + } + + let clickhouse_db = setup_clickhouse_database().await; + let store = NotifyingStore::new(); + let pipeline_id: PipelineId = random(); + let destination = clickhouse_db.build_destination(store.clone()); + + let table_ready = + store.notify_on_table_state_type(table_id, TableReplicationPhaseType::Ready).await; + + // --- WHEN: pipeline copies all rows --- + let mut pipeline = create_pipeline( + &database.config, + pipeline_id, + publication_name.to_owned(), + store, + destination, + ); + + pipeline.start().await.unwrap(); + table_ready.notified().await; + pipeline.shutdown_and_wait().await.unwrap(); + + // --- THEN: all rows arrive, spot-check a sample --- + let rows: Vec = clickhouse_db.query(LARGE_BATCH_SELECT).await; + assert_eq!(rows.len(), row_count, "all 1024 rows must arrive"); + + // Spot-check: first, last, powers of two, and a few interior points. + let sample_ids: &[usize] = &[1, 2, 4, 8, 16, 32, 64, 100, 128, 256, 500, 512, 750, 1000, 1024]; + for &id in sample_ids { + let r = &rows[id - 1]; + assert_eq!(r.id, id as i64, "row {id} id mismatch"); + assert_eq!(r.value, format!("val_{id:04}"), "row {id} value mismatch"); + assert_eq!(r.cdc_operation, "INSERT"); + assert_eq!(r.cdc_lsn, 0); + } +} + +/// # GIVEN +/// A ClickHouseClient pointed at the running test ClickHouse instance. +/// +/// # WHEN +/// `validate_connectivity()` is called. +/// +/// # THEN +/// It returns Ok(()). +#[tokio::test(flavor = "multi_thread")] +async fn validate_connectivity_succeeds_against_running_clickhouse() { + let client = ClickHouseClient::new( + get_clickhouse_url(), + get_clickhouse_user(), + get_clickhouse_password(), + "default", + ); + assert!(client.validate_connectivity().await.is_ok()); +} + +/// # GIVEN +/// A ClickHouseClient pointed at a URL where nothing is listening. +/// +/// # WHEN +/// `validate_connectivity()` is called. +/// +/// # THEN +/// It returns Err. +#[tokio::test(flavor = "multi_thread")] +async fn validate_connectivity_fails_against_unreachable_clickhouse() { + let client = ClickHouseClient::new( + Url::parse("http://localhost:1").unwrap(), + "nobody", + None::, + "default", + ); + assert!(client.validate_connectivity().await.is_err()); +} + +/// Row struct for the ADD COLUMN test after schema change. +/// Columns: id, name, age, email, score. +#[derive(clickhouse::Row, serde::Deserialize, Debug)] +struct AddColumnRow { + id: i64, + name: String, + age: i32, + email: Option, + score: Option, + cdc_operation: String, +} + +/// Tests that ALTER TABLE ADD COLUMN in Postgres propagates to ClickHouse +/// and subsequent inserts include the new column. +/// +/// # GIVEN +/// +/// A Postgres table with columns (id serial PK, name text not null, age integer +/// not null) and one row ('Alice', 25), copied to ClickHouse. +/// +/// # WHEN +/// +/// A nullable `email text` column and a `score integer NOT NULL DEFAULT 0` +/// column are added in Postgres, and a row ('Bob', 30, 'bob@example.com', 7) +/// is inserted with the new schema. +/// +/// # THEN +/// +/// The ClickHouse table has `email` and `score` columns. Alice's row has NULL +/// for both added columns because ClickHouse does not backfill historical CDC +/// rows. Bob's row has 'bob@example.com' and 7. The destination metadata +/// snapshot_id has increased. +#[tokio::test(flavor = "multi_thread")] +async fn schema_change_add_column() { + init_test_tracing(); + install_crypto_provider(); + + let clickhouse_table_name = "test_schema__add__col"; + + // --- GIVEN: table with one row, copied to ClickHouse --- + let database = spawn_source_database().await; + let table_name = test_table_name("schema_add_col"); + + let table_id = database + .create_table( + table_name.clone(), + true, + &[("name", "text not null"), ("age", "integer not null")], + ) + .await + .expect("Failed to create table"); + + let publication_name = "test_pub_ch_schema_add"; + database + .create_publication(publication_name, std::slice::from_ref(&table_name)) + .await + .expect("Failed to create publication"); + + database + .run_sql(&format!( + "INSERT INTO {} (name, age) VALUES ('Alice', 25)", + table_name.as_quoted_identifier(), + )) + .await + .expect("Failed to insert Alice"); + + let clickhouse_db = setup_clickhouse_database().await; + let store = NotifyingStore::new(); + let pipeline_id: PipelineId = random(); + let destination = clickhouse_db.build_destination(store.clone()); + + let table_ready = + store.notify_on_table_state_type(table_id, TableReplicationPhaseType::Ready).await; + + let mut pipeline = create_pipeline( + &database.config, + pipeline_id, + publication_name.to_owned(), + store.clone(), + destination, + ); + + pipeline.start().await.unwrap(); + table_ready.notified().await; + + // Verify initial state. + let initial_columns = clickhouse_db.column_names(clickhouse_table_name).await; + assert_eq!(initial_columns, vec!["id", "name", "age"]); + + let initial_metadata = store + .get_applied_destination_table_metadata(table_id) + .await + .unwrap() + .expect("metadata should exist after table creation"); + let initial_snapshot_id = initial_metadata.snapshot_id; + + // --- WHEN: add column, then insert with new schema --- + database + .alter_table( + table_name.clone(), + &[ + TableModification::AddColumn { name: "email", data_type: "text" }, + TableModification::AddColumn { + name: "score", + data_type: "integer not null default 0", + }, + ], + ) + .await + .unwrap(); + + database + .run_sql(&format!( + "INSERT INTO {} (name, age, email, score) VALUES ('Bob', 30, 'bob@example.com', 7)", + table_name.as_quoted_identifier(), + )) + .await + .expect("Failed to insert Bob"); + + // Poll until Bob's row arrives (2 rows total = Alice from copy + Bob from + // streaming). + let select = concat!( + "SELECT id, name, age, email, score, cdc_operation ", + "FROM \"test_schema__add__col\" ", + "ORDER BY id", + ); + let mut rows: Vec = Vec::new(); + for _ in 0..50 { + // The SELECT will fail if the email column doesn't exist yet, so + // catch errors and retry. + if let Ok(r) = clickhouse_db.db_client().query(select).fetch_all::().await + && r.len() >= 2 + { + rows = r; + break; + } + sleep(Duration::from_millis(200)).await; + } + assert!( + rows.len() >= 2, + "timed out waiting for schema_change_add_column rows: got {} of 2", + rows.len() + ); + + pipeline.shutdown_and_wait().await.unwrap(); + + // --- THEN: ClickHouse has the new columns, both rows present --- + let final_columns = clickhouse_db.column_names(clickhouse_table_name).await; + assert_eq!(final_columns, vec!["id", "name", "age", "email", "score"]); + + let final_column_types = clickhouse_db.column_types(clickhouse_table_name).await; + assert_eq!( + final_column_types, + vec![ + ("id".to_owned(), "Int64".to_owned()), + ("name".to_owned(), "String".to_owned()), + ("age".to_owned(), "Int32".to_owned()), + ("email".to_owned(), "Nullable(String)".to_owned()), + ("score".to_owned(), "Nullable(Int32)".to_owned()), + ] + ); + + assert_eq!(rows.len(), 2, "expected Alice + Bob"); + + // Alice: pre-change row, added columns should be NULL. + assert_eq!(rows[0].id, 1); + assert_eq!(rows[0].name, "Alice"); + assert_eq!(rows[0].age, 25); + assert_eq!(rows[0].email, None, "Alice's email should be NULL (column added after her row)"); + assert_eq!(rows[0].score, None, "Alice's score should be NULL (column added after her row)"); + assert_eq!(rows[0].cdc_operation, "INSERT"); + + // Bob: post-change row, added columns present. + assert_eq!(rows[1].id, 2); + assert_eq!(rows[1].name, "Bob"); + assert_eq!(rows[1].age, 30); + assert_eq!(rows[1].email, Some("bob@example.com".to_owned())); + assert_eq!(rows[1].score, Some(7)); + assert_eq!(rows[1].cdc_operation, "INSERT"); + + // Metadata snapshot_id should have advanced. + let final_metadata = store + .get_applied_destination_table_metadata(table_id) + .await + .unwrap() + .expect("metadata should exist after schema change"); + assert!( + final_metadata.snapshot_id > initial_snapshot_id, + "snapshot_id should increase after schema change" + ); +} + +/// Row struct for the combined schema change test after all changes. +/// Columns: id, full_name (renamed), status (kept), email (added). +/// age is dropped. +#[derive(clickhouse::Row, serde::Deserialize, Debug)] +struct CombinedSchemaChangeRow { + id: i64, + full_name: String, + status: Option, + email: Option, + cdc_operation: String, +} + +/// Tests that multiple schema changes (ADD, DROP, RENAME) in Postgres all +/// propagate to ClickHouse correctly. +/// +/// # GIVEN +/// +/// A Postgres table with columns (id serial PK, name text not null, age integer +/// not null, status text) and one row ('Alice', 25, 'active'), copied to +/// ClickHouse. +/// +/// # WHEN +/// +/// Three schema changes are applied: +/// 1. RENAME COLUMN name TO full_name +/// 2. DROP COLUMN age +/// 3. ADD COLUMN email text +/// A new row ('Bob', 'pending', 'bob@example.com') is inserted with the updated +/// schema. +/// +/// # THEN +/// +/// The ClickHouse table has columns: id, full_name, status, email. +/// - 'age' is dropped. +/// - 'name' is renamed to 'full_name'. +/// - 'email' is added. +/// Alice's row has 'Alice' under 'full_name', 'active' for status, NULL for +/// email. Bob's row has the new values. +/// The destination metadata snapshot_id has increased. +#[tokio::test(flavor = "multi_thread")] +async fn schema_change_add_drop_rename() { + init_test_tracing(); + install_crypto_provider(); + + let clickhouse_table_name = "test_schema__multi"; + + // --- GIVEN: table with one row, copied to ClickHouse --- + let database = spawn_source_database().await; + let table_name = test_table_name("schema_multi"); + + let table_id = database + .create_table( + table_name.clone(), + true, + &[("name", "text not null"), ("age", "integer not null"), ("status", "text")], + ) + .await + .expect("Failed to create table"); + + let publication_name = "test_pub_ch_schema_multi"; + database + .create_publication(publication_name, std::slice::from_ref(&table_name)) + .await + .expect("Failed to create publication"); + + database + .run_sql(&format!( + "INSERT INTO {} (name, age, status) VALUES ('Alice', 25, 'active')", + table_name.as_quoted_identifier(), + )) + .await + .expect("Failed to insert Alice"); + + let clickhouse_db = setup_clickhouse_database().await; + let store = NotifyingStore::new(); + let pipeline_id: PipelineId = random(); + let destination = clickhouse_db.build_destination(store.clone()); + + let table_ready = + store.notify_on_table_state_type(table_id, TableReplicationPhaseType::Ready).await; + + let mut pipeline = create_pipeline( + &database.config, + pipeline_id, + publication_name.to_owned(), + store.clone(), + destination, + ); + + pipeline.start().await.unwrap(); + table_ready.notified().await; + + // Verify initial schema. + let initial_columns = clickhouse_db.column_names(clickhouse_table_name).await; + assert_eq!(initial_columns, vec!["id", "name", "age", "status"]); + + let initial_metadata = store + .get_applied_destination_table_metadata(table_id) + .await + .unwrap() + .expect("metadata should exist after table creation"); + let initial_snapshot_id = initial_metadata.snapshot_id; + + // --- WHEN: rename + drop + add, then insert with new schema --- + database + .alter_table( + table_name.clone(), + &[TableModification::RenameColumn { old_name: "name", new_name: "full_name" }], + ) + .await + .unwrap(); + + database + .alter_table(table_name.clone(), &[TableModification::DropColumn { name: "age" }]) + .await + .unwrap(); + + database + .alter_table( + table_name.clone(), + &[TableModification::AddColumn { name: "email", data_type: "text" }], + ) + .await + .unwrap(); + + database + .run_sql(&format!( + "INSERT INTO {} (full_name, status, email) VALUES ('Bob', 'pending', \ + 'bob@example.com')", + table_name.as_quoted_identifier(), + )) + .await + .expect("Failed to insert Bob"); + + // Poll until Bob's row arrives. + let select = concat!( + "SELECT id, full_name, status, email, cdc_operation ", + "FROM \"test_schema__multi\" ", + "ORDER BY id", + ); + let mut rows: Vec = Vec::new(); + for _ in 0..50 { + if let Ok(r) = + clickhouse_db.db_client().query(select).fetch_all::().await + && r.len() >= 2 + { + rows = r; + break; + } + sleep(Duration::from_millis(200)).await; + } + assert!( + rows.len() >= 2, + "timed out waiting for schema_change_add_drop_rename rows: got {} of 2", + rows.len() + ); + + pipeline.shutdown_and_wait().await.unwrap(); + + // --- THEN: ClickHouse schema reflects all changes --- + let final_columns = clickhouse_db.column_names(clickhouse_table_name).await; + assert_eq!(final_columns, vec!["id", "full_name", "status", "email"]); + + assert_eq!(rows.len(), 2, "expected Alice + Bob"); + + // Alice: pre-change row. + assert_eq!(rows[0].id, 1); + assert_eq!(rows[0].full_name, "Alice", "renamed column should preserve data"); + assert_eq!(rows[0].status, Some("active".to_owned())); + assert_eq!(rows[0].email, None, "Alice's email should be NULL (added after her row)"); + assert_eq!(rows[0].cdc_operation, "INSERT"); + + // Bob: post-change row. + assert_eq!(rows[1].id, 2); + assert_eq!(rows[1].full_name, "Bob"); + assert_eq!(rows[1].status, Some("pending".to_owned())); + assert_eq!(rows[1].email, Some("bob@example.com".to_owned())); + assert_eq!(rows[1].cdc_operation, "INSERT"); + + // Metadata snapshot_id should have advanced. + let final_metadata = store + .get_applied_destination_table_metadata(table_id) + .await + .unwrap() + .expect("metadata should exist after schema change"); + assert!( + final_metadata.snapshot_id > initial_snapshot_id, + "snapshot_id should increase after schema change" + ); +} diff --git a/etl-destinations/tests/main.rs b/etl-destinations/tests/main.rs index 92582972c..e2fc3220a 100644 --- a/etl-destinations/tests/main.rs +++ b/etl-destinations/tests/main.rs @@ -2,6 +2,8 @@ mod support; #[cfg(all(feature = "bigquery", feature = "test-utils"))] mod bigquery_pipeline; +#[cfg(all(feature = "clickhouse", feature = "test-utils"))] +mod clickhouse_pipeline; #[cfg(feature = "ducklake")] mod ducklake_destination; #[cfg(feature = "ducklake")] diff --git a/etl-destinations/tests/support/clickhouse.rs b/etl-destinations/tests/support/clickhouse.rs new file mode 100644 index 000000000..acced45df --- /dev/null +++ b/etl-destinations/tests/support/clickhouse.rs @@ -0,0 +1,64 @@ +#![allow(dead_code)] + +/// A row read back from the ClickHouse `all_types_encoding` test table. +/// +/// Column-to-type mapping: +/// - `Date32` -> `i32` (signed days from 1970-01-01 in RowBinary) +/// - `DateTime64(6)` -> `i64` (microseconds since epoch in RowBinary) +/// - `UUID` -> `String` (via `toString()` in the SELECT query) +/// - `Array(Nullable(T))` -> `Vec>` +/// +/// Fields must match the SELECT column list in the test query exactly. +#[derive(clickhouse::Row, serde::Deserialize, Debug, Clone)] +pub struct AllTypesRow { + pub id: i64, + pub smallint_col: i16, + pub integer_col: i32, + pub bigint_col: i64, + pub real_col: f32, + pub double_col: f64, + pub numeric_col: String, + pub boolean_col: bool, + pub text_col: String, + pub varchar_col: String, + pub date_col: i32, // Date32 -> signed days from 1970-01-01 + pub timestamp_col: i64, // DateTime64(6) -> microseconds + pub timestamptz_col: i64, // DateTime64(6,'UTC') -> microseconds + pub time_col: String, + pub interval_col: String, + pub jsonb_col: String, + pub json_col: String, + pub integer_array_col: Vec>, + pub text_array_col: Vec>, + pub bytea_col: String, // hex-encoded + pub inet_col: String, + pub cidr_col: String, + pub macaddr_col: String, + pub uuid_col: String, // via toString() in SELECT + pub cdc_operation: String, +} + +/// A row read back from the ClickHouse `boundary_values` test table. +/// +/// Covers edge cases that the `all_types` test does not: nullable scalars, +/// NULL array elements, empty strings, and multi-byte UTF-8. +#[derive(clickhouse::Row, serde::Deserialize, Debug)] +pub struct BoundaryValuesRow { + pub id: i64, + pub nullable_text: Option, + pub nullable_int: Option, + pub int_array_col: Vec>, + pub text_array_col: Vec>, + pub cdc_operation: String, +} + +/// A row read back from a ClickHouse table with a single `Date32` column, +/// used to verify Postgres `date` round-tripping for values outside the Unix +/// epoch (pre-1970 and far-future). The `date_col` is the signed day offset +/// from 1970-01-01. +#[derive(clickhouse::Row, serde::Deserialize, Debug)] +pub struct DateBoundariesRow { + pub id: i64, + pub date_col: i32, + pub cdc_operation: String, +} diff --git a/etl-destinations/tests/support/mod.rs b/etl-destinations/tests/support/mod.rs index f149f0ba5..60f305046 100644 --- a/etl-destinations/tests/support/mod.rs +++ b/etl-destinations/tests/support/mod.rs @@ -1,4 +1,6 @@ pub mod bigquery; +#[cfg(all(feature = "clickhouse", feature = "test-utils"))] +pub mod clickhouse; #[cfg(feature = "ducklake")] pub mod ducklake; pub mod iceberg; diff --git a/etl-examples/Cargo.toml b/etl-examples/Cargo.toml index 11cb91be5..8ff0c2d76 100644 --- a/etl-examples/Cargo.toml +++ b/etl-examples/Cargo.toml @@ -20,7 +20,9 @@ path = "src/bin/ducklake.rs" clap = { workspace = true, default-features = true, features = ["std", "derive"] } etl = { workspace = true } -etl-destinations = { workspace = true, features = ["bigquery", "ducklake"] } +etl-config = { workspace = true } +etl-destinations = { workspace = true, features = ["bigquery", "clickhouse", "ducklake"] } +etl-telemetry = { workspace = true } rustls = { workspace = true, features = ["aws-lc-rs", "logging"] } tokio = { workspace = true, features = ["macros", "signal"] } tracing = { workspace = true, default-features = true } diff --git a/etl-examples/README.md b/etl-examples/README.md index 7515deac3..37e11fdc2 100644 --- a/etl-examples/README.md +++ b/etl-examples/README.md @@ -8,6 +8,7 @@ Postgres to various destinations using the ETL pipeline. | Example | Binary | Destination | Status | |---------|--------|-------------|--------| | [BigQuery](#bigquery) | `bigquery` | Google BigQuery (cloud data warehouse) | Stable | +| [ClickHouse](#clickhouse-setup) | `clickhouse` | ClickHouse (column-oriented OLAP database) | In progress | | [DuckLake](#ducklake) | `ducklake` | DuckLake (open data lake format) | In progress | --- @@ -95,6 +96,45 @@ cargo run --bin ducklake -p etl-examples -- \ The CLI also accepts plain local paths such as `./lake_data/` and normalizes them to absolute `file://` URLs before constructing the destination. +## ClickHouse Setup + +To run the ClickHouse example, you'll need a running ClickHouse instance accessible over HTTP(S). + +Create a publication in Postgres: + +```sql +create publication my_pub +for table table1, table2; +``` + +Then run the ClickHouse example: + +```bash +cargo run -p etl-examples --bin clickhouse -- \ + --db-host localhost \ + --db-port 5432 \ + --db-name postgres \ + --db-username postgres \ + --db-password password \ + --clickhouse-url http://localhost:8123 \ + --clickhouse-user default \ + --clickhouse-database default \ + --publication my_pub +``` + +Each Postgres table is replicated as an append-only `MergeTree` table. Two CDC metadata +columns are appended to every row: + +- `cdc_operation`: `INSERT`, `UPDATE`, or `DELETE` +- `cdc_lsn`: the Postgres LSN at the time of the change + +Table names are derived from the Postgres schema and table name using double-underscore +escaping (e.g. `public.orders` → `public_orders`, `my_schema.t` → `my__schema_t`). + +For HTTPS connections, provide an `https://` URL — TLS is handled automatically using +webpki root certificates. Use `--clickhouse-password` if your ClickHouse instance requires +authentication. + ### Example configuration This is a fuller local example that also enables a dedicated DuckDB log dump on diff --git a/etl-examples/src/bin/clickhouse.rs b/etl-examples/src/bin/clickhouse.rs new file mode 100644 index 000000000..4b3973b35 --- /dev/null +++ b/etl-examples/src/bin/clickhouse.rs @@ -0,0 +1,242 @@ +/* + +ClickHouse Example + +This example demonstrates how to use the pipeline to stream +data from Postgres to ClickHouse using change data capture (CDC). + +Each Postgres table is replicated as an append-only MergeTree table. +Two CDC metadata columns are appended to every row: + - `cdc_operation`: `INSERT`, `UPDATE`, or `DELETE` + - `cdc_lsn`: the Postgres LSN at the time of the change + +Table names are derived from the Postgres schema and table name using +double-underscore escaping (e.g. `public.orders` → `public__orders`). + +Prerequisites: +1. Postgres server with logical replication enabled (wal_level = logical) +2. A publication created in Postgres (CREATE PUBLICATION my_pub FOR ALL TABLES;) +3. A running ClickHouse instance accessible over HTTP(S) + +Usage: + cargo run -p etl-examples --bin clickhouse -- \ + --db-host localhost \ + --db-port 5432 \ + --db-name postgres \ + --db-username postgres \ + --db-password password \ + --clickhouse-url http://localhost:8123 \ + --clickhouse-user default \ + --clickhouse-database default \ + --publication my_pub + +For HTTPS connections, provide an `https://` URL — TLS is handled automatically +using webpki root certificates. Use `--clickhouse-password` if your ClickHouse instance +requires authentication. + +*/ + +use std::{error::Error, sync::Once}; + +use clap::{Args, Parser}; +use etl::{ + config::{ + BatchConfig, InvalidatedSlotBehavior, MemoryBackpressureConfig, PgConnectionConfig, + PipelineConfig, TableSyncCopyConfig, TcpKeepaliveConfig, TlsConfig, + }, + pipeline::Pipeline, + store::both::memory::MemoryStore, +}; +use etl_destinations::clickhouse::{ClickHouseDestination, ClickHouseInserterConfig}; +use tokio::signal; +use tracing::{error, info}; +use tracing_subscriber::{layer::SubscriberExt, util::SubscriberInitExt}; +use url::Url; + +/// Ensures crypto provider is only initialized once. +static INIT_CRYPTO: Once = Once::new(); + +/// Installs the default cryptographic provider for rustls. +fn install_crypto_provider() { + INIT_CRYPTO.call_once(|| { + rustls::crypto::aws_lc_rs::default_provider() + .install_default() + .expect("failed to install default crypto provider"); + }); +} + +/// Main application arguments combining database and ClickHouse configurations. +#[derive(Debug, Parser)] +#[command(name = "clickhouse", version, about, arg_required_else_help = true)] +struct AppArgs { + /// Postgres connection parameters + #[clap(flatten)] + db_args: DbArgs, + /// ClickHouse destination parameters + #[clap(flatten)] + clickhouse_args: ClickHouseArgs, + /// Postgres publication name (must be created beforehand with CREATE + /// PUBLICATION) + #[arg(long)] + publication: String, +} + +/// Postgres database connection configuration. +#[derive(Debug, Args)] +struct DbArgs { + /// Host on which Postgres is running (e.g., localhost or IP address) + #[arg(long)] + db_host: String, + /// Port on which Postgres is running (default: 5432) + #[arg(long)] + db_port: u16, + /// Postgres database name to connect to + #[arg(long)] + db_name: String, + /// Postgres database user name (must have REPLICATION privileges) + #[arg(long)] + db_username: String, + /// Postgres database user password (optional if using trust authentication) + #[arg(long)] + db_password: Option, +} + +/// ClickHouse destination configuration. +#[derive(Debug, Args)] +struct ClickHouseArgs { + /// ClickHouse HTTP(S) endpoint (e.g. http://localhost:8123 or https://host:8443) + #[arg(long)] + clickhouse_url: String, + /// ClickHouse user name + #[arg(long)] + clickhouse_user: String, + /// ClickHouse user password (optional) + #[arg(long)] + clickhouse_password: Option, + /// ClickHouse target database + #[arg(long)] + clickhouse_database: String, + /// Maximum time to wait for a batch to fill in milliseconds (lower values = + /// lower latency, less throughput) + #[arg(long, default_value = "5000")] + max_batch_fill_duration_ms: u64, + /// Maximum number of concurrent table sync workers (higher values = faster + /// initial sync, more resource usage) + #[arg(long, default_value = "4")] + max_table_sync_workers: u16, +} + +/// Entry point — handles error reporting and process exit. +#[tokio::main] +async fn main() -> Result<(), Box> { + if let Err(e) = main_impl().await { + error!("{e}"); + std::process::exit(1); + } + + Ok(()) +} + +/// Initialize structured logging with configurable log levels via RUST_LOG +/// environment variable. +fn init_tracing() { + tracing_subscriber::registry() + .with( + tracing_subscriber::EnvFilter::try_from_default_env() + .unwrap_or_else(|_| "clickhouse=info".into()), + ) + .with(tracing_subscriber::fmt::layer()) + .init(); +} + +/// Main implementation containing all pipeline setup and execution logic. +async fn main_impl() -> Result<(), Box> { + init_tracing(); + + // Install required crypto provider for TLS (used when clickhouse_url is https://) + install_crypto_provider(); + + let args = AppArgs::parse(); + + // Configure Postgres connection settings + // Note: TLS is disabled in this example — enable for production use + let pg_connection_config = PgConnectionConfig { + host: args.db_args.db_host, + port: args.db_args.db_port, + name: args.db_args.db_name, + username: args.db_args.db_username, + password: args.db_args.db_password.map(Into::into), + tls: TlsConfig { + trusted_root_certs: String::new(), + enabled: false, // Set to true and provide certs for production + }, + keepalive: TcpKeepaliveConfig::default(), + }; + + // Create in-memory store for tracking table replication states and schemas. + // In production, you might want to use a persistent store like PostgresStore. + let store = MemoryStore::new(); + + let pipeline_config = PipelineConfig { + id: 1, + publication_name: args.publication, + pg_connection: pg_connection_config, + batch: BatchConfig { + max_fill_ms: args.clickhouse_args.max_batch_fill_duration_ms, + memory_budget_ratio: BatchConfig::DEFAULT_MEMORY_BUDGET_RATIO, + }, + table_error_retry_delay_ms: 10000, + table_error_retry_max_attempts: 5, + max_table_sync_workers: args.clickhouse_args.max_table_sync_workers, + memory_refresh_interval_ms: 100, + memory_backpressure: Some(MemoryBackpressureConfig::default()), + table_sync_copy: TableSyncCopyConfig::default(), + invalidated_slot_behavior: InvalidatedSlotBehavior::default(), + max_copy_connections_per_table: PipelineConfig::DEFAULT_MAX_COPY_CONNECTIONS_PER_TABLE, + }; + + // Initialize the ClickHouse destination. + // Tables are created automatically as append-only MergeTree tables. + let clickhouse_destination = ClickHouseDestination::new( + Url::parse(&args.clickhouse_args.clickhouse_url)?, + args.clickhouse_args.clickhouse_user, + args.clickhouse_args.clickhouse_password, + args.clickhouse_args.clickhouse_database, + ClickHouseInserterConfig::default(), + store.clone(), + )?; + + let mut pipeline = Pipeline::new(pipeline_config, store, clickhouse_destination); + + info!( + "Starting ClickHouse CDC pipeline - connecting to Postgres and initializing replication..." + ); + + // Start the pipeline — this will: + // 1. Connect to Postgres + // 2. Initialize table states based on the publication + // 3. Start apply and table sync workers + // 4. Begin streaming replication data + pipeline.start().await?; + + info!("pipeline started, data replication is now active, press ctrl+c to stop"); + + let shutdown_signal = async { + signal::ctrl_c().await.expect("Failed to install Ctrl+C handler"); + info!("received ctrl+c signal, initiating graceful shutdown"); + }; + + tokio::select! { + result = pipeline.wait() => { + info!("pipeline completed normally (this usually indicates an error condition)"); + result?; + } + _ = shutdown_signal => { + info!("gracefully shutting down pipeline and cleaning up resources"); + } + } + + info!("pipeline stopped, all resources cleaned up"); + + Ok(()) +} diff --git a/etl-replicator/Cargo.toml b/etl-replicator/Cargo.toml index 63cdef8a9..2eb17b4f5 100644 --- a/etl-replicator/Cargo.toml +++ b/etl-replicator/Cargo.toml @@ -16,7 +16,7 @@ egress = ["etl/egress", "etl-destinations/egress"] configcat = { workspace = true } etl = { workspace = true } etl-config = { workspace = true, features = ["supabase"] } -etl-destinations = { workspace = true, features = ["bigquery", "iceberg", "ducklake"] } +etl-destinations = { workspace = true, features = ["bigquery", "clickhouse", "ducklake", "iceberg"] } etl-telemetry = { workspace = true } metrics = { workspace = true } reqwest = { workspace = true, features = ["rustls-tls", "json"] } @@ -25,7 +25,7 @@ secrecy = { workspace = true } sentry = { workspace = true } serde = { workspace = true, features = ["derive"] } serde_json = { workspace = true } -sqlx = { workspace = true, features = ["runtime-tokio-rustls", "postgres"] } +sqlx = { workspace = true, features = ["runtime-tokio-rustls", "postgres", "migrate"] } tokio = { workspace = true, features = ["rt-multi-thread", "macros", "signal"] } tracing = { workspace = true, default-features = true } diff --git a/etl-replicator/src/core.rs b/etl-replicator/src/core.rs index d5f4c2155..6313610d8 100644 --- a/etl-replicator/src/core.rs +++ b/etl-replicator/src/core.rs @@ -16,6 +16,7 @@ use etl_config::{ }; use etl_destinations::{ bigquery::BigQueryDestination, + clickhouse::{ClickHouseDestination, ClickHouseInserterConfig}, ducklake::{DuckLakeDestination, S3Config as DucklakeS3Config}, iceberg::{ DestinationNamespace, IcebergClient, IcebergDestination, S3_ACCESS_KEY_ID, S3_ENDPOINT, @@ -186,6 +187,19 @@ pub(crate) async fn start_replicator_with_config( ) .await?; + let pipeline = Pipeline::new(replicator_config.pipeline, state_store, destination); + start_pipeline(pipeline).await?; + } + DestinationConfig::ClickHouse { url, user, password, database } => { + let destination = ClickHouseDestination::new( + url.clone(), + user, + password.as_ref().map(|p| p.expose_secret().to_owned()), + database, + ClickHouseInserterConfig::default(), + state_store.clone(), + )?; + let pipeline = Pipeline::new(replicator_config.pipeline, state_store, destination); start_pipeline(pipeline).await?; } diff --git a/etl-replicator/src/init/destination.rs b/etl-replicator/src/init/destination.rs index a24aee647..6dc8fb36d 100644 --- a/etl-replicator/src/init/destination.rs +++ b/etl-replicator/src/init/destination.rs @@ -1,7 +1,8 @@ use etl::{destination::Destination, store::both::postgres::PostgresStore}; use etl_config::shared::{DestinationConfig, IcebergConfig}; use etl_destinations::{ - bigquery::BigQueryDestination, ducklake::DuckLakeDestination, iceberg::IcebergDestination, + bigquery::BigQueryDestination, clickhouse::ClickHouseDestination, + ducklake::DuckLakeDestination, iceberg::IcebergDestination, }; use crate::error_reporting::ErrorReportingStateStore; @@ -16,5 +17,8 @@ pub(crate) fn destination_name(destination_config: &DestinationConfig) -> &'stat config: IcebergConfig::Supabase { .. } | IcebergConfig::Rest { .. }, } => IcebergDestination::::name(), DestinationConfig::Ducklake { .. } => DuckLakeDestination::::name(), + DestinationConfig::ClickHouse { .. } => { + ClickHouseDestination::::name() + } } } diff --git a/scripts/docker-compose.yaml b/scripts/docker-compose.yaml index 5da035c0d..b9fb80217 100644 --- a/scripts/docker-compose.yaml +++ b/scripts/docker-compose.yaml @@ -39,6 +39,25 @@ services: timeout: 5s retries: 5 + clickhouse: + image: clickhouse/clickhouse-server:latest + environment: + CLICKHOUSE_USER: ${CLICKHOUSE_USER:-etl} + CLICKHOUSE_PASSWORD: ${CLICKHOUSE_PASSWORD:-etl} + CLICKHOUSE_DEFAULT_ACCESS_MANAGEMENT: 1 + ports: + - "${CLICKHOUSE_HTTP_PORT:-8123}:8123" + - "${CLICKHOUSE_NATIVE_PORT:-9000}:9000" + volumes: + - ${CLICKHOUSE_DATA_VOLUME:-clickhouse_data}:/var/lib/clickhouse + restart: unless-stopped + healthcheck: + test: ["CMD-SHELL", "clickhouse-client --user \"$$CLICKHOUSE_USER\" --password \"$$CLICKHOUSE_PASSWORD\" --query \"SELECT 1\""] + interval: 5s + timeout: 5s + retries: 10 + start_period: 10s + catalog-postgres: image: postgres:${LAKEKEEPER_POSTGRES_VERSION:-18} environment: @@ -167,6 +186,8 @@ services: volumes: postgres_data: driver: local + clickhouse_data: + driver: local lakekeeper_postgres_data: driver: local minio_data: diff --git a/scripts/init.sh b/scripts/init.sh index 8eb2a0d89..fe2f4e573 100755 --- a/scripts/init.sh +++ b/scripts/init.sh @@ -27,10 +27,16 @@ DB_PASSWORD="${POSTGRES_PASSWORD:=postgres}" DB_NAME="${POSTGRES_DB:=postgres}" DB_PORT="${POSTGRES_PORT:=5430}" DB_HOST="${POSTGRES_HOST:=localhost}" +CLICKHOUSE_HTTP_PORT="${CLICKHOUSE_HTTP_PORT:=8123}" +CLICKHOUSE_NATIVE_PORT="${CLICKHOUSE_NATIVE_PORT:=9000}" +CLICKHOUSE_USER="${CLICKHOUSE_USER:=etl}" +CLICKHOUSE_PASSWORD="${CLICKHOUSE_PASSWORD:=etl}" # Docker compose setup +USING_DOCKER_COMPOSE=0 if [[ -z "${SKIP_DOCKER}" ]] then + USING_DOCKER_COMPOSE=1 echo "🐳 Starting all services with Docker Compose..." # Export environment variables for docker-compose @@ -38,14 +44,26 @@ then export POSTGRES_PASSWORD="${DB_PASSWORD}" export POSTGRES_DB="${DB_NAME}" export POSTGRES_PORT="${DB_PORT}" + export CLICKHOUSE_HTTP_PORT="${CLICKHOUSE_HTTP_PORT}" + export CLICKHOUSE_NATIVE_PORT="${CLICKHOUSE_NATIVE_PORT}" + export CLICKHOUSE_USER="${CLICKHOUSE_USER}" + export CLICKHOUSE_PASSWORD="${CLICKHOUSE_PASSWORD}" # Handle persistent storage if [[ -n "${POSTGRES_DATA_VOLUME}" ]]; then - echo "📁 Setting up persistent storage at ${POSTGRES_DATA_VOLUME}" + echo "📁 Setting up Postgres persistent storage at ${POSTGRES_DATA_VOLUME}" mkdir -p "${POSTGRES_DATA_VOLUME}" export POSTGRES_DATA_VOLUME="${POSTGRES_DATA_VOLUME}" else - echo "📁 No storage path specified, using default Docker volume" + echo "📁 No Postgres storage path specified, using default Docker volume" + fi + + if [[ -n "${CLICKHOUSE_DATA_VOLUME}" ]]; then + echo "📁 Setting up ClickHouse persistent storage at ${CLICKHOUSE_DATA_VOLUME}" + mkdir -p "${CLICKHOUSE_DATA_VOLUME}" + export CLICKHOUSE_DATA_VOLUME="${CLICKHOUSE_DATA_VOLUME}" + else + echo "📁 No ClickHouse storage path specified, using default Docker volume" fi # Pull latest images before starting services @@ -57,17 +75,39 @@ then echo "✅ All services started" fi -# Wait for Postgres to be ready -echo "⏳ Waiting for Postgres to be ready..." -until docker-compose -f ./scripts/docker-compose.yaml exec -T source-postgres pg_isready -U postgres > /dev/null 2>&1; do - echo "Waiting for Postgres..." - sleep 1 -done - -echo "✅ Postgres is up and running on port ${DB_PORT}" - # Export DATABASE_URL for potential use by other scripts export DATABASE_URL=postgres://${DB_USER}:${DB_PASSWORD}@${DB_HOST}:${DB_PORT}/${DB_NAME} + +# Wait for Postgres to be ready +if [[ "${USING_DOCKER_COMPOSE}" == "1" ]]; then + echo "⏳ Waiting for Postgres to be ready..." + until docker-compose -f ./scripts/docker-compose.yaml exec -T source-postgres pg_isready -U postgres > /dev/null 2>&1; do + echo "Waiting for Postgres..." + sleep 1 + done + + echo "✅ Postgres is up and running on port ${DB_PORT}" + + echo "⏳ Waiting for ClickHouse to be ready..." + until docker-compose -f ./scripts/docker-compose.yaml exec -T clickhouse clickhouse-client --user "$CLICKHOUSE_USER" --password "$CLICKHOUSE_PASSWORD" --query "SELECT 1" > /dev/null 2>&1; do + echo "Waiting for ClickHouse..." + sleep 1 + done + + echo "✅ ClickHouse is up and running on port ${CLICKHOUSE_HTTP_PORT}" + echo "🔗 ClickHouse HTTP URL: http://localhost:${CLICKHOUSE_HTTP_PORT}" + echo "🧪 ClickHouse test env: TESTS_CLICKHOUSE_URL=http://localhost:${CLICKHOUSE_HTTP_PORT} TESTS_CLICKHOUSE_USER=${CLICKHOUSE_USER} TESTS_CLICKHOUSE_PASSWORD=${CLICKHOUSE_PASSWORD}" +else + echo "⏳ Waiting for Postgres to be ready..." + until psql "${DATABASE_URL}" -c "select 1" > /dev/null 2>&1; do + echo "Waiting for Postgres..." + sleep 1 + done + + echo "✅ Postgres is up and running on port ${DB_PORT}" + echo "ℹ️ SKIP_DOCKER is set; skipping ClickHouse readiness checks." +fi + echo "🔗 Database URL: ${DATABASE_URL}" # Run database migrations diff --git a/scripts/test-clickhouse.sh b/scripts/test-clickhouse.sh new file mode 100755 index 000000000..669c7e5ac --- /dev/null +++ b/scripts/test-clickhouse.sh @@ -0,0 +1,80 @@ +#!/usr/bin/env bash +set -euo pipefail + +COMPOSE_FILE="${COMPOSE_FILE:-./scripts/docker-compose.yaml}" +DOCKER_COMPOSE_BIN="${DOCKER_COMPOSE_BIN:-docker-compose}" +POSTGRES_SERVICE="${POSTGRES_SERVICE:-source-postgres}" +CLICKHOUSE_SERVICE="${CLICKHOUSE_SERVICE:-clickhouse}" +POSTGRES_PORT="${POSTGRES_PORT:-5430}" +POSTGRES_USER="${POSTGRES_USER:-postgres}" +POSTGRES_PASSWORD="${POSTGRES_PASSWORD:-postgres}" +CLICKHOUSE_HTTP_PORT="${CLICKHOUSE_HTTP_PORT:-8123}" +CLICKHOUSE_USER="${CLICKHOUSE_USER:-etl}" +CLICKHOUSE_PASSWORD="${CLICKHOUSE_PASSWORD:-etl}" +CARGO_TOOLCHAIN="${CARGO_TOOLCHAIN:-}" +# Cargo integration test target name. ClickHouse tests live in tests/main.rs under +# the clickhouse_pipeline module; use TEST_NAME_FILTER to select that module or a +# specific test. Set TEST_NAME_FILTER='' to run all tests in the target. +TEST_TARGET="${TEST_TARGET:-main}" +TEST_NAME_FILTER="${TEST_NAME_FILTER-clickhouse_pipeline}" +CARGO_PACKAGE="${CARGO_PACKAGE:-etl-destinations}" +FEATURES="${FEATURES:-clickhouse,test-utils}" + +require_cmd() { + if ! command -v "$1" >/dev/null 2>&1; then + echo >&2 "❌ Error: required command '$1' is not installed." + exit 1 + fi +} + +require_cmd "$DOCKER_COMPOSE_BIN" +require_cmd cargo + +if [[ -z "${SKIP_DOCKER:-}" ]]; then + echo "🐳 Starting local Postgres and ClickHouse services..." + "$DOCKER_COMPOSE_BIN" -f "$COMPOSE_FILE" up -d "$POSTGRES_SERVICE" "$CLICKHOUSE_SERVICE" + + echo "⏳ Waiting for Postgres to be ready..." + until "$DOCKER_COMPOSE_BIN" -f "$COMPOSE_FILE" exec -T "$POSTGRES_SERVICE" pg_isready -U "$POSTGRES_USER" >/dev/null 2>&1; do + echo "Waiting for Postgres..." + sleep 1 + done + + echo "⏳ Waiting for ClickHouse to be ready..." + until "$DOCKER_COMPOSE_BIN" -f "$COMPOSE_FILE" exec -T "$CLICKHOUSE_SERVICE" clickhouse-client --user "$CLICKHOUSE_USER" --password "$CLICKHOUSE_PASSWORD" --query "SELECT 1" >/dev/null 2>&1; do + echo "Waiting for ClickHouse..." + sleep 1 + done +fi + +export TESTS_DATABASE_HOST="${TESTS_DATABASE_HOST:-localhost}" +export TESTS_DATABASE_PORT="${TESTS_DATABASE_PORT:-$POSTGRES_PORT}" +export TESTS_DATABASE_USERNAME="${TESTS_DATABASE_USERNAME:-$POSTGRES_USER}" +export TESTS_DATABASE_PASSWORD="${TESTS_DATABASE_PASSWORD:-$POSTGRES_PASSWORD}" +export TESTS_CLICKHOUSE_URL="${TESTS_CLICKHOUSE_URL:-http://localhost:$CLICKHOUSE_HTTP_PORT}" +export TESTS_CLICKHOUSE_USER="${TESTS_CLICKHOUSE_USER:-$CLICKHOUSE_USER}" +export TESTS_CLICKHOUSE_PASSWORD="${TESTS_CLICKHOUSE_PASSWORD:-$CLICKHOUSE_PASSWORD}" + +if [[ -n "$CARGO_TOOLCHAIN" ]]; then + CARGO_CMD=(cargo "+$CARGO_TOOLCHAIN") +else + CARGO_CMD=(cargo) +fi + +echo "🧪 Running ClickHouse destination test with:" +echo " TESTS_DATABASE_HOST=$TESTS_DATABASE_HOST" +echo " TESTS_DATABASE_PORT=$TESTS_DATABASE_PORT" +echo " TESTS_DATABASE_USERNAME=$TESTS_DATABASE_USERNAME" +echo " TESTS_CLICKHOUSE_URL=$TESTS_CLICKHOUSE_URL" +echo " TESTS_CLICKHOUSE_USER=$TESTS_CLICKHOUSE_USER" +echo " TESTS_CLICKHOUSE_PASSWORD=${TESTS_CLICKHOUSE_PASSWORD:+[set]}" +echo " cargo toolchain=${CARGO_TOOLCHAIN:-project default}" + +TEST_ARGS=(test -p "$CARGO_PACKAGE" --features "$FEATURES" --test "$TEST_TARGET") +if [[ -n "$TEST_NAME_FILTER" ]]; then + TEST_ARGS+=("$TEST_NAME_FILTER") +fi +TEST_ARGS+=(-- --nocapture) + +echo "🚀 ${CARGO_CMD[*]} ${TEST_ARGS[*]}" +"${CARGO_CMD[@]}" "${TEST_ARGS[@]}" diff --git a/xtask/src/commands/nextest.rs b/xtask/src/commands/nextest.rs index fa6bfddea..f98a802c0 100644 --- a/xtask/src/commands/nextest.rs +++ b/xtask/src/commands/nextest.rs @@ -14,7 +14,7 @@ use clap::{Args, ValueEnum}; const SHARED_PG_FILTER: &str = "\ test(exclusive_) | binary_id(etl::main) | (binary_id(etl-destinations::main) & \ - test(/^(bigquery_pipeline|ducklake_destination|ducklake_pipeline|iceberg_destination)::/)) | \ + test(/^(bigquery_pipeline|clickhouse_pipeline|ducklake_destination|ducklake_pipeline|iceberg_destination)::/)) | \ (binary_id(etl-destinations) & test(/ducklake::core::tests::postgres_backed::/))"; use super::shared::{DEFAULT_BASE_PORT, DEFAULT_PG_SHARD_COUNT};