diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index 39b0ec00f..cc83763fc 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -169,4 +169,3 @@ jobs: fail-on-error: false github-token: ${{ secrets.GITHUB_TOKEN }} path-to-lcov: lcov.info - debug: true diff --git a/.gitignore b/.gitignore index 22d1f250c..fefdf1991 100644 --- a/.gitignore +++ b/.gitignore @@ -33,3 +33,5 @@ pyvenv.cfg # Log files *.log + +lcov.info diff --git a/Cargo.toml b/Cargo.toml index b5a0240b7..9115607cc 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -32,7 +32,7 @@ actix-web = { version = "4.11.0", default-features = false } actix-web-httpauth = { version = "0.8.2", default-features = false } actix-web-metrics = { version = "0.3.0", default-features = false } anyhow = { version = "1.0.98", default-features = false } -arrow = { version = "55.0", default-features = false } +arrow = { version = "56.2.0", default-features = false } async-trait = { version = "0.1.88" } aws-lc-rs = { version = "1.13.3", default-features = false } base64 = { version = "0.22.1", default-features = false } @@ -43,6 +43,8 @@ clap = { version = "4.5.42", default-features = false } config = { version = "0.14", default-features = false } const-oid = { version = "0.9.6", default-features = false } constant_time_eq = { version = "0.4.2" } +dashmap = { version = "6.1.0", default-features = false } +deltalake = { git = "https://github.com/delta-io/delta-rs.git", rev = "d30b11f673b0111dbb0f904bf89d5b917ea652ed", default-features = false } fail = { version = "0.5.1", default-features = false } futures = { version = "0.3.31", default-features = false } gcp-bigquery-client = { version = "0.27.0", default-features = false } @@ -53,7 +55,7 @@ k8s-openapi = { version = "0.25.0", default-features = false } kube = { version = "1.1.0", default-features = false } metrics = { version = "0.24.2", default-features = false } metrics-exporter-prometheus = { version = "0.17.2", default-features = false } -parquet = { version = "55.0", default-features = false } +parquet = { version = "55.0.0", default-features = false } pg_escape = { version = "0.1.1", default-features = false } pin-project-lite = { version = "0.2.16", default-features = false } postgres-replication = { git = "https://github.com/MaterializeInc/rust-postgres", default-features = false, rev = "c4b473b478b3adfbf8667d2fbe895d8423f1290b" } @@ -84,3 +86,7 @@ x509-cert = { version = "0.2.2", default-features = false } [profile.bench] debug = true + +[profile.dev.package] +insta.opt-level = 3 +similar.opt-level = 3 \ No newline at end of file diff --git a/etl-api/src/configs/destination.rs b/etl-api/src/configs/destination.rs index 260e4bb8b..107d9ad93 100644 --- a/etl-api/src/configs/destination.rs +++ b/etl-api/src/configs/destination.rs @@ -1,3 +1,5 @@ +use std::collections::HashMap; + use etl_config::SerializableSecretString; use etl_config::shared::DestinationConfig; use secrecy::ExposeSecret; @@ -14,6 +16,7 @@ const DEFAULT_MAX_CONCURRENT_STREAMS: usize = 8; #[derive(Debug, Clone, Serialize, Deserialize, ToSchema)] #[serde(rename_all = "snake_case")] +#[non_exhaustive] pub enum FullApiDestinationConfig { Memory, BigQuery { @@ -30,6 +33,16 @@ pub enum FullApiDestinationConfig { #[serde(skip_serializing_if = "Option::is_none")] max_concurrent_streams: Option, }, + DeltaLake { + #[schema(example = "s3://my-bucket/my-path")] + base_uri: String, + #[schema(example = "{\"aws_access_key_id\": \"https://my-endpoint.com\"}")] + storage_options: Option>, + #[schema(example = "{\"my_table\": [\"date\"]}")] + partition_columns: Option>>, + #[schema(example = 100)] + optimize_after_commits: Option, + }, } impl From for FullApiDestinationConfig { @@ -49,6 +62,17 @@ impl From for FullApiDestinationConfig { max_staleness_mins, max_concurrent_streams: Some(max_concurrent_streams), }, + StoredDestinationConfig::DeltaLake { + base_uri, + storage_options, + partition_columns, + optimize_after_commits, + } => Self::DeltaLake { + base_uri, + storage_options, + partition_columns, + optimize_after_commits, + }, } } } @@ -64,6 +88,12 @@ pub enum StoredDestinationConfig { max_staleness_mins: Option, max_concurrent_streams: usize, }, + DeltaLake { + base_uri: String, + storage_options: Option>, + partition_columns: Option>>, + optimize_after_commits: Option, + }, } impl StoredDestinationConfig { @@ -83,6 +113,17 @@ impl StoredDestinationConfig { max_staleness_mins, max_concurrent_streams, }, + Self::DeltaLake { + base_uri, + storage_options, + partition_columns, + optimize_after_commits, + } => DestinationConfig::DeltaLake { + base_uri, + storage_options, + partition_columns, + optimize_after_commits, + }, } } } @@ -105,6 +146,17 @@ impl From for StoredDestinationConfig { max_concurrent_streams: max_concurrent_streams .unwrap_or(DEFAULT_MAX_CONCURRENT_STREAMS), }, + FullApiDestinationConfig::DeltaLake { + base_uri, + storage_options, + partition_columns, + optimize_after_commits, + } => Self::DeltaLake { + base_uri, + storage_options, + partition_columns, + optimize_after_commits, + }, } } } @@ -136,12 +188,24 @@ impl Encrypt for StoredDestinationConfig { max_concurrent_streams, }) } + Self::DeltaLake { + base_uri, + storage_options, + partition_columns, + optimize_after_commits, + } => Ok(EncryptedStoredDestinationConfig::DeltaLake { + base_uri, + storage_options, + partition_columns, + optimize_after_commits, + }), } } } #[derive(Debug, Clone, Serialize, Deserialize)] #[serde(rename_all = "snake_case")] +#[non_exhaustive] pub enum EncryptedStoredDestinationConfig { Memory, BigQuery { @@ -151,6 +215,12 @@ pub enum EncryptedStoredDestinationConfig { max_staleness_mins: Option, max_concurrent_streams: usize, }, + DeltaLake { + base_uri: String, + storage_options: Option>, + partition_columns: Option>>, + optimize_after_commits: Option, + }, } impl Store for EncryptedStoredDestinationConfig {} @@ -182,6 +252,17 @@ impl Decrypt for EncryptedStoredDestinationConfig { max_concurrent_streams, }) } + Self::DeltaLake { + base_uri, + storage_options, + partition_columns, + optimize_after_commits, + } => Ok(StoredDestinationConfig::DeltaLake { + base_uri, + storage_options, + partition_columns, + optimize_after_commits, + }), } } } diff --git a/etl-benchmarks/Cargo.toml b/etl-benchmarks/Cargo.toml index dcde1aad9..6b43abc98 100644 --- a/etl-benchmarks/Cargo.toml +++ b/etl-benchmarks/Cargo.toml @@ -7,10 +7,15 @@ rust-version.workspace = true repository.workspace = true homepage.workspace = true +[features] +default = ["bigquery", "deltalake"] +bigquery = ["etl-destinations/bigquery"] +deltalake = ["etl-destinations/deltalake"] + [dev-dependencies] etl = { workspace = true, features = ["test-utils"] } etl-config = { workspace = true } -etl-destinations = { workspace = true, features = ["bigquery"] } +etl-destinations = { workspace = true } etl-postgres = { workspace = true, features = ["sqlx"] } etl-telemetry = { workspace = true } diff --git a/etl-benchmarks/Dockerfile.tpch-seeder b/etl-benchmarks/Dockerfile.tpch-seeder new file mode 100644 index 000000000..21b2c2f2a --- /dev/null +++ b/etl-benchmarks/Dockerfile.tpch-seeder @@ -0,0 +1,14 @@ +# Builds an image that bundles go-tpc along with psql client tools for loading TPC-H data. +FROM golang:1.22 AS builder + +ARG GO_TPC_VERSION=latest + +RUN apt-get update \ + && apt-get install -y --no-install-recommends postgresql-client \ + && rm -rf /var/lib/apt/lists/* + +RUN go install github.com/pingcap/go-tpc/cmd/go-tpc@${GO_TPC_VERSION} + +ENV PATH="/go/bin:${PATH}" + +ENTRYPOINT ["/bin/sh", "-c"] diff --git a/etl-benchmarks/README.md b/etl-benchmarks/README.md index 9f093e17d..b7fe0712c 100644 --- a/etl-benchmarks/README.md +++ b/etl-benchmarks/README.md @@ -13,6 +13,7 @@ Before running benchmarks, ensure you have: - A Postgres database set up - A publication created with the tables you want to benchmark - For BigQuery benchmarks: GCP project, dataset, and service account key file +- For Delta Lake benchmarks: Accessible storage URI (e.g., `s3://bucket/path`) and any required object store credentials ## Quick Start @@ -55,6 +56,23 @@ cargo bench --bench table_copies --features bigquery -- --log-target terminal ru --bq-sa-key-file /path/to/service-account-key.json ``` +### 4. Run Delta Lake Benchmark + +Benchmark against a Delta Lake table store: + +```bash +cargo bench --bench table_copies -- --log-target terminal run \ + --host localhost --port 5432 --database bench \ + --username postgres --password mypass \ + --publication-name bench_pub \ + --table-ids 1,2,3 \ + --destination delta-lake \ + --delta-base-uri s3://my-bucket/my-warehouse \ + --delta-storage-option endpoint=http://localhost:9010 \ + --delta-storage-option access_key_id=minio \ + --delta-storage-option secret_access_key=minio-secret +``` + ## Command Reference ### Common Parameters @@ -68,7 +86,7 @@ cargo bench --bench table_copies --features bigquery -- --log-target terminal ru | `--password` | Postgres password | (optional) | | `--publication-name` | Publication to replicate from | `bench_pub` | | `--table-ids` | Comma-separated table IDs to replicate | (required) | -| `--destination` | Destination type (`null` or `big-query`) | `null` | +| `--destination` | Destination type (`null`, `big-query`, or `delta-lake`) | `null` | ### Performance Tuning Parameters @@ -87,6 +105,13 @@ cargo bench --bench table_copies --features bigquery -- --log-target terminal ru | `--bq-sa-key-file` | Service account key file path | Yes | | `--bq-max-staleness-mins` | Max staleness in minutes | No | +### Delta Lake Parameters + +| Parameter | Description | Required for Delta Lake | +| ------------------------- | ------------------------------------------------ | ----------------------- | +| `--delta-base-uri` | Base URI for Delta tables (e.g., `s3://bucket`) | Yes | +| `--delta-storage-option` | Extra storage option in `key=value` form. Repeat per option. | No | + ### Logging Options | Parameter | Description | @@ -130,3 +155,22 @@ cargo bench --bench table_copies --features bigquery -- --log-target terminal ru ``` The benchmark will measure the time it takes to complete the initial table copy phase for all specified tables. + +## Local Docker Environment + +Start a ready-to-benchmark Postgres instance seeded with TPC-H data via Docker Compose: + +```bash +cd etl-benchmarks +docker compose up postgres tpch-seeder +``` + +The `tpch-seeder` service builds a lightweight image (see `Dockerfile.tpch-seeder`) that bundles the [`go-tpc`](https://github.com/pingcap/go-tpc) binary and runs the TPC-H loader after Postgres becomes healthy. Adjust credentials, port mapping, scale factor, or the go-tpc version by exporting `POSTGRES_USER`, `POSTGRES_PASSWORD`, `POSTGRES_DB`, `POSTGRES_PORT`, `TPCH_SCALE_FACTOR`, or `GO_TPC_VERSION` before launching Compose. Pass `--build` (or `--pull`) when changing `GO_TPC_VERSION` so Compose rebuilds the seeder image. + +To add an S3-compatible target for Delta Lake benchmarking, enable the optional `minio` profile: + +```bash +docker compose --profile minio up postgres tpch-seeder minio minio-setup +``` + +This exposes MinIO on `http://localhost:9010` (console on `http://localhost:9011`) with credentials `minio-admin` / `minio-admin-password` and creates the bucket defined by `MINIO_BUCKET` (default `delta-dev-and-test`). diff --git a/etl-benchmarks/benches/table_copies.rs b/etl-benchmarks/benches/table_copies.rs index 7537faa88..53ddeeb4d 100644 --- a/etl-benchmarks/benches/table_copies.rs +++ b/etl-benchmarks/benches/table_copies.rs @@ -8,10 +8,13 @@ use etl::types::{Event, TableRow}; use etl_config::Environment; use etl_config::shared::{BatchConfig, PgConnectionConfig, PipelineConfig, TlsConfig}; use etl_destinations::bigquery::{BigQueryDestination, install_crypto_provider_for_bigquery}; +use etl_destinations::deltalake::{DeltaDestinationConfig, DeltaLakeDestination}; use etl_postgres::types::TableId; use etl_telemetry::tracing::init_tracing; use sqlx::postgres::PgPool; +use std::collections::HashMap; use std::error::Error; +use std::str::FromStr; use tracing::info; #[derive(Parser, Debug)] @@ -52,8 +55,11 @@ enum DestinationType { Null, /// Use BigQuery as the destination BigQuery, + /// Use Delta Lake as the destination + DeltaLake, } +#[allow(clippy::large_enum_variant)] #[derive(Subcommand, Debug)] enum Commands { /// Run the table copies benchmark @@ -112,6 +118,12 @@ enum Commands { /// BigQuery maximum concurrent streams (optional) #[arg(long, default_value = "32")] bq_max_concurrent_streams: usize, + /// Delta Lake table base URI (required when using Delta Lake destination) + #[arg(long)] + delta_base_uri: Option, + /// Delta Lake object store storage option in the form key=value. Repeat to set multiple options. + #[arg(long = "delta-storage-option", value_parser = parse_key_val::)] + delta_storage_options: Vec<(String, String)>, }, /// Prepare the benchmark environment by cleaning up replication slots Prepare { @@ -136,6 +148,21 @@ enum Commands { }, } +fn parse_key_val(s: &str) -> Result<(T, U), String> +where + T: FromStr, + T::Err: std::fmt::Display, + U: FromStr, + U::Err: std::fmt::Display, +{ + let pos = s + .find('=') + .ok_or_else(|| format!("expected key=value but missing '=' in '{s}'"))?; + let key = T::from_str(&s[..pos]).map_err(|e| e.to_string())?; + let value = U::from_str(&s[pos + 1..]).map_err(|e| e.to_string())?; + Ok((key, value)) +} + #[tokio::main] async fn main() -> Result<(), Box> { // Filter out the --bench argument that cargo might add @@ -170,6 +197,8 @@ async fn main() -> Result<(), Box> { bq_sa_key_file, bq_max_staleness_mins, bq_max_concurrent_streams, + delta_base_uri, + delta_storage_options, } => { start_pipeline(RunArgs { host, @@ -190,6 +219,8 @@ async fn main() -> Result<(), Box> { bq_sa_key_file, bq_max_staleness_mins, bq_max_concurrent_streams, + delta_base_uri, + delta_storage_options, }) .await } @@ -234,6 +265,8 @@ struct RunArgs { bq_sa_key_file: Option, bq_max_staleness_mins: Option, bq_max_concurrent_streams: usize, + delta_base_uri: Option, + delta_storage_options: Vec<(String, String)>, } #[derive(Debug)] @@ -300,23 +333,43 @@ async fn prepare_benchmark(args: PrepareArgs) -> Result<(), Box> { } async fn start_pipeline(args: RunArgs) -> Result<(), Box> { + let RunArgs { + host, + port, + database, + username, + password, + tls_enabled, + tls_certs, + publication_name, + batch_max_size, + batch_max_fill_ms, + max_table_sync_workers, + table_ids, + destination, + bq_project_id, + bq_dataset_id, + bq_sa_key_file, + bq_max_staleness_mins, + bq_max_concurrent_streams, + delta_base_uri, + delta_storage_options, + } = args; + info!("Starting ETL pipeline benchmark"); - info!( - "Database: {}@{}:{}/{}", - args.username, args.host, args.port, args.database - ); - info!("Table IDs: {:?}", args.table_ids); - info!("Destination: {:?}", args.destination); + info!("Database: {}@{}:{}/{}", username, host, port, database); + info!("Table IDs: {:?}", table_ids); + info!("Destination: {:?}", destination); let pg_connection_config = PgConnectionConfig { - host: args.host, - port: args.port, - name: args.database, - username: args.username, - password: args.password.map(|p| p.into()), + host, + port, + name: database, + username: username.clone(), + password: password.map(|p| p.into()), tls: TlsConfig { - trusted_root_certs: args.tls_certs, - enabled: args.tls_enabled, + trusted_root_certs: tls_certs, + enabled: tls_enabled, }, }; @@ -324,31 +377,29 @@ async fn start_pipeline(args: RunArgs) -> Result<(), Box> { let pipeline_config = PipelineConfig { id: 1, - publication_name: args.publication_name, + publication_name, pg_connection: pg_connection_config, batch: BatchConfig { - max_size: args.batch_max_size, - max_fill_ms: args.batch_max_fill_ms, + max_size: batch_max_size, + max_fill_ms: batch_max_fill_ms, }, table_error_retry_delay_ms: 10000, table_error_retry_max_attempts: 5, - max_table_sync_workers: args.max_table_sync_workers, + max_table_sync_workers, }; // Create the appropriate destination based on the argument - let destination = match args.destination { + let destination = match destination { DestinationType::Null => BenchDestination::Null(NullDestination), DestinationType::BigQuery => { install_crypto_provider_for_bigquery(); - let project_id = args - .bq_project_id + let project_id = bq_project_id .ok_or("BigQuery project ID is required when using BigQuery destination")?; - let dataset_id = args - .bq_dataset_id + let dataset_id = bq_dataset_id .ok_or("BigQuery dataset ID is required when using BigQuery destination")?; - let sa_key_file = args.bq_sa_key_file.ok_or( + let sa_key_file = bq_sa_key_file.ok_or( "BigQuery service account key file is required when using BigQuery destination", )?; @@ -356,18 +407,36 @@ async fn start_pipeline(args: RunArgs) -> Result<(), Box> { project_id, dataset_id, &sa_key_file, - args.bq_max_staleness_mins, - args.bq_max_concurrent_streams, + bq_max_staleness_mins, + bq_max_concurrent_streams, store.clone(), ) .await?; BenchDestination::BigQuery(bigquery_dest) } + DestinationType::DeltaLake => { + let base_uri = delta_base_uri + .ok_or("Delta Lake base URI is required when using Delta Lake destination")?; + let storage_options = if delta_storage_options.is_empty() { + None + } else { + Some(delta_storage_options.into_iter().collect::>()) + }; + + let config = DeltaDestinationConfig { + base_uri, + storage_options, + table_config: HashMap::new(), + }; + + let delta_destination = DeltaLakeDestination::new(store.clone(), config); + BenchDestination::DeltaLake(delta_destination) + } }; let mut table_copied_notifications = vec![]; - for table_id in &args.table_ids { + for table_id in &table_ids { let table_copied = store .notify_on_table_state_type( TableId::new(*table_id), @@ -383,7 +452,7 @@ async fn start_pipeline(args: RunArgs) -> Result<(), Box> { info!( "Waiting for all {} tables to complete copy phase...", - args.table_ids.len() + table_ids.len() ); for notification in table_copied_notifications { notification.notified().await; @@ -405,6 +474,7 @@ struct NullDestination; enum BenchDestination { Null(NullDestination), BigQuery(BigQueryDestination), + DeltaLake(DeltaLakeDestination), } impl Destination for BenchDestination { @@ -416,6 +486,7 @@ impl Destination for BenchDestination { match self { BenchDestination::Null(dest) => dest.truncate_table(table_id).await, BenchDestination::BigQuery(dest) => dest.truncate_table(table_id).await, + BenchDestination::DeltaLake(dest) => dest.truncate_table(table_id).await, } } @@ -427,6 +498,7 @@ impl Destination for BenchDestination { match self { BenchDestination::Null(dest) => dest.write_table_rows(table_id, table_rows).await, BenchDestination::BigQuery(dest) => dest.write_table_rows(table_id, table_rows).await, + BenchDestination::DeltaLake(dest) => dest.write_table_rows(table_id, table_rows).await, } } @@ -434,6 +506,7 @@ impl Destination for BenchDestination { match self { BenchDestination::Null(dest) => dest.write_events(events).await, BenchDestination::BigQuery(dest) => dest.write_events(events).await, + BenchDestination::DeltaLake(dest) => dest.write_events(events).await, } } } diff --git a/etl-benchmarks/docker-compose.yml b/etl-benchmarks/docker-compose.yml new file mode 100644 index 000000000..c7432b4e0 --- /dev/null +++ b/etl-benchmarks/docker-compose.yml @@ -0,0 +1,100 @@ +# Docker Compose setup for ETL benchmarks. +# +# Provides a Postgres database seeded with TPC-H data using the go-tpc project. +# Includes an optional "minio" profile for running an S3-compatible object +# store useful when benchmarking Delta Lake destinations. +services: + postgres: + image: postgres:16 + container_name: etl-benchmarks-postgres + environment: + POSTGRES_USER: ${POSTGRES_USER:-postgres} + POSTGRES_PASSWORD: ${POSTGRES_PASSWORD:-postgres} + POSTGRES_DB: ${POSTGRES_DB:-bench} + ports: + - "${POSTGRES_PORT:-5430}:5432" + volumes: + - postgres-data:/var/lib/postgresql/data + healthcheck: + test: ["CMD-SHELL", "pg_isready -U ${POSTGRES_USER:-postgres} -d ${POSTGRES_DB:-bench}"] + interval: 10s + timeout: 5s + retries: 10 + start_period: 10s + + tpch-seeder: + build: + context: . + dockerfile: Dockerfile.tpch-seeder + args: + GO_TPC_VERSION: ${GO_TPC_VERSION:-latest} + container_name: etl-benchmarks-tpch-seeder + depends_on: + postgres: + condition: service_healthy + environment: + POSTGRES_HOST: postgres + POSTGRES_PORT: ${POSTGRES_PORT_INTERNAL:-5432} + POSTGRES_DB: ${POSTGRES_DB:-bench} + POSTGRES_USER: ${POSTGRES_USER:-postgres} + POSTGRES_PASSWORD: ${POSTGRES_PASSWORD:-postgres} + TPCH_SCALE_FACTOR: ${TPCH_SCALE_FACTOR:-1} + command: >- + set -euo pipefail; + echo "Waiting for Postgres at ${POSTGRES_HOST}:${POSTGRES_PORT}..."; + until pg_isready -h ${POSTGRES_HOST} -p ${POSTGRES_PORT} -U ${POSTGRES_USER}; do + sleep 2; + done; + echo "Postgres is ready. Loading TPC-H data..."; + go-tpc tpch prepare \ + --sf ${TPCH_SCALE_FACTOR} \ + -H ${POSTGRES_HOST} \ + -P ${POSTGRES_PORT} \ + -D ${POSTGRES_DB} \ + -U ${POSTGRES_USER} \ + -p ${POSTGRES_PASSWORD} \ + --analyze; + echo "TPC-H data loaded."; + restart: "no" + + # Optional MinIO profile for S3-compatible storage used by Delta Lake. + minio: + profiles: ["minio"] + image: minio/minio:RELEASE.2024-10-13T13-34-11Z + container_name: etl-benchmarks-minio + environment: + MINIO_ROOT_USER: ${MINIO_ROOT_USER:-minio-admin} + MINIO_ROOT_PASSWORD: ${MINIO_ROOT_PASSWORD:-minio-admin-password} + command: server /data --console-address ":9001" + ports: + - "${MINIO_PORT:-9010}:9000" + - "${MINIO_CONSOLE_PORT:-9011}:9001" + volumes: + - minio-data:/data + + minio-setup: + profiles: ["minio"] + image: minio/mc:RELEASE.2024-10-06T11-22-18Z + container_name: etl-benchmarks-minio-setup + depends_on: + minio: + condition: service_started + environment: + MINIO_ROOT_USER: ${MINIO_ROOT_USER:-minio-admin} + MINIO_ROOT_PASSWORD: ${MINIO_ROOT_PASSWORD:-minio-admin-password} + MINIO_BUCKET: ${MINIO_BUCKET:-delta-dev-and-test} + entrypoint: ["/bin/sh", "-c"] + command: >- + set -euo pipefail; + until mc alias set local http://minio:9000 ${MINIO_ROOT_USER} ${MINIO_ROOT_PASSWORD}; do + echo "Waiting for MinIO..."; + sleep 2; + done; + mc mb -p local/${MINIO_BUCKET}; + mc anonymous set download local/${MINIO_BUCKET}; + echo "MinIO bucket '${MINIO_BUCKET}' ready."; + restart: "no" + +volumes: + postgres-data: + minio-data: diff --git a/etl-benchmarks/scripts/benchmark.sh b/etl-benchmarks/scripts/benchmark.sh index 19d2c8281..708ad06d2 100755 --- a/etl-benchmarks/scripts/benchmark.sh +++ b/etl-benchmarks/scripts/benchmark.sh @@ -8,6 +8,7 @@ set -eo pipefail # Supported destinations: # - null: Discards all data (fastest, default) # - big-query: Streams data to Google BigQuery +# - delta-lake: Writes to a Delta Lake table store (delta-rs) # # Environment Variables: # Database Configuration: @@ -15,7 +16,7 @@ set -eo pipefail # # Benchmark Configuration: # HYPERFINE_RUNS, PUBLICATION_NAME, BATCH_MAX_SIZE, BATCH_MAX_FILL_MS, MAX_TABLE_SYNC_WORKERS -# DESTINATION (null or big-query) +# DESTINATION (null, big-query, or delta-lake) # LOG_TARGET (terminal or file) - Where to send logs (default: terminal) # DRY_RUN (true/false) - Show commands without executing them # @@ -24,6 +25,11 @@ set -eo pipefail # BQ_DATASET_ID - BigQuery dataset ID # BQ_SA_KEY_FILE - Path to service account key JSON file # BQ_MAX_STALENESS_MINS - Optional staleness setting +# BQ_MAX_CONCURRENT_STREAMS - Optional concurrent stream hint +# +# Delta Lake Configuration (required when DESTINATION=delta-lake): +# DELTA_BASE_URI - Base URI for the Delta Lake tables, e.g. s3://bucket/path +# DELTA_STORAGE_OPTIONS - Optional comma-separated key=value pairs for object store options # # Examples: # # Run with null destination and terminal logs (default) @@ -41,6 +47,12 @@ set -eo pipefail # BQ_DATASET_ID=my_dataset \ # BQ_SA_KEY_FILE=/path/to/sa-key.json \ # ./etl-benchmarks/scripts/benchmark.sh +# +# # Run with Delta Lake destination (MinIO example) +# DESTINATION=delta-lake \ +# DELTA_BASE_URI=s3://delta-dev-and-test/bench \ +# DELTA_STORAGE_OPTIONS="endpoint=http://localhost:9010,access_key_id=minio,secret_access_key=minio-password,allow_http=true" \ +# ./etl-benchmarks/scripts/benchmark.sh # Check if hyperfine is installed if ! [ -x "$(command -v hyperfine)" ]; then @@ -75,12 +87,16 @@ MAX_TABLE_SYNC_WORKERS="${MAX_TABLE_SYNC_WORKERS:=8}" LOG_TARGET="${LOG_TARGET:=terminal}" # terminal or file # Destination configuration -DESTINATION="${DESTINATION:=null}" # null or big-query +DESTINATION="${DESTINATION:=null}" # null, big-query, or delta-lake BQ_PROJECT_ID="${BQ_PROJECT_ID:=}" BQ_DATASET_ID="${BQ_DATASET_ID:=}" BQ_SA_KEY_FILE="${BQ_SA_KEY_FILE:=}" BQ_MAX_STALENESS_MINS="${BQ_MAX_STALENESS_MINS:=}" BQ_MAX_CONCURRENT_STREAMS="${BQ_MAX_CONCURRENT_STREAMS:=}" +DELTA_BASE_URI="${DELTA_BASE_URI:=}" +DELTA_STORAGE_OPTIONS="${DELTA_STORAGE_OPTIONS:=}" + +IFS=',' read -r -a DELTA_STORAGE_OPTIONS_ARRAY <<< "${DELTA_STORAGE_OPTIONS}" # Optional dry-run mode DRY_RUN="${DRY_RUN:=false}" @@ -106,6 +122,11 @@ if [[ "${DESTINATION}" == "big-query" ]]; then if [[ -n "${BQ_MAX_CONCURRENT_STREAMS}" ]]; then echo " BigQuery Max Concurrent Streams: ${BQ_MAX_CONCURRENT_STREAMS}" fi +elif [[ "${DESTINATION}" == "delta-lake" ]]; then + echo " Delta Lake Base URI: ${DELTA_BASE_URI}" + if [[ -n "${DELTA_STORAGE_OPTIONS}" ]]; then + echo " Delta Lake Storage Options: ${DELTA_STORAGE_OPTIONS}" + fi fi # Get table IDs from the database for TPC-C tables @@ -143,17 +164,34 @@ if [[ "${DESTINATION}" == "big-query" ]]; then echo "❌ Error: BigQuery service account key file does not exist: ${BQ_SA_KEY_FILE}" exit 1 fi +elif [[ "${DESTINATION}" == "delta-lake" ]]; then + if [[ -z "${DELTA_BASE_URI}" ]]; then + echo "❌ Error: DELTA_BASE_URI environment variable is required when using Delta Lake destination." + exit 1 + fi + for opt in "${DELTA_STORAGE_OPTIONS_ARRAY[@]}"; do + [[ -z "${opt}" ]] && continue + if [[ "${opt}" != *=* ]]; then + echo "❌ Error: Invalid DELTA_STORAGE_OPTIONS entry '${opt}'. Expected key=value format." + exit 1 + fi + done fi # Determine if we need BigQuery features FEATURES_FLAG="" -if [[ "${DESTINATION}" == "big-query" ]]; then - FEATURES_FLAG="--features bigquery" -fi +case "${DESTINATION}" in + big-query) + FEATURES_FLAG="--features bigquery" + ;; + delta-lake) + FEATURES_FLAG="--features deltalake" + ;; +esac # Validate destination option -if [[ "${DESTINATION}" != "null" && "${DESTINATION}" != "big-query" ]]; then - echo "❌ Error: Invalid destination '${DESTINATION}'. Supported values: null, big-query" +if [[ "${DESTINATION}" != "null" && "${DESTINATION}" != "big-query" && "${DESTINATION}" != "delta-lake" ]]; then + echo "❌ Error: Invalid destination '${DESTINATION}'. Supported values: null, big-query, delta-lake" exit 1 fi @@ -188,6 +226,12 @@ if [[ "${DESTINATION}" == "big-query" ]]; then if [[ -n "${BQ_MAX_CONCURRENT_STREAMS}" ]]; then RUN_CMD="${RUN_CMD} --bq-max-concurrent-streams ${BQ_MAX_CONCURRENT_STREAMS}" fi +elif [[ "${DESTINATION}" == "delta-lake" ]]; then + RUN_CMD="${RUN_CMD} --delta-base-uri ${DELTA_BASE_URI}" + for opt in "${DELTA_STORAGE_OPTIONS_ARRAY[@]}"; do + [[ -z "${opt}" ]] && continue + RUN_CMD="${RUN_CMD} --delta-storage-option ${opt}" + done fi echo "" diff --git a/etl-benchmarks/scripts/mem_profile.sh b/etl-benchmarks/scripts/mem_profile.sh index 132a5fc25..20424b470 100755 --- a/etl-benchmarks/scripts/mem_profile.sh +++ b/etl-benchmarks/scripts/mem_profile.sh @@ -20,7 +20,9 @@ set -euo pipefail # RUN_LABEL Run name label for trace. Default: auto timestamped # TRACE_DIR Output directory for traces. Default: target/instruments # LOG_TARGET Benchmark logs target (terminal|file). Default: terminal -# DESTINATION Destination (null|big-query). Default: null +# DESTINATION Destination (null|big-query|delta-lake). Default: null +# DELTA_BASE_URI Required when DESTINATION=delta-lake (e.g., s3://bucket/path) +# DELTA_STORAGE_OPTIONS Optional comma-separated key=value list for object store config # # Database connection (same defaults as benchmark.sh / prepare_tpcc.sh): # POSTGRES_USER Default: postgres @@ -105,10 +107,14 @@ BATCH_MAX_FILL_MS="${BATCH_MAX_FILL_MS:=10000}" MAX_TABLE_SYNC_WORKERS="${MAX_TABLE_SYNC_WORKERS:=8}" LOG_TARGET="${LOG_TARGET:=terminal}" DESTINATION="${DESTINATION:=null}" +DELTA_BASE_URI="${DELTA_BASE_URI:=}" +DELTA_STORAGE_OPTIONS="${DELTA_STORAGE_OPTIONS:=}" + +IFS=',' read -r -a DELTA_STORAGE_OPTIONS_ARRAY <<< "${DELTA_STORAGE_OPTIONS}" # Validate destination -if [[ "${DESTINATION}" != "null" && "${DESTINATION}" != "big-query" ]]; then - echo "❌ Invalid DESTINATION='${DESTINATION}'. Supported: null, big-query" >&2 +if [[ "${DESTINATION}" != "null" && "${DESTINATION}" != "big-query" && "${DESTINATION}" != "delta-lake" ]]; then + echo "❌ Invalid DESTINATION='${DESTINATION}'. Supported: null, big-query, delta-lake" >&2 exit 1 fi if [[ "${LOG_TARGET}" != "terminal" && "${LOG_TARGET}" != "file" ]]; then @@ -124,6 +130,18 @@ if [[ "${DESTINATION}" == "big-query" ]]; then echo "❌ BigQuery SA key file not found: ${BQ_SA_KEY_FILE}" >&2 exit 1 fi +elif [[ "${DESTINATION}" == "delta-lake" ]]; then + if [[ -z "${DELTA_BASE_URI}" ]]; then + echo "❌ DELTA_BASE_URI is required for DESTINATION=delta-lake" >&2 + exit 1 + fi + for opt in "${DELTA_STORAGE_OPTIONS_ARRAY[@]}"; do + [[ -z "${opt}" ]] && continue + if [[ "${opt}" != *=* ]]; then + echo "❌ Invalid DELTA_STORAGE_OPTIONS entry '${opt}'. Expected key=value." >&2 + exit 1 + fi + done fi echo "🧪 Memory profiling with cargo-instruments" @@ -134,6 +152,12 @@ echo " Label: ${RUN_LABEL}" echo " Trace dir:${TRACE_DIR}" echo " Open UI: ${OPEN_TRACE}" echo " Dest: ${DESTINATION}" +if [[ "${DESTINATION}" == "delta-lake" ]]; then + echo " Delta URI:${DELTA_BASE_URI}" + if [[ -n "${DELTA_STORAGE_OPTIONS}" ]]; then + echo " Delta Opts:${DELTA_STORAGE_OPTIONS}" + fi +fi # Build common bench arg tail build_bench_args() { @@ -167,6 +191,12 @@ build_bench_args() { args+=("--destination" "${DESTINATION}") if [[ "${DESTINATION}" == "big-query" ]]; then args+=("--bq-project-id" "${BQ_PROJECT_ID}" "--bq-dataset-id" "${BQ_DATASET_ID}" "--bq-sa-key-file" "${BQ_SA_KEY_FILE}") + elif [[ "${DESTINATION}" == "delta-lake" ]]; then + args+=("--delta-base-uri" "${DELTA_BASE_URI}") + for opt in "${DELTA_STORAGE_OPTIONS_ARRAY[@]}"; do + [[ -z "${opt}" ]] && continue + args+=("--delta-storage-option" "${opt}") + done fi printf '%q ' "${args[@]}" } diff --git a/etl-config/src/shared/destination.rs b/etl-config/src/shared/destination.rs index 75639e22c..ba3992990 100644 --- a/etl-config/src/shared/destination.rs +++ b/etl-config/src/shared/destination.rs @@ -1,3 +1,5 @@ +use std::collections::HashMap; + use serde::{Deserialize, Serialize}; use crate::SerializableSecretString; @@ -8,6 +10,7 @@ use crate::SerializableSecretString; /// Each variant corresponds to a different supported destination system. #[derive(Debug, Clone, Serialize, Deserialize)] #[serde(rename_all = "snake_case")] +#[non_exhaustive] pub enum DestinationConfig { /// In-memory destination for ephemeral or test data. Memory, @@ -41,4 +44,12 @@ pub enum DestinationConfig { /// - and the configured batch size. max_concurrent_streams: usize, }, + DeltaLake { + base_uri: String, + storage_options: Option>, + #[serde(skip_serializing_if = "Option::is_none")] + partition_columns: Option>>, + #[serde(skip_serializing_if = "Option::is_none")] + optimize_after_commits: Option, + }, } diff --git a/etl-destinations/Cargo.toml b/etl-destinations/Cargo.toml index 062ac3926..a5ec61bb4 100644 --- a/etl-destinations/Cargo.toml +++ b/etl-destinations/Cargo.toml @@ -18,20 +18,31 @@ bigquery = [ iceberg = [ "dep:iceberg", "dep:iceberg-catalog-rest", - "dep:arrow", "dep:parquet", "dep:uuid", + "dep:arrow", +] +deltalake = [ + "dep:dashmap", + "dep:deltalake", + "dep:futures", + "dep:tokio", + "dep:tracing", + "dep:arrow", ] [dependencies] etl = { workspace = true } -chrono = { workspace = true } arrow = { workspace = true, optional = true } +chrono = { workspace = true } +dashmap = { workspace = true, optional = true } +deltalake = { workspace = true, optional = true, default-features = false, features = ["datafusion", "rustls"] } gcp-bigquery-client = { workspace = true, optional = true, features = [ "rust-tls", "aws-lc-rs", ] } +futures = { workspace = true, optional = true } iceberg = { workspace = true, optional = true } iceberg-catalog-rest = { workspace = true, optional = true } parquet = { workspace = true, optional = true, features = ["async", "arrow"] } @@ -47,6 +58,8 @@ uuid = { workspace = true, optional = true, features = ["v4"] } [dev-dependencies] etl = { workspace = true, features = ["test-utils"] } etl-telemetry = { workspace = true } +deltalake = { workspace = true, default-features = false, features = ["rustls", "datafusion", "s3"] } +insta = { workspace = true } base64 = { workspace = true } chrono = { workspace = true } diff --git a/etl-destinations/src/iceberg/encoding.rs b/etl-destinations/src/arrow/encoding.rs similarity index 88% rename from etl-destinations/src/iceberg/encoding.rs rename to etl-destinations/src/arrow/encoding.rs index 9a6c9f33c..eb722d993 100644 --- a/etl-destinations/src/iceberg/encoding.rs +++ b/etl-destinations/src/arrow/encoding.rs @@ -2,17 +2,21 @@ use std::sync::Arc; use arrow::{ array::{ - ArrayRef, ArrowPrimitiveType, BooleanBuilder, FixedSizeBinaryBuilder, LargeBinaryBuilder, - ListBuilder, PrimitiveBuilder, RecordBatch, StringBuilder, TimestampMicrosecondBuilder, + ArrayRef, ArrowPrimitiveType, BinaryBuilder, BooleanBuilder, Decimal128Array, + Decimal128Builder, FixedSizeBinaryBuilder, LargeBinaryBuilder, ListBuilder, + PrimitiveBuilder, RecordBatch, StringBuilder, TimestampMicrosecondBuilder, }, datatypes::{ - DataType, Date32Type, FieldRef, Float32Type, Float64Type, Int32Type, Int64Type, Schema, - Time64MicrosecondType, TimeUnit, TimestampMicrosecondType, + DataType, Date32Type, FieldRef, Float32Type, Float64Type, Int16Type, Int32Type, Int64Type, + Schema, Time64MicrosecondType, TimeUnit, TimestampMicrosecondType, UInt32Type, }, error::ArrowError, }; use chrono::{NaiveDate, NaiveTime}; -use etl::types::{ArrayCell, Cell, DATE_FORMAT, TIME_FORMAT, TIMESTAMP_FORMAT, TableRow}; +use etl::{ + conversions::numeric::Sign, + types::{ArrayCell, Cell, DATE_FORMAT, TIME_FORMAT, TIMESTAMP_FORMAT, TableRow}, +}; pub const UNIX_EPOCH: NaiveDate = NaiveDate::from_ymd_opt(1970, 1, 1).expect("unix epoch is a valid date"); @@ -21,6 +25,30 @@ const MIDNIGHT: NaiveTime = NaiveTime::from_hms_opt(0, 0, 0).expect("midnight is const UUID_BYTE_WIDTH: i32 = 16; +/// Extract numeric precision from Postgres atttypmod +/// Based on: https://stackoverflow.com/questions/72725508/how-to-calculate-numeric-precision-and-other-vals-from-atttypmod +pub fn extract_numeric_precision(atttypmod: i32) -> u8 { + if atttypmod == -1 { + // No limit specified, use maximum precision + 38 + } else { + let precision = ((atttypmod - 4) >> 16) & 65535; + std::cmp::min(precision as u8, 38) // Cap at Arrow's max precision + } +} + +/// Extract numeric scale from Postgres atttypmod +/// Based on: https://stackoverflow.com/questions/72725508/how-to-calculate-numeric-precision-and-other-vals-from-atttypmod +pub fn extract_numeric_scale(atttypmod: i32) -> u8 { + if atttypmod == -1 { + // No limit specified, use reasonable default scale + 18 + } else { + let scale = (atttypmod - 4) & 65535; + std::cmp::min(scale as u8, 38) // Cap at reasonable scale + } +} + /// Converts a slice of [`TableRow`]s to an Arrow [`RecordBatch`]. /// /// This function transforms tabular data from the ETL pipeline's internal format @@ -61,16 +89,28 @@ fn build_array_for_field(rows: &[TableRow], field_idx: usize, data_type: &DataTy DataType::Float32 => build_primitive_array::(rows, field_idx, cell_to_f32), DataType::Float64 => build_primitive_array::(rows, field_idx, cell_to_f64), DataType::Utf8 => build_string_array(rows, field_idx), - DataType::LargeBinary => build_binary_array(rows, field_idx), + DataType::Binary => build_binary_array(rows, field_idx), + DataType::LargeBinary => build_large_binary_array(rows, field_idx), DataType::Date32 => build_primitive_array::(rows, field_idx, cell_to_date32), DataType::Time64(TimeUnit::Microsecond) => { build_primitive_array::(rows, field_idx, cell_to_time64) } + DataType::Time64(TimeUnit::Nanosecond) => { + build_primitive_array::(rows, field_idx, cell_to_time64) + } DataType::Timestamp(TimeUnit::Microsecond, Some(tz)) => { build_timestamptz_array(rows, field_idx, tz) } DataType::Timestamp(TimeUnit::Microsecond, None) => { - build_primitive_array::(rows, field_idx, cell_to_timestamp) + // Support both naive timestamps and time-of-day encoded as timestamp-from-epoch. + build_primitive_array::( + rows, + field_idx, + cell_to_timestamp_or_time, + ) + } + DataType::Decimal128(precision, scale) => { + build_decimal128_array(rows, field_idx, *precision, *scale) } DataType::FixedSizeBinary(UUID_BYTE_WIDTH) => build_uuid_array(rows, field_idx), DataType::List(field) => build_list_array(rows, field_idx, field.clone()), @@ -121,7 +161,24 @@ macro_rules! impl_array_builder { impl_array_builder!(build_boolean_array, BooleanBuilder, cell_to_bool); impl_array_builder!(build_string_array, StringBuilder, cell_to_string); -impl_array_builder!(build_binary_array, LargeBinaryBuilder, cell_to_bytes); +impl_array_builder!(build_binary_array, BinaryBuilder, cell_to_bytes); +impl_array_builder!(build_large_binary_array, LargeBinaryBuilder, cell_to_bytes); + +/// Builds a decimal128 array from [`TableRow`]s for a specific field. +fn build_decimal128_array( + rows: &[TableRow], + field_idx: usize, + precision: u8, + scale: i8, +) -> ArrayRef { + let values: Vec> = rows + .iter() + .map(|row| cell_to_decimal128(&row.values[field_idx], precision, scale)) + .collect(); + + let decimal_type = DataType::Decimal128(precision, scale); + Arc::new(Decimal128Array::from(values).with_data_type(decimal_type)) +} /// Builds a timezone-aware timestamp array from [`TableRow`]s. /// @@ -136,7 +193,12 @@ fn build_timestamptz_array(rows: &[TableRow], field_idx: usize, tz: &str) -> Arr let mut builder = TimestampMicrosecondBuilder::new().with_timezone(tz); for row in rows { - let arrow_value = cell_to_timestamptz(&row.values[field_idx]); + // Accept either timestamptz values or time-of-day (encoded from epoch date). + let arrow_value = match &row.values[field_idx] { + Cell::TimestampTz(_) => cell_to_timestamptz(&row.values[field_idx]), + Cell::Time(_) => cell_to_time_as_timestamp(&row.values[field_idx]), + _ => None, + }; builder.append_option(arrow_value); } @@ -235,6 +297,101 @@ fn cell_to_f64(cell: &Cell) -> Option { } } +/// Converts a [`Cell`] to a decimal128 value. +fn cell_to_decimal128(cell: &Cell, precision: u8, scale: i8) -> Option { + match cell { + Cell::Numeric(n) => pg_numeric_to_decimal_i128(n, precision as i32, scale as i32), + _ => None, + } +} + +/// Convert PgNumeric to a scaled i128 matching Decimal128(precision, scale) exactly using string math. +fn pg_numeric_to_decimal_i128( + n: &etl::types::PgNumeric, + precision: i32, + scale: i32, +) -> Option { + if precision <= 0 || scale < 0 || scale > precision { + return None; + } + + match n { + etl::types::PgNumeric::NaN + | etl::types::PgNumeric::PositiveInfinity + | etl::types::PgNumeric::NegativeInfinity => None, + etl::types::PgNumeric::Value { + sign, + weight, + scale: _, + digits, + } => { + if digits.is_empty() { + return Some(0); + } + + // Compose base-10000 groups into an integer accumulator. + let mut acc: i128 = 0; + for &g in digits.iter() { + let gi = g as i128; + acc = acc.checked_mul(10_000)?.checked_add(gi)?; + } + + // Decimal 10^ exponent to align composed base-10000 integer with actual value, + // then apply desired target scale. Do NOT use pg_scale here; value is fully + // described by digits and weight. + let shift_groups = *weight as i32 - (digits.len() as i32 - 1); + let exp10 = shift_groups * 4 + scale; + + // Apply 10^exp10 scaling with checked math. + fn pow10_i128(mut e: i32) -> Option { + if e < 0 { + return None; + } + let mut r: i128 = 1; + while e > 0 { + r = r.checked_mul(10)?; + e -= 1; + } + Some(r) + } + + if exp10 >= 0 { + acc = acc.checked_mul(pow10_i128(exp10)?)?; + } else { + let div = pow10_i128(-exp10)?; + acc /= div; // truncate toward zero + } + + // Apply sign + let is_negative = matches!(sign, Sign::Negative); + if is_negative { + acc = -acc; + } + + // Enforce precision limit + fn count_digits(mut v: i128) -> i32 { + if v == 0 { + return 1; + } + if v < 0 { + v = -v; + } + let mut c = 0; + while v > 0 { + v /= 10; + c += 1; + } + c + } + if count_digits(acc) > precision { + return None; + } + + Some(acc) + } + } +} + /// Converts a [`Cell`] to a byte vector. /// /// Extracts binary data from [`Cell::Bytes`] variants by cloning the @@ -282,6 +439,20 @@ fn cell_to_timestamp(cell: &Cell) -> Option { } } +/// Converts a [`Cell`] to a 64-bit timestamp value (microseconds since Unix epoch), +/// accepting either a naive timestamp or a time-of-day. +/// +/// - [`Cell::Timestamp`] is converted like [`cell_to_timestamp`]. +/// - [`Cell::Time`] is converted to microseconds since Unix epoch by treating it as +/// a time on the Unix epoch date (1970-01-01T00:00:00). +fn cell_to_timestamp_or_time(cell: &Cell) -> Option { + match cell { + Cell::Timestamp(_) => cell_to_timestamp(cell), + Cell::Time(time) => time.signed_duration_since(MIDNIGHT).num_microseconds(), + _ => None, + } +} + /// Converts a [`Cell`] to a timezone-aware timestamp value (microseconds since Unix epoch). /// /// Transforms timezone-aware [`Cell::TimestampTz`] values into microseconds @@ -375,20 +546,27 @@ fn cell_to_array_cell(cell: &Cell) -> Option<&ArrayCell> { fn build_list_array(rows: &[TableRow], field_idx: usize, field: FieldRef) -> ArrayRef { match field.data_type() { DataType::Boolean => build_boolean_list_array(rows, field_idx, field), + DataType::Int16 => build_int16_list_array(rows, field_idx, field), DataType::Int32 => build_int32_list_array(rows, field_idx, field), DataType::Int64 => build_int64_list_array(rows, field_idx, field), + DataType::UInt32 => build_uint32_list_array(rows, field_idx, field), DataType::Float32 => build_float32_list_array(rows, field_idx, field), DataType::Float64 => build_float64_list_array(rows, field_idx, field), DataType::Utf8 => build_string_list_array(rows, field_idx, field), - DataType::LargeBinary => build_binary_list_array(rows, field_idx, field), + DataType::Binary => build_binary_list_array(rows, field_idx, field), + DataType::LargeBinary => build_large_binary_list_array(rows, field_idx, field), DataType::Date32 => build_date32_list_array(rows, field_idx, field), DataType::Time64(TimeUnit::Microsecond) => build_time64_list_array(rows, field_idx, field), + DataType::Time64(TimeUnit::Nanosecond) => build_time64_list_array(rows, field_idx, field), DataType::Timestamp(TimeUnit::Microsecond, None) => { build_timestamp_list_array(rows, field_idx, field) } DataType::Timestamp(TimeUnit::Microsecond, Some(_)) => { build_timestamptz_list_array(rows, field_idx, field) } + DataType::Decimal128(precision, scale) => { + build_decimal128_list_array(rows, field_idx, field.clone(), *precision, *scale) + } DataType::FixedSizeBinary(UUID_BYTE_WIDTH) => build_uuid_list_array(rows, field_idx, field), // For unsupported element types, fall back to string representation _ => build_list_array_for_strings(rows, field_idx, field), @@ -421,6 +599,32 @@ fn build_boolean_list_array(rows: &[TableRow], field_idx: usize, field: FieldRef Arc::new(list_builder.finish()) } +/// Builds a list array for 16-bit integer elements. +fn build_int16_list_array(rows: &[TableRow], field_idx: usize, field: FieldRef) -> ArrayRef { + let mut list_builder = + ListBuilder::new(PrimitiveBuilder::::new()).with_field(field.clone()); + + for row in rows { + if let Some(array_cell) = cell_to_array_cell(&row.values[field_idx]) { + match array_cell { + ArrayCell::I16(vec) => { + for item in vec { + list_builder.values().append_option(*item); + } + list_builder.append(true); + } + _ => { + return build_list_array_for_strings(rows, field_idx, field); + } + } + } else { + list_builder.append_null(); + } + } + + Arc::new(list_builder.finish()) +} + /// Builds a list array for 32-bit integer elements. fn build_int32_list_array(rows: &[TableRow], field_idx: usize, field: FieldRef) -> ArrayRef { let mut list_builder = @@ -485,6 +689,32 @@ fn build_int64_list_array(rows: &[TableRow], field_idx: usize, field: FieldRef) Arc::new(list_builder.finish()) } +/// Builds a list array for 32-bit unsigned integer elements. +fn build_uint32_list_array(rows: &[TableRow], field_idx: usize, field: FieldRef) -> ArrayRef { + let mut list_builder = + ListBuilder::new(PrimitiveBuilder::::new()).with_field(field.clone()); + + for row in rows { + if let Some(array_cell) = cell_to_array_cell(&row.values[field_idx]) { + match array_cell { + ArrayCell::U32(vec) => { + for item in vec { + list_builder.values().append_option(*item); + } + list_builder.append(true); + } + _ => { + return build_list_array_for_strings(rows, field_idx, field); + } + } + } else { + list_builder.append_null(); + } + } + + Arc::new(list_builder.finish()) +} + /// Builds a list array for 32-bit float elements. fn build_float32_list_array(rows: &[TableRow], field_idx: usize, field: FieldRef) -> ArrayRef { let mut list_builder = @@ -585,6 +815,34 @@ fn build_string_list_array(rows: &[TableRow], field_idx: usize, field: FieldRef) /// Builds a list array for binary elements. fn build_binary_list_array(rows: &[TableRow], field_idx: usize, field: FieldRef) -> ArrayRef { + let mut list_builder = ListBuilder::new(BinaryBuilder::new()).with_field(field.clone()); + + for row in rows { + if let Some(array_cell) = cell_to_array_cell(&row.values[field_idx]) { + match array_cell { + ArrayCell::Bytes(vec) => { + for item in vec { + match item { + Some(bytes) => list_builder.values().append_value(bytes), + None => list_builder.values().append_null(), + } + } + list_builder.append(true); + } + _ => { + return build_list_array_for_strings(rows, field_idx, field); + } + } + } else { + list_builder.append_null(); + } + } + + Arc::new(list_builder.finish()) +} + +/// Builds a list array for large binary elements. +fn build_large_binary_list_array(rows: &[TableRow], field_idx: usize, field: FieldRef) -> ArrayRef { let mut list_builder = ListBuilder::new(LargeBinaryBuilder::new()).with_field(field.clone()); for row in rows { @@ -683,6 +941,15 @@ fn build_timestamp_list_array(rows: &[TableRow], field_idx: usize, field: FieldR } list_builder.append(true); } + ArrayCell::Time(vec) => { + for item in vec { + let arrow_value = item.and_then(|time| { + time.signed_duration_since(MIDNIGHT).num_microseconds() + }); + list_builder.values().append_option(arrow_value); + } + list_builder.append(true); + } _ => { return build_list_array_for_strings(rows, field_idx, field); } @@ -717,6 +984,15 @@ fn build_timestamptz_list_array(rows: &[TableRow], field_idx: usize, field: Fiel } list_builder.append(true); } + ArrayCell::Time(vec) => { + for item in vec { + let arrow_value = item.and_then(|time| { + time.signed_duration_since(MIDNIGHT).num_microseconds() + }); + list_builder.values().append_option(arrow_value); + } + list_builder.append(true); + } _ => { return build_list_array_for_strings(rows, field_idx, field); } @@ -729,6 +1005,14 @@ fn build_timestamptz_list_array(rows: &[TableRow], field_idx: usize, field: Fiel Arc::new(list_builder.finish()) } +/// Converts a [`Cell::Time`] to a timestamp-from-epoch in microseconds. +fn cell_to_time_as_timestamp(cell: &Cell) -> Option { + match cell { + Cell::Time(time) => time.signed_duration_since(MIDNIGHT).num_microseconds(), + _ => None, + } +} + /// Builds a list array for UUID elements. fn build_uuid_list_array(rows: &[TableRow], field_idx: usize, field: FieldRef) -> ArrayRef { let mut list_builder = @@ -763,6 +1047,46 @@ fn build_uuid_list_array(rows: &[TableRow], field_idx: usize, field: FieldRef) - Arc::new(list_builder.finish()) } +/// Builds a list array for Decimal128 elements. +fn build_decimal128_list_array( + rows: &[TableRow], + field_idx: usize, + field: FieldRef, + precision: u8, + scale: i8, +) -> ArrayRef { + let mut list_builder = ListBuilder::new( + Decimal128Builder::new().with_data_type(DataType::Decimal128(precision, scale)), + ) + .with_field(field.clone()); + + for row in rows { + if let Some(array_cell) = cell_to_array_cell(&row.values[field_idx]) { + match array_cell { + ArrayCell::Numeric(vec) => { + for item in vec { + let val = item.as_ref().and_then(|n| { + pg_numeric_to_decimal_i128(n, precision as i32, scale as i32) + }); + match val { + Some(v) => list_builder.values().append_value(v), + None => list_builder.values().append_null(), + } + } + list_builder.append(true); + } + _ => { + return build_list_array_for_strings(rows, field_idx, field); + } + } + } else { + list_builder.append_null(); + } + } + + Arc::new(list_builder.finish()) +} + /// Builds a list array for string elements. /// /// This function creates an Arrow list array with string elements by processing @@ -1331,12 +1655,11 @@ mod tests { }, ]; - let array_ref = build_array_for_field(&rows, 0, &DataType::LargeBinary); + let array_ref = build_array_for_field(&rows, 0, &DataType::Binary); let binary_array = array_ref .as_any() - .downcast_ref::() + .downcast_ref::() .unwrap(); - assert_eq!(binary_array.len(), 4); assert_eq!(binary_array.value(0), test_bytes); assert_eq!(binary_array.value(1), Vec::::new()); @@ -1344,6 +1667,37 @@ mod tests { assert!(binary_array.is_null(3)); } + #[test] + fn test_build_large_binary_array() { + let test_bytes = vec![1, 2, 3, 4, 5]; + let rows = vec![ + TableRow { + values: vec![Cell::Bytes(test_bytes.clone())], + }, + TableRow { + values: vec![Cell::Bytes(vec![])], + }, + TableRow { + values: vec![Cell::Null], + }, + TableRow { + values: vec![Cell::String("not bytes".to_string())], + }, + ]; + + let array_ref = build_array_for_field(&rows, 0, &DataType::LargeBinary); + let large_binary_array = array_ref + .as_any() + .downcast_ref::() + .unwrap(); + + assert_eq!(large_binary_array.len(), 4); + assert_eq!(large_binary_array.value(0), test_bytes); + assert_eq!(large_binary_array.value(1), Vec::::new()); + assert!(large_binary_array.is_null(2)); + assert!(large_binary_array.is_null(3)); + } + #[test] fn test_build_date32_array() { use chrono::NaiveDate; @@ -1512,6 +1866,47 @@ mod tests { assert!(uuid_array.is_null(2)); } + #[test] + fn test_build_decimal128_array() { + use arrow::datatypes::{Field, Schema}; + use etl::types::PgNumeric; + + let rows = vec![ + TableRow { + values: vec![Cell::Numeric("123.45".parse::().unwrap())], + }, + TableRow { + values: vec![Cell::Numeric("-0.01".parse::().unwrap())], + }, + TableRow { + values: vec![Cell::Null], + }, + TableRow { + values: vec![Cell::Numeric("0".parse::().unwrap())], + }, + ]; + + let schema = Schema::new(vec![Field::new( + "amount", + DataType::Decimal128(10, 2), + true, + )]); + + let batch = rows_to_record_batch(&rows, schema).unwrap(); + let dec_array = batch + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + + assert_eq!(dec_array.len(), 4); + assert_eq!(dec_array.data_type(), &DataType::Decimal128(10, 2)); + assert_eq!(dec_array.value(0), 12_345); // 123.45 -> 12345 (scale 2) + assert_eq!(dec_array.value(1), -1); // -0.01 -> -1 (scale 2) + assert!(dec_array.is_null(2)); + assert_eq!(dec_array.value(3), 0); + } + #[test] fn test_rows_to_record_batch_simple() { use arrow::datatypes::{Field, Schema}; @@ -2291,7 +2686,7 @@ mod tests { use arrow::array::ListArray; use arrow::datatypes::Field; - let field = Field::new("items", DataType::LargeBinary, true); + let field = Field::new("items", DataType::Binary, true); let field_ref = Arc::new(field); let test_bytes_1 = vec![1, 2, 3, 4, 5]; @@ -2329,7 +2724,7 @@ mod tests { let first_list = list_array.value(0); let binary_array = first_list .as_any() - .downcast_ref::() + .downcast_ref::() .unwrap(); assert_eq!(binary_array.len(), 3); assert_eq!(binary_array.value(0), test_bytes_1); @@ -2341,7 +2736,7 @@ mod tests { let second_list = list_array.value(1); let binary_array = second_list .as_any() - .downcast_ref::() + .downcast_ref::() .unwrap(); assert_eq!(binary_array.len(), 1); assert_eq!(binary_array.value(0), empty_bytes); @@ -2733,6 +3128,59 @@ mod tests { assert!(list_array.is_null(3)); } + #[test] + fn test_build_decimal128_list_array() { + use arrow::array::ListArray; + use arrow::datatypes::Field; + use etl::types::PgNumeric; + + let precision: u8 = 10; + let scale: i8 = 2; + + let field = Field::new("item", DataType::Decimal128(precision, scale), true); + let field_ref = Arc::new(field); + + let rows = vec![ + TableRow { + values: vec![Cell::Array(ArrayCell::Numeric(vec![ + Some("123.45".parse::().unwrap()), + None, + Some("-0.01".parse::().unwrap()), + ]))], + }, + TableRow { + values: vec![Cell::Array(ArrayCell::Numeric(vec![]))], + }, // empty list + TableRow { + values: vec![Cell::Null], + }, // null list + ]; + + let array_ref = build_decimal128_list_array(&rows, 0, field_ref.clone(), precision, scale); + let list_array = array_ref.as_any().downcast_ref::().unwrap(); + + assert_eq!(list_array.len(), 3); + + // Row 0 + assert!(!list_array.is_null(0)); + let first_list = list_array.value(0); + let dec_array = first_list + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(dec_array.len(), 3); + assert_eq!(dec_array.value(0), 12_345); // 123.45 + assert!(dec_array.is_null(1)); + assert_eq!(dec_array.value(2), -1); // -0.01 + + // Row 1: empty list + assert!(!list_array.is_null(1)); + assert_eq!(list_array.value(1).len(), 0); + + // Row 2: null list + assert!(list_array.is_null(2)); + } + #[test] fn test_build_list_array_for_strings() { use arrow::array::ListArray; diff --git a/etl-destinations/src/arrow/mod.rs b/etl-destinations/src/arrow/mod.rs new file mode 100644 index 000000000..3a973a3d3 --- /dev/null +++ b/etl-destinations/src/arrow/mod.rs @@ -0,0 +1,3 @@ +pub mod encoding; + +pub use encoding::*; diff --git a/etl-destinations/src/deltalake/config.rs b/etl-destinations/src/deltalake/config.rs new file mode 100644 index 000000000..aa831acb2 --- /dev/null +++ b/etl-destinations/src/deltalake/config.rs @@ -0,0 +1,56 @@ +use std::num::NonZeroU64; + +use deltalake::parquet::{ + basic::Compression, + file::properties::{WriterProperties, WriterVersion}, +}; + +const DEFAULT_PARQUET_VERSION: WriterVersion = WriterVersion::PARQUET_1_0; +const DEFAULT_COMPRESSION: Compression = Compression::SNAPPY; +const DEFAULT_COMPACT_AFTER_COMMITS: u64 = 100; + +/// Configuration for a Delta table +#[derive(Debug, Clone)] +pub struct DeltaTableConfig { + /// Whether the table is append-only, i.e no updates or deletes are allowed + pub append_only: bool, + /// Parquet version to use for the table + pub parquet_version: WriterVersion, + /// Compression to use for the table + pub compression: Compression, + /// Columns to use for Z-ordering + pub z_order_columns: Option>, + /// Run OPTIMIZE every N commits (None = disabled) + pub compact_after_commits: Option, + /// Run Z-ordering every N commits (None = disabled) + pub z_order_after_commits: Option, +} + +impl From for WriterProperties { + fn from(value: DeltaTableConfig) -> Self { + WriterProperties::from(&value) + } +} + +impl From<&DeltaTableConfig> for WriterProperties { + fn from(value: &DeltaTableConfig) -> Self { + let mut builder = WriterProperties::builder(); + builder = builder.set_writer_version(value.parquet_version); + builder = builder.set_compression(value.compression); + builder.build() + } +} + +impl Default for DeltaTableConfig { + fn default() -> Self { + Self { + append_only: false, + parquet_version: DEFAULT_PARQUET_VERSION, + // good default + compression: DEFAULT_COMPRESSION, + z_order_columns: None, + compact_after_commits: Some(NonZeroU64::new(DEFAULT_COMPACT_AFTER_COMMITS).unwrap()), + z_order_after_commits: None, + } + } +} diff --git a/etl-destinations/src/deltalake/core.rs b/etl-destinations/src/deltalake/core.rs new file mode 100644 index 000000000..ef2d6faaf --- /dev/null +++ b/etl-destinations/src/deltalake/core.rs @@ -0,0 +1,489 @@ +use dashmap::DashMap; +use dashmap::Entry::{Occupied, Vacant}; +use deltalake::datafusion::logical_expr::Expr; +use deltalake::table::builder::parse_table_uri; +use deltalake::{DeltaOps, DeltaTable, DeltaTableBuilder, DeltaTableError, TableProperty}; +use etl::destination::Destination; +use etl::error::{ErrorKind, EtlResult}; +use etl::store::schema::SchemaStore; +use etl::store::state::StateStore; +use etl::types::{Event, TableId, TableRow as PgTableRow, TableSchema as PgTableSchema}; +use etl::{bail, etl_error}; +use futures::future::try_join_all; +use std::collections::HashMap; +use std::sync::Arc; +use tokio::sync::Mutex; +use tracing::{info, trace}; + +use crate::arrow::rows_to_record_batch; +use crate::deltalake::config::DeltaTableConfig; +use crate::deltalake::events::{materialize_events, materialize_events_append_only}; +use crate::deltalake::maintenance::TableMaintenanceState; +use crate::deltalake::operations::{append_to_table, delete_from_table, merge_to_table}; +use crate::deltalake::schema::{postgres_to_arrow_schema, postgres_to_delta_schema}; + +/// Configuration for Delta Lake destination +#[derive(Debug, Clone)] +pub struct DeltaDestinationConfig { + /// Base URI for Delta table storage (e.g., "s3://bucket/warehouse", "file:///tmp/delta") + pub base_uri: String, + /// Optional storage options passed to underlying object store + pub storage_options: Option>, + /// Table configuration (per table) + pub table_config: HashMap>, +} + +/// Delta Lake destination implementation +#[derive(Clone)] +pub struct DeltaLakeDestination { + store: S, + config: DeltaDestinationConfig, + /// Cache of opened Delta tables, keyed by postgres table id + // This isn't using a RWLock because we are overwhelmingly write-heavy + table_cache: DashMap>>, + /// Tracks in-flight maintenance tasks and the versions they cover. + maintenance: DashMap>, +} + +impl DeltaLakeDestination +where + S: StateStore + SchemaStore + Send + Sync, +{ + /// Create a new Delta Lake destination + pub fn new(store: S, config: DeltaDestinationConfig) -> Self { + Self { + store, + config, + table_cache: DashMap::new(), + maintenance: DashMap::new(), + } + } + + fn config_for_table_name(&self, table_name: &str) -> Arc { + self.config + .table_config + .get(table_name) + .cloned() + .unwrap_or_else(|| Arc::new(DeltaTableConfig::default())) + } + + fn maintenance_state_for( + &self, + table_id: TableId, + current_version: i64, + ) -> Arc { + match self.maintenance.entry(table_id) { + Occupied(entry) => Arc::clone(entry.get()), + Vacant(entry) => { + let state = Arc::new(TableMaintenanceState::new(current_version)); + entry.insert(state.clone()); + state + } + } + } + + /// Returns a cached table handle or loads it if missing. + async fn table_handle(&self, table_id: &TableId) -> EtlResult>> { + let handle = match self.table_cache.entry(*table_id) { + Occupied(entry) => entry.into_ref(), + Vacant(entry) => { + let table = self.get_or_create_table(table_id).await?; + entry.insert(Arc::new(Mutex::new(table))) + } + } + .downgrade(); + + Ok(Arc::clone(handle.value())) + } + + /// Gets or creates a Delta table for a given table id if it doesn't exist. + async fn get_or_create_table(&self, table_id: &TableId) -> EtlResult { + let table_schema = self + .store + .get_table_schema(table_id) + .await? + .ok_or_else(|| { + etl_error!( + ErrorKind::MissingTableSchema, + "Table schema not found", + format!("Schema for table {} not found in store", table_id.0) + ) + })?; + + let table_name = &table_schema.name.name; + let pg_table_schema = &table_schema.name.schema; + let table_path = parse_table_uri(format!( + "{}/{}/{}", + self.config.base_uri, pg_table_schema, table_name + )) + .map_err(|e| etl_error!(ErrorKind::DestinationError, "Failed to parse table path", e))?; + + let mut table_builder = DeltaTableBuilder::from_uri(table_path).map_err(|e| { + etl_error!( + ErrorKind::DestinationError, + "Failed to create Delta table builder", + e + ) + })?; + if let Some(storage_options) = &self.config.storage_options { + table_builder = table_builder.with_storage_options(storage_options.clone()); + } + let mut table = table_builder.build().map_err(|e| { + etl_error!( + ErrorKind::DestinationError, + "Failed to build Delta table", + e + ) + })?; + + let ops: DeltaOps = match table.load().await { + Ok(_) => return Ok(table), + Err(DeltaTableError::NotATable(_)) => table.into(), + Err(e) => { + bail!(ErrorKind::DestinationError, "Failed to load Delta table", e); + } + }; + + let delta_schema = postgres_to_delta_schema(&table_schema).map_err(|e| { + etl_error!( + ErrorKind::ConversionError, + "Failed to convert table schema to Delta schema", + e + ) + })?; + + let config = self.config_for_table_name(table_name); + + let mut builder = ops + .create() + // TODO(abhi): Figure out how to avoid the clone + .with_columns(delta_schema.fields().cloned()); + + if config.append_only { + builder = builder + .with_configuration_property(TableProperty::AppendOnly, Some("true".to_string())); + } + + let table = builder.await.map_err(|e| { + etl_error!( + ErrorKind::DestinationError, + "Failed to create Delta table", + e + ) + })?; + + Ok(table) + } + + /// Process events grouped by table + async fn process_events_by_table(&self, events: Vec) -> EtlResult<()> { + if events.is_empty() { + return Ok(()); + } + + let mut events_by_table: HashMap> = HashMap::new(); + + for event in events.into_iter() { + match event { + Event::Insert(ref e) => { + events_by_table.entry(e.table_id).or_default().push(event); + } + Event::Update(ref e) => { + events_by_table.entry(e.table_id).or_default().push(event); + } + Event::Delete(ref e) => { + events_by_table.entry(e.table_id).or_default().push(event); + } + Event::Truncate(ref e) => { + // Truncate events affect multiple tables (relation IDs) + for &rel_id in &e.rel_ids { + let table_id = TableId(rel_id); + events_by_table + .entry(table_id) + .or_default() + .push(event.clone()); + } + } + Event::Relation(ref e) => { + // Schema change events - store the table schema + let table_id = e.table_schema.id; + events_by_table.entry(table_id).or_default().push(event); + } + Event::Begin(_) | Event::Commit(_) | Event::Unsupported => { + // Skip transaction control events - they don't affect specific tables + } + } + } + + info!("Processing events for {} tables", events_by_table.len()); + + let tasks: Vec<_> = events_by_table + .into_iter() + .filter(|(_, events)| !events.is_empty()) + .map(|(table_id, events)| self.process_table_events(table_id, events)) + .collect(); + + try_join_all(tasks).await?; + + Ok(()) + } + + /// Process events for a specific table, compacting them into a single consistent state + async fn process_table_events(&self, table_id: TableId, events: Vec) -> EtlResult<()> { + let table_schema = self + .store + .get_table_schema(&table_id) + .await? + .ok_or_else(|| { + etl_error!( + ErrorKind::MissingTableSchema, + "Table schema not found", + table_id + ) + })?; + + let is_append_only = self + .config_for_table_name(&table_schema.name.name) + .append_only; + + if is_append_only { + let row_refs = materialize_events_append_only(&events, &table_schema)?; + let rows: Vec = row_refs.into_iter().cloned().collect(); + self.write_table_rows_internal(&table_id, &rows).await?; + } else { + let (delete_predicates, row_refs) = materialize_events(&events, &table_schema)?; + let rows: Vec = row_refs.into_iter().cloned().collect(); + self.execute_delete_append_transaction_expr( + table_id, + &table_schema, + &rows, + delete_predicates, + ) + .await?; + } + + Ok(()) + } + + /// Execute delete+append transaction for CDC using DataFusion expressions for keys + async fn execute_delete_append_transaction_expr( + &self, + table_id: TableId, + table_schema: &PgTableSchema, + upsert_rows: &[PgTableRow], + delete_predicates: Vec, + ) -> EtlResult<()> { + let combined_predicate = delete_predicates.into_iter().reduce(|acc, e| acc.or(e)); + + if upsert_rows.is_empty() && combined_predicate.is_none() { + return Ok(()); + } + + let config = self.config_for_table_name(&table_schema.name.name); + let table = self.table_handle(&table_id).await?; + + if upsert_rows.is_empty() { + if let Some(combined_predicate) = combined_predicate { + trace!("Deleting rows from table {}", table_id); + + let mut table_guard = table.lock().await; + delete_from_table(&mut table_guard, config.as_ref(), combined_predicate) + .await + .map_err(|e| { + etl_error!( + ErrorKind::DestinationError, + "Failed to delete rows from Delta table", + format!("Error deleting from table for table_id {}: {}", table_id, e) + ) + })?; + + let version = table_guard.version().unwrap_or_default(); + drop(table_guard); + + self.maybe_schedule_maintenance(table_id, table, version, config) + .await?; + } + return Ok(()); + } + + trace!( + "Appending {} upserted rows to table {}", + upsert_rows.len(), + table_id, + ); + + let mut table_guard = table.lock().await; + + merge_to_table( + &mut table_guard, + config.as_ref(), + table_schema, + upsert_rows, + combined_predicate, + ) + .await + .map_err(|e| { + etl_error!( + ErrorKind::DestinationError, + "Failed to append rows to Delta table", + format!( + "Error appending to table for table_id {}: {}", + table_id.0, e + ) + ) + })?; + + let version = table_guard.version().ok_or_else(|| { + etl_error!( + ErrorKind::DestinationError, + "Failed to get version from Delta table", + format!("Error getting version from table for table_id {}", table_id) + ) + })?; + drop(table_guard); + + self.maybe_schedule_maintenance(table_id, table, version, config) + .await?; + + Ok(()) + } + + async fn write_table_rows_internal( + &self, + table_id: &TableId, + table_rows: &[PgTableRow], + ) -> EtlResult<()> { + if table_rows.is_empty() { + return Ok(()); + } + + let table = self.table_handle(table_id).await?; + + let table_schema = self + .store + .get_table_schema(table_id) + .await? + .ok_or_else(|| { + etl_error!( + ErrorKind::MissingTableSchema, + "Table schema not found", + format!("Schema for table {} not found in store", table_id.0) + ) + })?; + + let row_length = table_rows.len(); + trace!("Writing {} rows to Delta table", row_length); + + let mut table_guard = table.lock().await; + let arrow_schema = postgres_to_arrow_schema(&table_schema).map_err(|e| { + etl_error!( + ErrorKind::ConversionError, + "Failed to convert table schema to Arrow schema", + e + ) + })?; + + let config = self.config_for_table_name(&table_schema.name.name); + + let record_batch = rows_to_record_batch(table_rows, arrow_schema.clone()).map_err(|e| { + etl_error!( + ErrorKind::ConversionError, + "Failed to encode table rows", + format!("Error converting to Arrow: {}", e) + ) + })?; + append_to_table(&mut table_guard, config.as_ref(), record_batch) + .await + .map_err(|e| { + etl_error!( + ErrorKind::DestinationError, + "Failed to write to Delta table", + format!("Error writing to table for table_id {}: {}", table_id, e) + ) + })?; + + info!( + "Successfully wrote {} rows to Delta table for table_id: {}", + row_length, table_id.0 + ); + + let version = table_guard.version().unwrap_or_default(); + drop(table_guard); + + self.maybe_schedule_maintenance(*table_id, table, version, config) + .await?; + + Ok(()) + } + + /// Schedules compaction or Z-ordering tasks if thresholds are met. + async fn maybe_schedule_maintenance( + &self, + table_id: TableId, + table: Arc>, + table_version: i64, + config: Arc, + ) -> EtlResult<()> { + if table_version < 0 { + panic!("Table version is less than 0"); + } + + if config.compact_after_commits.is_none() && config.z_order_after_commits.is_none() { + return Ok(()); + } + + let maintenance_state = self.maintenance_state_for(table_id, table_version); + + maintenance_state + .maybe_run_compaction( + table_id, + Arc::clone(&table), + Arc::clone(&config), + table_version, + ) + .await; + maintenance_state + .maybe_run_zorder( + table_id, + Arc::clone(&table), + Arc::clone(&config), + table_version, + ) + .await; + + Ok(()) + } +} + +impl Destination for DeltaLakeDestination +where + S: StateStore + SchemaStore + Send + Sync, +{ + fn name() -> &'static str { + "deltalake" + } + + async fn truncate_table(&self, _table_id: TableId) -> EtlResult<()> { + Ok(()) + } + + async fn write_table_rows( + &self, + table_id: TableId, + table_rows: Vec, + ) -> EtlResult<()> { + self.write_table_rows_internal(&table_id, &table_rows).await + } + + async fn write_events(&self, events: Vec) -> EtlResult<()> { + if events.is_empty() { + return Ok(()); + } + + info!("Processing {} events for Delta destination", events.len()); + + self.process_events_by_table(events).await?; + + Ok(()) + } +} diff --git a/etl-destinations/src/deltalake/events.rs b/etl-destinations/src/deltalake/events.rs new file mode 100644 index 000000000..7096d28c1 --- /dev/null +++ b/etl-destinations/src/deltalake/events.rs @@ -0,0 +1,420 @@ +use crate::deltalake::util::LWWReg; +use deltalake::datafusion::{common::HashMap, prelude::Expr}; +use etl::{ + error::EtlResult, + types::{Event, PgLsn, TableRow as PgTableRow, TableSchema as PgTableSchema}, +}; +use tracing::warn; + +use crate::deltalake::expr::build_pk_expr; + +#[derive(Debug, Clone, PartialEq)] +enum RowOp<'a> { + Upsert(&'a PgTableRow), + Delete, +} + +pub fn materialize_events_append_only<'a>( + events: &'a [Event], + table_schema: &PgTableSchema, +) -> EtlResult> { + let mut crdt_by_key: HashMap> = HashMap::new(); + + for event in events.iter() { + match event { + Event::Insert(e) => { + let marker = (e.commit_lsn, e.start_lsn); + let pk_expr = build_pk_expr(table_schema, &e.table_row)?; + let entry = crdt_by_key.entry(pk_expr).or_insert_with(|| LWWReg { + val: RowOp::Upsert(&e.table_row), + marker, + }); + entry.update(RowOp::Upsert(&e.table_row), marker); + } + Event::Update(_) => { + warn!("Received update event for append-only table, ignoring"); + } + Event::Delete(_) => { + warn!("Received delete event for append-only table, ignoring"); + } + Event::Relation(_) + | Event::Begin(_) + | Event::Commit(_) + | Event::Truncate(_) + | Event::Unsupported => { + // Skip non-row events + } + } + } + + let mut upsert_rows: Vec<&PgTableRow> = Vec::new(); + for (_, reg) in crdt_by_key.into_iter() { + if let RowOp::Upsert(row) = reg.val { + upsert_rows.push(row) + } + } + + Ok(upsert_rows) +} + +/// Materialize events into delete and upsert predicates +pub fn materialize_events<'a>( + events: &'a [Event], + table_schema: &PgTableSchema, +) -> EtlResult<(Vec, Vec<&'a PgTableRow>)> { + let mut crdt_by_key: HashMap> = HashMap::new(); + + for event in events.iter() { + match event { + Event::Insert(e) => { + let marker = (e.commit_lsn, e.start_lsn); + let pk_expr = build_pk_expr(table_schema, &e.table_row)?; + let entry = crdt_by_key.entry(pk_expr).or_insert_with(|| LWWReg { + val: RowOp::Upsert(&e.table_row), + marker, + }); + entry.update(RowOp::Upsert(&e.table_row), marker); + } + Event::Update(e) => { + let marker = (e.commit_lsn, e.start_lsn); + let pk_expr = build_pk_expr(table_schema, &e.table_row)?; + let entry = crdt_by_key.entry(pk_expr).or_insert_with(|| LWWReg { + val: RowOp::Upsert(&e.table_row), + marker, + }); + entry.update(RowOp::Upsert(&e.table_row), marker); + } + Event::Delete(e) => { + if let Some((_, ref old_row)) = e.old_table_row { + let marker = (e.commit_lsn, e.start_lsn); + let pk_expr = build_pk_expr(table_schema, old_row)?; + let entry = crdt_by_key.entry(pk_expr).or_insert_with(|| LWWReg { + val: RowOp::Delete, + marker, + }); + entry.update(RowOp::Delete, marker); + } else { + warn!("Delete event missing old_table_row for table"); + } + } + Event::Truncate(_) => { + // TODO(abhi): Implement truncate event handling + warn!("Truncate event not implemented"); + } + Event::Relation(_) | Event::Begin(_) | Event::Commit(_) | Event::Unsupported => { + // Skip non-row events + } + } + } + + let mut delete_predicates: Vec = Vec::new(); + let mut upsert_rows: Vec<&PgTableRow> = Vec::new(); + + for (expr, reg) in crdt_by_key.into_iter() { + match reg.val { + RowOp::Delete => delete_predicates.push(expr), + RowOp::Upsert(row) => upsert_rows.push(row), + } + } + + Ok((delete_predicates, upsert_rows)) +} + +#[cfg(test)] +mod tests { + use super::*; + use etl::types::{ + Cell as PgCell, ColumnSchema as PgColumnSchema, DeleteEvent, InsertEvent, TableId, + TableName, Type as PgType, UpdateEvent, + }; + use insta::assert_debug_snapshot; + + fn schema_single_pk(table_id: TableId) -> PgTableSchema { + PgTableSchema::new( + table_id, + TableName::new("public".to_string(), "t".to_string()), + vec![ + PgColumnSchema { + name: "id".to_string(), + typ: PgType::INT8, + modifier: -1, + primary: true, + nullable: false, + }, + PgColumnSchema { + name: "name".to_string(), + typ: PgType::TEXT, + modifier: -1, + primary: false, + nullable: true, + }, + ], + ) + } + + fn row(id: i64, name: &str) -> PgTableRow { + PgTableRow { + values: vec![PgCell::I64(id), PgCell::String(name.to_string())], + } + } + + fn schema_composite_pk(table_id: TableId) -> PgTableSchema { + PgTableSchema::new( + table_id, + TableName::new("public".to_string(), "t".to_string()), + vec![ + PgColumnSchema { + name: "tenant_id".to_string(), + typ: PgType::INT4, + modifier: -1, + primary: true, + nullable: false, + }, + PgColumnSchema { + name: "user_id".to_string(), + typ: PgType::INT8, + modifier: -1, + primary: true, + nullable: false, + }, + PgColumnSchema { + name: "name".to_string(), + typ: PgType::TEXT, + modifier: -1, + primary: false, + nullable: true, + }, + ], + ) + } + + fn row_composite(tenant: i32, user: i64, name: &str) -> PgTableRow { + PgTableRow { + values: vec![ + PgCell::I32(tenant), + PgCell::I64(user), + PgCell::String(name.to_string()), + ], + } + } + + #[test] + fn lww_reg_uses_commit_then_start_lsn() { + let table_id = TableId(1); + let schema = schema_single_pk(table_id); + + // Earlier commit/start pair + let e1 = Event::Insert(InsertEvent { + start_lsn: PgLsn::from(10u64), + commit_lsn: PgLsn::from(20u64), + table_id, + table_row: row(1, "a"), + }); + // Later commit wins + let e2 = Event::Insert(InsertEvent { + start_lsn: PgLsn::from(11u64), + commit_lsn: PgLsn::from(21u64), + table_id, + table_row: row(1, "b"), + }); + + let events = vec![e1, e2]; + + let (deletes, upserts) = materialize_events(&events, &schema).unwrap(); + assert!(deletes.is_empty()); + assert_debug_snapshot!(upserts, @r#" + [ + TableRow { + values: [ + I64( + 1, + ), + String( + "b", + ), + ], + }, + ] + "#); + } + + #[test] + fn delete_overrides_prior_upsert_for_same_pk() { + let table_id = TableId(1); + let schema = schema_single_pk(table_id); + + let ins = Event::Insert(InsertEvent { + start_lsn: PgLsn::from(1u64), + commit_lsn: PgLsn::from(2u64), + table_id, + table_row: row(1, "a"), + }); + let del = Event::Delete(DeleteEvent { + start_lsn: PgLsn::from(3u64), + commit_lsn: PgLsn::from(4u64), + table_id, + old_table_row: Some((false, row(1, "a"))), + }); + + let events = vec![ins, del]; + + let (deletes, upserts) = materialize_events(&events, &schema).unwrap(); + assert_debug_snapshot!(deletes, @r#" + [ + BinaryExpr( + BinaryExpr { + left: Column( + Column { + relation: None, + name: "id", + }, + ), + op: Eq, + right: Literal( + Int64(1), + None, + ), + }, + ), + ] + "#); + assert!(upserts.is_empty()); + } + + #[test] + fn update_on_append_only_is_ignored() { + let table_id = TableId(1); + let schema = schema_single_pk(table_id); + let ins = Event::Insert(InsertEvent { + start_lsn: PgLsn::from(1u64), + commit_lsn: PgLsn::from(2u64), + table_id, + table_row: row(1, "a"), + }); + let upd = Event::Update(UpdateEvent { + start_lsn: PgLsn::from(3u64), + commit_lsn: PgLsn::from(4u64), + table_id, + table_row: row(1, "b"), + old_table_row: Some((false, row(1, "a"))), + }); + + let events = vec![ins, upd]; + + let upserts = materialize_events_append_only(&events, &schema).unwrap(); + assert_debug_snapshot!(upserts, @r#" + [ + TableRow { + values: [ + I64( + 1, + ), + String( + "a", + ), + ], + }, + ] + "#); + } + + #[test] + fn composite_pk_predicate_and_lww() { + let table_id = TableId(42); + let schema = schema_composite_pk(table_id); + + // Inserts for two different composite PKs + let ins1 = Event::Insert(InsertEvent { + start_lsn: PgLsn::from(1u64), + commit_lsn: PgLsn::from(2u64), + table_id, + table_row: row_composite(10, 100, "a"), + }); + let ins2 = Event::Insert(InsertEvent { + start_lsn: PgLsn::from(1u64), + commit_lsn: PgLsn::from(2u64), + table_id, + table_row: row_composite(10, 101, "b"), + }); + + // Update to the first composite key with later commit/start + let upd1 = Event::Update(UpdateEvent { + start_lsn: PgLsn::from(3u64), + commit_lsn: PgLsn::from(4u64), + table_id, + table_row: row_composite(10, 100, "a2"), + old_table_row: Some((false, row_composite(10, 100, "a"))), + }); + + // Delete the second composite key with even later lsn + let del2 = Event::Delete(DeleteEvent { + start_lsn: PgLsn::from(5u64), + commit_lsn: PgLsn::from(6u64), + table_id, + old_table_row: Some((false, row_composite(10, 101, "b"))), + }); + + let events = vec![ins1, ins2, upd1, del2]; + + let (deletes, upserts) = materialize_events(&events, &schema).unwrap(); + + // We expect one delete predicate (for tenant_id=10 AND user_id=101) + // and one upsert (tenant_id=10 AND user_id=100 with name=a2) + assert_debug_snapshot!(deletes, @r#" + [ + BinaryExpr( + BinaryExpr { + left: BinaryExpr( + BinaryExpr { + left: Column( + Column { + relation: None, + name: "tenant_id", + }, + ), + op: Eq, + right: Literal( + Int32(10), + None, + ), + }, + ), + op: And, + right: BinaryExpr( + BinaryExpr { + left: Column( + Column { + relation: None, + name: "user_id", + }, + ), + op: Eq, + right: Literal( + Int64(101), + None, + ), + }, + ), + }, + ), + ] + "#); + assert_debug_snapshot!(upserts, @r#" + [ + TableRow { + values: [ + I32( + 10, + ), + I64( + 100, + ), + String( + "a2", + ), + ], + }, + ] + "#); + } +} diff --git a/etl-destinations/src/deltalake/expr.rs b/etl-destinations/src/deltalake/expr.rs new file mode 100644 index 000000000..01106b4b4 --- /dev/null +++ b/etl-destinations/src/deltalake/expr.rs @@ -0,0 +1,599 @@ +// Utilities related to constructing DataFusion expressions + +use deltalake::datafusion::prelude::Expr; +use deltalake::datafusion::scalar::ScalarValue; +use deltalake::datafusion::{common::Column, prelude::lit}; +use etl::{ + error::{ErrorKind, EtlResult}, + etl_error, + types::{ + Cell as PgCell, ColumnSchema as PgColumnSchema, TableId, TableName, TableRow as PgTableRow, + TableSchema as PgTableSchema, + }, +}; + +use crate::{arrow::rows_to_record_batch, deltalake::schema::postgres_to_arrow_schema}; + +/// Build a DataFusion predicate `Expr` representing equality over all primary key columns +/// for the provided `row` according to `table_schema`. +pub fn build_pk_expr(table_schema: &PgTableSchema, row: &PgTableRow) -> EtlResult { + let mut pk_expr: Option = None; + for (idx, column_schema) in table_schema.column_schemas.iter().enumerate() { + if !column_schema.primary { + continue; + } + let value_expr = cell_to_scalar_expr(&row.values[idx], column_schema)?; + let this_col_expr = + Expr::Column(Column::new_unqualified(column_schema.name.clone())).eq(value_expr); + pk_expr = Some(match pk_expr { + None => this_col_expr, + Some(acc) => acc.and(this_col_expr), + }); + } + + // In practice, this should never happen as the tables we're replicating are guaranteed to have primary keys + pk_expr.ok_or(etl_error!( + ErrorKind::ConversionError, + "Table has no primary key columns" + )) +} + +/// Convert a Postgres [`PgCell`] into a DataFusion [`Expr`] literal. +fn cell_to_scalar_expr(cell: &PgCell, column_schema: &PgColumnSchema) -> EtlResult { + let single_col_schema = PgTableSchema { + id: TableId::new(0), + name: TableName::new("foo".to_string(), "bar".to_string()), + column_schemas: vec![column_schema.clone()], + }; + + let arrow_schema = postgres_to_arrow_schema(&single_col_schema).map_err(|e| { + etl_error!( + ErrorKind::ConversionError, + "Failed to convert table schema to Arrow schema", + e + ) + })?; + let temp_row = vec![PgTableRow::new(vec![cell.clone()])]; + let array = rows_to_record_batch(&temp_row, arrow_schema).map_err(|e| { + etl_error!( + ErrorKind::ConversionError, + "Failed to convert row to Arrow array", + e + ) + })?; + let array = array.column(0); + let scalar_value = ScalarValue::try_from_array(array, 0).map_err(|e| { + etl_error!( + ErrorKind::ConversionError, + "Failed to convert cell to scalar expression", + e + ) + })?; + Ok(lit(scalar_value)) +} + +/// Turns a set of primary key column expressions into qualified equality expressions +/// matching merge target/source. +/// +/// Takes column expressions and creates qualified equality comparisons between +/// source and target aliases for merge operations. +/// +/// # Examples +/// - `col("id")` becomes `source.id = target.id` +/// - `[col("tenant_id"), col("user_id")]` becomes `source.tenant_id = target.tenant_id AND source.user_id = target.user_id` +pub fn qualify_primary_keys( + primary_keys: Vec, + source_alias: &str, + target_alias: &str, +) -> Option { + primary_keys + .into_iter() + .filter_map(|key_expr| { + // Extract column name from the expression + let column_name = match key_expr { + Expr::Column(column) => column.name, + _ => return None, + }; + + let source_col = Expr::Column(Column::new(Some(source_alias), &column_name)); + let target_col = Expr::Column(Column::new(Some(target_alias), &column_name)); + + Some(source_col.eq(target_col)) + }) + .fold(None, |acc: Option, eq_expr| match acc { + None => Some(eq_expr), + Some(acc) => Some(acc.and(eq_expr)), + }) +} + +#[cfg(test)] +mod tests { + use super::*; + use chrono::{NaiveDate, NaiveDateTime, NaiveTime}; + use deltalake::datafusion::logical_expr::{col, lit}; + use etl::types::{ColumnSchema as PgColumnSchema, TableName, Type as PgType}; + use insta::assert_debug_snapshot; + + /// Create a test table schema with various column types. + fn create_test_schema() -> PgTableSchema { + PgTableSchema { + id: etl::types::TableId(1), + name: TableName::new("public".to_string(), "test_table".to_string()), + column_schemas: vec![ + PgColumnSchema::new("id".to_string(), PgType::INT8, -1, false, true), // Primary key + PgColumnSchema::new("name".to_string(), PgType::TEXT, -1, true, false), + PgColumnSchema::new("age".to_string(), PgType::INT4, -1, true, false), + PgColumnSchema::new("is_active".to_string(), PgType::BOOL, -1, true, false), + PgColumnSchema::new("created_at".to_string(), PgType::TIMESTAMP, -1, true, false), + ], + } + } + + /// Create a test table schema with multiple primary key columns. + fn create_composite_pk_schema() -> PgTableSchema { + PgTableSchema { + id: etl::types::TableId(2), + name: TableName::new("public".to_string(), "composite_pk_table".to_string()), + column_schemas: vec![ + PgColumnSchema::new("tenant_id".to_string(), PgType::INT4, -1, false, true), // Primary key 1 + PgColumnSchema::new("user_id".to_string(), PgType::INT8, -1, false, true), // Primary key 2 + PgColumnSchema::new("data".to_string(), PgType::TEXT, -1, true, false), + ], + } + } + + /// Create a test row matching the test schema. + fn create_test_row() -> PgTableRow { + let timestamp = NaiveDateTime::new( + NaiveDate::from_ymd_opt(2024, 6, 15).unwrap(), + NaiveTime::from_hms_opt(12, 30, 45).unwrap(), + ); + + PgTableRow::new(vec![ + PgCell::I64(12345), + PgCell::String("John Doe".to_string()), + PgCell::I32(30), + PgCell::Bool(true), + PgCell::Timestamp(timestamp), + ]) + } + + /// Create a test row for composite primary key schema. + fn create_composite_pk_row() -> PgTableRow { + PgTableRow::new(vec![ + PgCell::I32(1), // tenant_id + PgCell::I64(42), // user_id + PgCell::String("test data".to_string()), // data + ]) + } + + #[test] + fn test_build_pk_expr_single_primary_key() { + let schema = create_test_schema(); + let row = create_test_row(); + + let pk_expr = build_pk_expr(&schema, &row).unwrap(); + + assert_debug_snapshot!(pk_expr, @r#" + BinaryExpr( + BinaryExpr { + left: Column( + Column { + relation: None, + name: "id", + }, + ), + op: Eq, + right: Literal( + Int64(12345), + None, + ), + }, + ) + "#); + } + + #[test] + fn test_build_pk_expr_composite_primary_key() { + let schema = create_composite_pk_schema(); + let row = create_composite_pk_row(); + + let pk_expr = build_pk_expr(&schema, &row).unwrap(); + + assert_debug_snapshot!(pk_expr, @r#" + BinaryExpr( + BinaryExpr { + left: BinaryExpr( + BinaryExpr { + left: Column( + Column { + relation: None, + name: "tenant_id", + }, + ), + op: Eq, + right: Literal( + Int32(1), + None, + ), + }, + ), + op: And, + right: BinaryExpr( + BinaryExpr { + left: Column( + Column { + relation: None, + name: "user_id", + }, + ), + op: Eq, + right: Literal( + Int64(42), + None, + ), + }, + ), + }, + ) + "#); + } + + #[test] + fn test_build_pk_expr_no_primary_keys() { + // Create schema with no primary key columns + let schema = PgTableSchema { + id: etl::types::TableId(4), + name: TableName::new("public".to_string(), "no_pk_table".to_string()), + column_schemas: vec![ + PgColumnSchema::new("col1".to_string(), PgType::TEXT, -1, true, false), + PgColumnSchema::new("col2".to_string(), PgType::INT4, -1, true, false), + ], + }; + let row = PgTableRow::new(vec![PgCell::String("test".to_string()), PgCell::I32(42)]); + + let result = build_pk_expr(&schema, &row); + assert!(result.is_err()); + } + + #[test] + fn test_build_pk_expr_with_nulls_in_primary_key() { + let schema = create_test_schema(); + let row_with_null_pk = PgTableRow::new(vec![ + PgCell::Null, // NULL in primary key column + PgCell::String("John Doe".to_string()), + PgCell::I32(30), + PgCell::Bool(true), + PgCell::Timestamp(NaiveDateTime::new( + NaiveDate::from_ymd_opt(2024, 6, 15).unwrap(), + NaiveTime::from_hms_opt(12, 30, 45).unwrap(), + )), + ]); + + let pk_expr = build_pk_expr(&schema, &row_with_null_pk); + assert!(pk_expr.is_err()); + } + + #[test] + fn test_build_pk_expr_expression_structure() { + let schema = create_composite_pk_schema(); + let row = create_composite_pk_row(); + + let pk_expr = build_pk_expr(&schema, &row).unwrap(); + + assert_debug_snapshot!(pk_expr, @r#" + BinaryExpr( + BinaryExpr { + left: BinaryExpr( + BinaryExpr { + left: Column( + Column { + relation: None, + name: "tenant_id", + }, + ), + op: Eq, + right: Literal( + Int32(1), + None, + ), + }, + ), + op: And, + right: BinaryExpr( + BinaryExpr { + left: Column( + Column { + relation: None, + name: "user_id", + }, + ), + op: Eq, + right: Literal( + Int64(42), + None, + ), + }, + ), + }, + ) + "#); + } + + #[test] + fn test_qualify_primary_keys_single_column() { + use deltalake::datafusion::prelude::col; + + let primary_keys = vec![col("id")]; + let result = qualify_primary_keys(primary_keys, "source", "target").unwrap(); + + assert_debug_snapshot!(result, @r#" + BinaryExpr( + BinaryExpr { + left: Column( + Column { + relation: Some( + Bare { + table: "source", + }, + ), + name: "id", + }, + ), + op: Eq, + right: Column( + Column { + relation: Some( + Bare { + table: "target", + }, + ), + name: "id", + }, + ), + }, + ) + "#); + } + + #[test] + fn test_qualify_primary_keys_composite_columns() { + let primary_keys = vec![col("tenant_id"), col("user_id")]; + let result = qualify_primary_keys(primary_keys, "src", "tgt").unwrap(); + + assert_debug_snapshot!(result, @r#" + BinaryExpr( + BinaryExpr { + left: BinaryExpr( + BinaryExpr { + left: Column( + Column { + relation: Some( + Bare { + table: "src", + }, + ), + name: "tenant_id", + }, + ), + op: Eq, + right: Column( + Column { + relation: Some( + Bare { + table: "tgt", + }, + ), + name: "tenant_id", + }, + ), + }, + ), + op: And, + right: BinaryExpr( + BinaryExpr { + left: Column( + Column { + relation: Some( + Bare { + table: "src", + }, + ), + name: "user_id", + }, + ), + op: Eq, + right: Column( + Column { + relation: Some( + Bare { + table: "tgt", + }, + ), + name: "user_id", + }, + ), + }, + ), + }, + ) + "#); + } + + #[test] + fn test_qualify_primary_keys_multiple_columns() { + let primary_keys = vec![col("a"), col("b"), col("c")]; + let result = qualify_primary_keys(primary_keys, "s", "t").unwrap(); + + assert_debug_snapshot!(result, @r#" + BinaryExpr( + BinaryExpr { + left: BinaryExpr( + BinaryExpr { + left: BinaryExpr( + BinaryExpr { + left: Column( + Column { + relation: Some( + Bare { + table: "s", + }, + ), + name: "a", + }, + ), + op: Eq, + right: Column( + Column { + relation: Some( + Bare { + table: "t", + }, + ), + name: "a", + }, + ), + }, + ), + op: And, + right: BinaryExpr( + BinaryExpr { + left: Column( + Column { + relation: Some( + Bare { + table: "s", + }, + ), + name: "b", + }, + ), + op: Eq, + right: Column( + Column { + relation: Some( + Bare { + table: "t", + }, + ), + name: "b", + }, + ), + }, + ), + }, + ), + op: And, + right: BinaryExpr( + BinaryExpr { + left: Column( + Column { + relation: Some( + Bare { + table: "s", + }, + ), + name: "c", + }, + ), + op: Eq, + right: Column( + Column { + relation: Some( + Bare { + table: "t", + }, + ), + name: "c", + }, + ), + }, + ), + }, + ) + "#); + } + + #[test] + fn test_qualify_primary_keys_empty_list() { + let primary_keys: Vec = vec![]; + + let res = qualify_primary_keys(primary_keys, "source", "target"); + assert!(res.is_none()); + } + + #[test] + fn test_qualify_primary_keys_invalid_expression() { + // Pass a literal instead of a column expression + let primary_keys = vec![lit(42)]; + let res = qualify_primary_keys(primary_keys, "source", "target"); + assert!(res.is_none()); + } + + #[test] + fn test_qualify_primary_keys_case_sensitivity() { + let primary_keys = vec![ + Expr::Column(Column::new_unqualified("id")), + Expr::Column(Column::new_unqualified("NAME")), + ]; + let result = qualify_primary_keys(primary_keys, "source", "target").unwrap(); + + assert_debug_snapshot!(result, @r#" + BinaryExpr( + BinaryExpr { + left: BinaryExpr( + BinaryExpr { + left: Column( + Column { + relation: Some( + Bare { + table: "source", + }, + ), + name: "id", + }, + ), + op: Eq, + right: Column( + Column { + relation: Some( + Bare { + table: "target", + }, + ), + name: "id", + }, + ), + }, + ), + op: And, + right: BinaryExpr( + BinaryExpr { + left: Column( + Column { + relation: Some( + Bare { + table: "source", + }, + ), + name: "NAME", + }, + ), + op: Eq, + right: Column( + Column { + relation: Some( + Bare { + table: "target", + }, + ), + name: "NAME", + }, + ), + }, + ), + }, + ) + "#); + } +} diff --git a/etl-destinations/src/deltalake/maintenance.rs b/etl-destinations/src/deltalake/maintenance.rs new file mode 100644 index 000000000..e332a23d5 --- /dev/null +++ b/etl-destinations/src/deltalake/maintenance.rs @@ -0,0 +1,183 @@ +use std::convert::TryFrom; +use std::sync::Arc; + +use deltalake::DeltaTable; +use tokio::sync::Mutex; +use tokio::task::JoinHandle; +use tracing::{debug, error, instrument, trace}; + +use etl::types::TableId; + +use crate::deltalake::config::DeltaTableConfig; +use crate::deltalake::operations::{compact_table, zorder_table}; + +#[derive(Debug)] +pub struct TableMaintenanceInner { + pub last_compacted_version: i64, + pub last_zordered_version: i64, + pub compaction_task: Option>, + pub zorder_task: Option>, +} + +#[derive(Debug)] +pub struct TableMaintenanceState { + pub(crate) inner: Mutex, +} + +impl TableMaintenanceState { + pub fn new(initial_version: i64) -> Self { + Self { + inner: Mutex::new(TableMaintenanceInner { + last_compacted_version: initial_version, + last_zordered_version: initial_version, + compaction_task: None, + zorder_task: None, + }), + } + } + + /// Await any in-flight compaction, then if the `compact_after_commits` threshold is met, + /// run compaction. This guarantees serialization of compaction runs relative to table writes. + #[instrument(skip(self, table, config), fields(table_id = table_id.0, table_version))] + pub async fn maybe_run_compaction( + self: &Arc, + table_id: TableId, + table: Arc>, + config: Arc, + table_version: i64, + ) { + if let Some(handle) = { + let mut state = self.inner.lock().await; + state.compaction_task.take() + } { + if let Err(err) = handle.await { + error!(table_id = table_id.0, error = %err, "Compaction task join failed"); + } + } + + let should_compact = { + let state = self.inner.lock().await; + match config.compact_after_commits { + Some(compact_after) => { + if let Ok(threshold) = i64::try_from(compact_after.get()) { + table_version.saturating_sub(state.last_compacted_version) >= threshold + } else { + false + } + } + None => false, + } + }; + + if !should_compact { + debug!( + table_id = table_id.0, + version = table_version, + "Skipping Delta table compaction task" + ); + return; + } + + let task_state = Arc::clone(self); + let task_table = Arc::clone(&table); + let task_config = Arc::clone(&config); + + let handle = tokio::spawn(async move { + trace!( + table_id = table_id.0, + "Starting Delta table compaction task" + ); + let mut table_guard = task_table.lock().await; + if let Err(err) = compact_table(&mut table_guard, task_config.as_ref()).await { + error!(table_id = table_id.0, error = %err, "Delta table compaction task failed"); + return; + } + let version = table_guard.version().unwrap_or(table_version); + trace!( + table_id = table_id.0, + version, "Finished Delta table compaction task" + ); + drop(table_guard); + + let mut state = task_state.inner.lock().await; + state.last_compacted_version = version; + }); + + let mut state = self.inner.lock().await; + state.compaction_task = Some(handle); + } + + /// Await any in-flight Z-ordering, then if the `z_order_after_commits` threshold is met, + /// run Z-order. Serializes Z-order runs relative to table writes. + #[instrument(skip(self, table, config), fields(table_id = table_id.0, table_version))] + pub async fn maybe_run_zorder( + self: &Arc, + table_id: TableId, + table: Arc>, + config: Arc, + table_version: i64, + ) { + // Join any finished task to propagate panics and free resources. + if let Some(handle) = { + let mut state = self.inner.lock().await; + state.zorder_task.take() + } { + if let Err(err) = handle.await { + error!(table_id = table_id.0, error = %err, "Z-order task join failed"); + } + } + + let (should_zorder, columns) = { + let state = self.inner.lock().await; + match ( + config.z_order_columns.as_ref(), + config.z_order_after_commits, + ) { + (Some(columns), Some(zorder_after)) if !columns.is_empty() => { + if let Ok(threshold) = i64::try_from(zorder_after.get()) { + let should = + table_version.saturating_sub(state.last_zordered_version) >= threshold; + (should, columns.clone()) + } else { + (false, Vec::new()) + } + } + _ => (false, Vec::new()), + } + }; + + if !should_zorder { + debug!( + table_id = table_id.0, + version = table_version, + "Skipping Delta table Z-order task" + ); + return; + } + + let task_state = Arc::clone(self); + let task_table = Arc::clone(&table); + let task_config = Arc::clone(&config); + + let handle = tokio::spawn(async move { + trace!(table_id = table_id.0, columns = ?columns, "Starting Delta table Z-order task"); + let mut table_guard = task_table.lock().await; + if let Err(err) = zorder_table(&mut table_guard, task_config.as_ref(), columns).await { + error!(table_id = table_id.0, error = %err, "Delta table Z-order task failed"); + return; + } + let version = table_guard.version().unwrap_or(table_version); + trace!( + table_id = table_id.0, + version, "Finished Delta table Z-order task" + ); + drop(table_guard); + + let mut state = task_state.inner.lock().await; + state.last_zordered_version = version; + }); + + let mut state = self.inner.lock().await; + state.zorder_task = Some(handle); + } +} diff --git a/etl-destinations/src/deltalake/mod.rs b/etl-destinations/src/deltalake/mod.rs new file mode 100644 index 000000000..d7fc0b5c1 --- /dev/null +++ b/etl-destinations/src/deltalake/mod.rs @@ -0,0 +1,11 @@ +mod config; +mod core; +pub(crate) mod events; +pub(crate) mod expr; +mod maintenance; +mod operations; +mod schema; +pub(crate) mod util; + +pub use config::DeltaTableConfig; +pub use core::{DeltaDestinationConfig, DeltaLakeDestination}; diff --git a/etl-destinations/src/deltalake/operations/append.rs b/etl-destinations/src/deltalake/operations/append.rs new file mode 100644 index 000000000..7c83f7cdd --- /dev/null +++ b/etl-destinations/src/deltalake/operations/append.rs @@ -0,0 +1,25 @@ +use deltalake::{ + DeltaResult, DeltaTable, + arrow::array::RecordBatch, + writer::{DeltaWriter, RecordBatchWriter}, +}; +use tracing::{instrument, trace}; + +use crate::deltalake::config::DeltaTableConfig; + +/// Appends a record batch to a Delta table +#[instrument(skip(table, config, record_batch), fields(num_rows = record_batch.num_rows()))] +pub async fn append_to_table( + table: &mut DeltaTable, + config: &DeltaTableConfig, + record_batch: RecordBatch, +) -> DeltaResult<()> { + trace!("Creating RecordBatchWriter for append"); + let mut writer = RecordBatchWriter::for_table(table)?; + writer = writer.with_writer_properties(config.into()); + trace!("Writing record batch to Delta table"); + writer.write(record_batch).await?; + trace!("Flushing and committing append"); + writer.flush_and_commit(table).await?; + Ok(()) +} diff --git a/etl-destinations/src/deltalake/operations/delete.rs b/etl-destinations/src/deltalake/operations/delete.rs new file mode 100644 index 000000000..e32debe49 --- /dev/null +++ b/etl-destinations/src/deltalake/operations/delete.rs @@ -0,0 +1,24 @@ +use deltalake::{ + DeltaResult, DeltaTable, datafusion::prelude::Expr, operations::delete::DeleteBuilder, +}; + +use crate::deltalake::config::DeltaTableConfig; +use tracing::{instrument, trace}; + +#[instrument(skip(table, config, delete_predicate))] +pub async fn delete_from_table( + table: &mut DeltaTable, + config: &DeltaTableConfig, + delete_predicate: Expr, +) -> DeltaResult<()> { + trace!("Building delete builder with predicate"); + let delete_builder = DeleteBuilder::new((*table).log_store(), table.snapshot()?.clone()) + .with_predicate(delete_predicate) + .with_writer_properties(config.into()); + // TODO(abhi): Do something with the metrics + trace!("Executing delete operation"); + let (deleted_table, _metrics) = delete_builder.await?; + *table = deleted_table; + trace!("Delete operation completed"); + Ok(()) +} diff --git a/etl-destinations/src/deltalake/operations/merge.rs b/etl-destinations/src/deltalake/operations/merge.rs new file mode 100644 index 000000000..7fe1f61ce --- /dev/null +++ b/etl-destinations/src/deltalake/operations/merge.rs @@ -0,0 +1,136 @@ +use deltalake::DeltaTableError; +use deltalake::datafusion::common::Column; +use deltalake::datafusion::prelude::SessionContext; +use deltalake::operations::merge::MergeBuilder; +use deltalake::{DeltaResult, DeltaTable, datafusion::prelude::Expr}; +use etl::types::{TableRow as PgTableRow, TableSchema as PgTableSchema}; +use tracing::{instrument, trace}; + +use crate::arrow::rows_to_record_batch; +use crate::deltalake::config::DeltaTableConfig; +use crate::deltalake::expr::qualify_primary_keys; +use crate::deltalake::schema::postgres_to_arrow_schema; + +pub(crate) fn source_qualified_column_expr(column_name: &str, source_alias: &str) -> Expr { + Expr::Column(Column::new(Some(source_alias), column_name)) +} + +#[instrument( + skip(table, config, table_schema, upsert_rows, delete_predicate), + fields(upsert_count = upsert_rows.len(), has_delete = delete_predicate.is_some()) +)] +pub async fn merge_to_table( + table: &mut DeltaTable, + config: &DeltaTableConfig, + table_schema: &PgTableSchema, + upsert_rows: &[PgTableRow], + delete_predicate: Option, +) -> DeltaResult<()> { + trace!("Building Arrow schema and source batch for merge"); + let arrow_schema = postgres_to_arrow_schema(table_schema)?; + let rows = rows_to_record_batch(upsert_rows, arrow_schema)?; + + let ctx = SessionContext::new(); + let batch = ctx.read_batch(rows)?; + + // TODO(abhi): We should proabbly be passing this information in + let primary_keys = table_schema + .column_schemas + .iter() + .filter(|col| col.primary) + .map(|col| Expr::Column(Column::new_unqualified(col.name.clone()))) + .collect(); + + let qualified_primary_keys = qualify_primary_keys(primary_keys, "source", "target") + .ok_or(DeltaTableError::generic("Failed to qualify primary keys"))?; + + trace!("Creating merge builder"); + let merge_builder = MergeBuilder::new( + // TODO(abhi): Is there a way to do this while avoiding the clone/general hackiness? + (*table).log_store(), + table.snapshot()?.clone(), + qualified_primary_keys, + batch, + ); + + // TODO(abhi): Clean up this mess + let all_columns: Vec<&str> = table_schema + .column_schemas + .iter() + .map(|col| col.name.as_str()) + .collect(); + + let mut merge_builder = merge_builder + .with_writer_properties(config.into()) + .with_source_alias("source") + .with_target_alias("target") + .when_not_matched_insert(|insert| { + all_columns.iter().fold(insert, |insert, &column| { + insert.set( + column.to_string(), + source_qualified_column_expr(column, "source"), + ) + }) + })? + .when_matched_update(|update| { + all_columns.iter().fold(update, |update, &column| { + update.update( + column.to_string(), + source_qualified_column_expr(column, "source"), + ) + }) + })?; + + if let Some(delete_predicate) = delete_predicate { + merge_builder = merge_builder + .when_not_matched_by_source_delete(|delete| delete.predicate(delete_predicate))?; + } + // TODO(abhi): Do something with the metrics + trace!("Executing merge operation"); + let (merged_table, _metrics) = merge_builder.await?; + trace!("Merge operation completed"); + *table = merged_table; + Ok(()) +} + +#[cfg(test)] +mod tests { + use super::*; + use insta::assert_debug_snapshot; + + #[test] + fn source_qualified_column_expr_preserves_case_and_alias() { + let expr = source_qualified_column_expr("CASESensitivecolumn", "source"); + + assert_debug_snapshot!(expr, @r#" + Column( + Column { + relation: Some( + Bare { + table: "source", + }, + ), + name: "CASESensitivecolumn", + }, + ) + "#); + } + + #[test] + fn source_qualified_column_expr_handles_lowercase() { + let expr = source_qualified_column_expr("lowercasecolumn", "source"); + + assert_debug_snapshot!(expr, @r#" + Column( + Column { + relation: Some( + Bare { + table: "source", + }, + ), + name: "lowercasecolumn", + }, + ) + "#); + } +} diff --git a/etl-destinations/src/deltalake/operations/mod.rs b/etl-destinations/src/deltalake/operations/mod.rs new file mode 100644 index 000000000..992fc20cf --- /dev/null +++ b/etl-destinations/src/deltalake/operations/mod.rs @@ -0,0 +1,9 @@ +mod append; +mod delete; +mod merge; +mod optimize; + +pub use append::append_to_table; +pub use delete::delete_from_table; +pub use merge::merge_to_table; +pub use optimize::{compact_table, zorder_table}; diff --git a/etl-destinations/src/deltalake/operations/optimize.rs b/etl-destinations/src/deltalake/operations/optimize.rs new file mode 100644 index 000000000..0dc89eb23 --- /dev/null +++ b/etl-destinations/src/deltalake/operations/optimize.rs @@ -0,0 +1,40 @@ +use deltalake::operations::optimize::{OptimizeBuilder, OptimizeType}; +use deltalake::parquet::file::properties::WriterProperties; +use deltalake::{DeltaResult, DeltaTable}; +use tracing::{instrument, trace}; + +use crate::deltalake::config::DeltaTableConfig; + +/// Optimizes a Delta table by compacting small files into larger ones. +#[instrument(skip(table, config))] +pub async fn compact_table(table: &mut DeltaTable, config: &DeltaTableConfig) -> DeltaResult<()> { + trace!("Starting table compaction"); + let writer_properties = WriterProperties::from(config); + let optimize_builder = OptimizeBuilder::new(table.log_store(), table.snapshot()?.clone()); + let (optimized_table, _metrics) = optimize_builder + .with_writer_properties(writer_properties) + .with_type(OptimizeType::Compact) + .await?; + *table = optimized_table; + trace!("Finished table compaction"); + Ok(()) +} + +/// Optimizes a Delta table by performing Z-order clustering on the provided columns. +#[instrument(skip(table, config, columns), fields(columns = ?columns))] +pub async fn zorder_table( + table: &mut DeltaTable, + config: &DeltaTableConfig, + columns: Vec, +) -> DeltaResult<()> { + trace!("Starting table Z-order optimization"); + let writer_properties = WriterProperties::from(config); + let optimize_builder = OptimizeBuilder::new(table.log_store(), table.snapshot()?.clone()); + let (optimized_table, _metrics) = optimize_builder + .with_writer_properties(writer_properties) + .with_type(OptimizeType::ZOrder(columns)) + .await?; + *table = optimized_table; + trace!("Finished table Z-order optimization"); + Ok(()) +} diff --git a/etl-destinations/src/deltalake/schema.rs b/etl-destinations/src/deltalake/schema.rs new file mode 100644 index 000000000..8ceae97e6 --- /dev/null +++ b/etl-destinations/src/deltalake/schema.rs @@ -0,0 +1,90 @@ +use arrow::datatypes::Schema; +use deltalake::kernel::engine::arrow_conversion::TryFromKernel; +use deltalake::kernel::{DataType as DeltaDataType, StructField as DeltaStructField}; +use deltalake::{ArrayType, DecimalType, DeltaResult, Schema as DeltaSchema}; + +use deltalake::PrimitiveType; +use etl::types::{ColumnSchema as PgColumnSchema, TableSchema as PgTableSchema, Type as PgType}; + +use crate::arrow::{extract_numeric_precision, extract_numeric_scale}; + +/// Converts a Postgres type to equivalent Delta type +pub fn postgres_type_to_delta_type(typ: &PgType, modifier: i32) -> DeltaDataType { + match typ { + &PgType::BOOL => DeltaDataType::Primitive(PrimitiveType::Boolean), + &PgType::CHAR | &PgType::BPCHAR | &PgType::VARCHAR | &PgType::NAME | &PgType::TEXT => { + DeltaDataType::Primitive(PrimitiveType::String) + } + &PgType::INT2 | &PgType::INT4 => DeltaDataType::Primitive(PrimitiveType::Integer), + &PgType::INT8 => DeltaDataType::Primitive(PrimitiveType::Long), + &PgType::FLOAT4 => DeltaDataType::Primitive(PrimitiveType::Float), + &PgType::FLOAT8 => DeltaDataType::Primitive(PrimitiveType::Double), + &PgType::NUMERIC => { + let precision = extract_numeric_precision(modifier); + let scale = extract_numeric_scale(modifier); + let decimal_type = DecimalType::try_new(precision, scale) + .map(PrimitiveType::Decimal) + .unwrap_or(PrimitiveType::String); + DeltaDataType::Primitive(decimal_type) + } + &PgType::DATE => DeltaDataType::Primitive(PrimitiveType::Date), + &PgType::TIME => DeltaDataType::Primitive(PrimitiveType::Timestamp), + // NOTE: Postgres type is TIMESTAMP(TZ), not TIMESTAMP(NTZ) + &PgType::TIMESTAMP => DeltaDataType::Primitive(PrimitiveType::TimestampNtz), + &PgType::TIMESTAMPTZ => DeltaDataType::Primitive(PrimitiveType::Timestamp), + &PgType::UUID => DeltaDataType::Primitive(PrimitiveType::String), + &PgType::JSON | &PgType::JSONB => DeltaDataType::Primitive(PrimitiveType::String), + &PgType::OID => DeltaDataType::Primitive(PrimitiveType::Long), + &PgType::BYTEA => DeltaDataType::Primitive(PrimitiveType::Binary), + &PgType::BOOL_ARRAY => create_delta_list_type(PrimitiveType::Boolean), + &PgType::CHAR_ARRAY + | &PgType::BPCHAR_ARRAY + | &PgType::VARCHAR_ARRAY + | &PgType::NAME_ARRAY + | &PgType::TEXT_ARRAY => create_delta_list_type(PrimitiveType::String), + &PgType::INT2_ARRAY | &PgType::INT4_ARRAY => create_delta_list_type(PrimitiveType::Integer), + &PgType::INT8_ARRAY => create_delta_list_type(PrimitiveType::Long), + &PgType::FLOAT4_ARRAY => create_delta_list_type(PrimitiveType::Float), + &PgType::FLOAT8_ARRAY => create_delta_list_type(PrimitiveType::Double), + &PgType::NUMERIC_ARRAY => create_delta_list_type(PrimitiveType::String), + &PgType::DATE_ARRAY => create_delta_list_type(PrimitiveType::Date), + &PgType::TIME_ARRAY => create_delta_list_type(PrimitiveType::Timestamp), + // NOTE: Postgres type is TIMESTAMP(TZ), not TIMESTAMP(NTZ) + &PgType::TIMESTAMP_ARRAY => create_delta_list_type(PrimitiveType::TimestampNtz), + &PgType::TIMESTAMPTZ_ARRAY => create_delta_list_type(PrimitiveType::Timestamp), + &PgType::UUID_ARRAY => create_delta_list_type(PrimitiveType::String), + &PgType::JSON_ARRAY | &PgType::JSONB_ARRAY => create_delta_list_type(PrimitiveType::String), + &PgType::OID_ARRAY => create_delta_list_type(PrimitiveType::Long), + &PgType::BYTEA_ARRAY => create_delta_list_type(PrimitiveType::Binary), + _ => DeltaDataType::Primitive(PrimitiveType::String), + } +} + +fn create_delta_list_type(element_type: PrimitiveType) -> DeltaDataType { + let array_type = Box::new(ArrayType::new(element_type.into(), true)); + + DeltaDataType::Array(array_type) +} + +pub fn postgres_column_schema_to_delta_field(schema: &PgColumnSchema) -> DeltaStructField { + let field_type = postgres_type_to_delta_type(&schema.typ, schema.modifier); + DeltaStructField::new(&schema.name, field_type, schema.nullable) +} + +/// Convert a Postgres `PgTableSchema` to a Delta `Schema` +pub fn postgres_to_delta_schema(schema: &PgTableSchema) -> DeltaResult { + let fields: Vec = schema + .column_schemas + .iter() + .map(postgres_column_schema_to_delta_field) + .collect(); + + let delta_schema = DeltaSchema::try_new(fields)?; + Ok(delta_schema) +} + +pub fn postgres_to_arrow_schema(schema: &PgTableSchema) -> DeltaResult { + let delta_schema = postgres_to_delta_schema(schema)?; + let arrow_schema = Schema::try_from_kernel(&delta_schema)?; + Ok(arrow_schema) +} diff --git a/etl-destinations/src/deltalake/util/lwwreg.rs b/etl-destinations/src/deltalake/util/lwwreg.rs new file mode 100644 index 000000000..5938c8f30 --- /dev/null +++ b/etl-destinations/src/deltalake/util/lwwreg.rs @@ -0,0 +1,120 @@ +#![allow(dead_code)] +/// Vendored from `crdts` crate. +/// License: Apache-2.0 https://github.com/rust-crdt/rust-crdt/blob/master/LICENSE +use std::{error, fmt}; + +/// `LWWReg` is a simple CRDT that contains an arbitrary value +/// along with an `Ord` that tracks causality. It is the responsibility +/// of the user to guarantee that the source of the causal element +/// is monotonic. Don't use timestamps unless you are comfortable +/// with divergence. +/// +/// `M` is a marker. It must grow monotonically *and* must be globally unique +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +pub struct LWWReg { + /// `val` is the opaque element contained within this CRDT + pub val: V, + /// `marker` should be a monotonic value associated with this val + pub marker: M, +} + +impl Default for LWWReg { + fn default() -> Self { + Self { + val: V::default(), + marker: M::default(), + } + } +} + +/// The Type of validation errors that may occur for an LWWReg. +#[derive(Debug, PartialEq)] +pub enum Validation { + /// A conflicting change to a CRDT is witnessed by a dot that already exists. + ConflictingMarker, +} + +impl error::Error for Validation { + fn description(&self) -> &str { + match self { + Validation::ConflictingMarker => { + "A marker must be used exactly once, re-using the same marker breaks associativity" + } + } + } +} + +impl fmt::Display for Validation { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + write!(f, "{self:?}") + } +} + +impl LWWReg { + /// Construct a new LwwReg initialized with the given value and marker + pub fn new(val: V, marker: M) -> Self { + LWWReg { val, marker } + } + + /// Updates value witnessed by the given marker. + pub fn update(&mut self, val: V, marker: M) { + if self.marker < marker { + self.val = val; + self.marker = marker; + } + } + + /// An update is invalid if the marker is exactly the same as + pub fn validate_update(&self, val: &V, marker: &M) -> Result<(), Validation> { + if &self.marker == marker && val != &self.val { + Err(Validation::ConflictingMarker) + } else { + Ok(()) + } + } +} + +#[cfg(test)] +mod test { + use super::*; + + #[test] + fn test_default() { + let reg = LWWReg::default(); + assert_eq!(reg, LWWReg { val: "", marker: 0 }); + } + + #[test] + fn test_update() { + let mut reg = LWWReg { + val: 123, + marker: 0, + }; + + // normal update: new marker is a descended of current marker + // EXPECTED: success, the val and marker are update + reg.update(32, 2); + assert_eq!(reg, LWWReg { val: 32, marker: 2 }); + + // stale update: new marker is an ancester of the current marker + // EXPECTED: succes, no-op + reg.update(57, 1); + assert_eq!(reg, LWWReg { val: 32, marker: 2 }); + + // redundant update: new marker and val is same as of the current state + // EXPECTED: success, no-op + reg.update(32, 2); + assert_eq!(reg, LWWReg { val: 32, marker: 2 }); + + // bad update: new marker same as of the current marker but not value + // EXPECTED: error + assert_eq!( + reg.validate_update(&4000, &2), + Err(Validation::ConflictingMarker) + ); + + // Applying the update despite the validation error is a no-op + reg.update(4000, 2); + assert_eq!(reg, LWWReg { val: 32, marker: 2 }); + } +} diff --git a/etl-destinations/src/deltalake/util/mod.rs b/etl-destinations/src/deltalake/util/mod.rs new file mode 100644 index 000000000..4d5b3eab4 --- /dev/null +++ b/etl-destinations/src/deltalake/util/mod.rs @@ -0,0 +1,3 @@ +mod lwwreg; + +pub(crate) use lwwreg::LWWReg; diff --git a/etl-destinations/src/iceberg/client.rs b/etl-destinations/src/iceberg/client.rs index 07e1b474a..3703a0be9 100644 --- a/etl-destinations/src/iceberg/client.rs +++ b/etl-destinations/src/iceberg/client.rs @@ -1,6 +1,6 @@ use std::{collections::HashMap, sync::Arc}; -use arrow::array::RecordBatch; +use arrow_55::array::RecordBatch; use etl::{ error::EtlResult, types::{TableRow, TableSchema}, diff --git a/etl-destinations/src/iceberg/error.rs b/etl-destinations/src/iceberg/error.rs index 5d8686fe4..49e117cc6 100644 --- a/etl-destinations/src/iceberg/error.rs +++ b/etl-destinations/src/iceberg/error.rs @@ -1,4 +1,4 @@ -use arrow::error::ArrowError; +use crate::arrow::arrow_compat::arrow::error::ArrowError; use etl::{ error::{ErrorKind, EtlError}, etl_error, @@ -71,6 +71,8 @@ pub(crate) fn arrow_error_to_etl_error(err: ArrowError) -> EtlError { ArrowError::RunEndIndexOverflowError => { (ErrorKind::InvalidData, "Arrow run end index overflow") } + #[cfg(feature = "arrow-56")] + ArrowError::OffsetOverflowError(_) => (ErrorKind::InvalidData, "Arrow offset overflow"), }; etl_error!(kind, description, err.to_string()) diff --git a/etl-destinations/src/iceberg/mod.rs b/etl-destinations/src/iceberg/mod.rs index 062ede74e..1da14c405 100644 --- a/etl-destinations/src/iceberg/mod.rs +++ b/etl-destinations/src/iceberg/mod.rs @@ -1,7 +1,10 @@ mod client; -mod encoding; mod error; mod schema; +mod encoding { + pub use crate::arrow::encoding::*; +} + pub use client::IcebergClient; pub use encoding::UNIX_EPOCH; diff --git a/etl-destinations/src/lib.rs b/etl-destinations/src/lib.rs index 0365fec35..cdb218921 100644 --- a/etl-destinations/src/lib.rs +++ b/etl-destinations/src/lib.rs @@ -3,7 +3,9 @@ //! Provides implementations of the ETL destination trait for various data warehouses //! and analytics platforms, enabling data replication from Postgres to cloud services. +#[cfg(feature = "deltalake")] +pub mod arrow; #[cfg(feature = "bigquery")] pub mod bigquery; -#[cfg(feature = "iceberg")] -pub mod iceberg; +#[cfg(feature = "deltalake")] +pub mod deltalake; diff --git a/etl-destinations/tests/deltalake_pipeline.rs b/etl-destinations/tests/deltalake_pipeline.rs new file mode 100644 index 000000000..f52f657a4 --- /dev/null +++ b/etl-destinations/tests/deltalake_pipeline.rs @@ -0,0 +1,1300 @@ +#![cfg(feature = "deltalake")] + +use deltalake::datafusion::prelude::SessionContext; +use etl::config::BatchConfig; +use etl::state::table::TableReplicationPhaseType; +use etl::test_utils::database::{spawn_source_database, test_table_name}; +use etl::test_utils::notify::NotifyingStore; +use etl::test_utils::pipeline::{create_pipeline, create_pipeline_with}; +use etl::test_utils::test_destination_wrapper::TestDestinationWrapper; +use etl::test_utils::test_schema::{TableSelection, insert_mock_data, setup_test_database_schema}; +use etl::types::{EventType, PipelineId, ToSql}; +use etl_telemetry::tracing::init_test_tracing; +use rand::random; + +use chrono::{DateTime, NaiveDate, NaiveDateTime, NaiveTime, Utc}; +use etl::types::PgNumeric; +use serde_json::json; +use std::collections::HashMap; +use std::num::NonZeroU64; +use std::str::FromStr; +use std::sync::Arc; +use uuid::Uuid; + +use deltalake::arrow::util::pretty::pretty_format_batches; +use deltalake::{DeltaResult, DeltaTable, DeltaTableError}; +use insta::assert_snapshot; + +use crate::support::deltalake::setup_delta_connection; + +mod support; + +pub async fn snapshot_table_string(table_name: &str, table: DeltaTable) -> DeltaResult { + let snapshot = table.snapshot()?; + let schema = snapshot.schema(); + + let mut out = String::new(); + out.push_str("# Schema\n"); + for field in schema.fields() { + out.push_str(&format!( + "- {}: {:?} nullable={}\n", + field.name(), + field.data_type(), + field.is_nullable() + )); + } + + out.push_str("\n# Data\n"); + let ctx = SessionContext::new(); + ctx.register_table(table_name, Arc::new(table))?; + let batches = ctx + .sql(&format!("SELECT * FROM {table_name} ORDER BY id")) + .await? + .collect() + .await?; + if batches.is_empty() { + out.push_str("\n"); + } else { + let formatted = pretty_format_batches(&batches).map_err(DeltaTableError::generic)?; + out.push_str(&formatted.to_string()); + out.push('\n'); + } + + Ok(out) +} + +macro_rules! assert_table_snapshot { + ($name:expr, $table:expr) => { + let snapshot_str = snapshot_table_string($name, $table) + .await + .expect("Should snapshot table"); + assert_snapshot!($name, snapshot_str, stringify!($table)); + }; +} + +#[tokio::test(flavor = "multi_thread")] +async fn append_only_ignores_updates_and_deletes() { + init_test_tracing(); + + let database = spawn_source_database().await; + let database_schema = setup_test_database_schema(&database, TableSelection::UsersOnly).await; + + let delta_database = setup_delta_connection().await; + + let store = NotifyingStore::new(); + + // Configure append_only for the users table only + let mut table_config = std::collections::HashMap::new(); + table_config.insert( + database_schema.users_schema().name.name.clone(), + Arc::new(etl_destinations::deltalake::DeltaTableConfig { + append_only: true, + ..Default::default() + }), + ); + + let raw_destination = delta_database + .build_destination_with_config(store.clone(), table_config) + .await; + let destination = TestDestinationWrapper::wrap(raw_destination); + + let pipeline_id: PipelineId = rand::random(); + let mut pipeline = create_pipeline( + &database.config, + pipeline_id, + database_schema.publication_name(), + store.clone(), + destination.clone(), + ); + + let users_state_notify = store + .notify_on_table_state_type( + database_schema.users_schema().id, + TableReplicationPhaseType::SyncDone, + ) + .await; + + pipeline.start().await.unwrap(); + users_state_notify.notified().await; + + let event_notify = destination + .wait_for_events_count(vec![ + (EventType::Insert, 1), + (EventType::Update, 2), + (EventType::Delete, 1), + ]) + .await; + + database + .insert_values( + database_schema.users_schema().name.clone(), + &["name", "age"], + &[&"append_user", &10], + ) + .await + .unwrap(); + + // Perform updates that should be ignored + database + .update_values( + database_schema.users_schema().name.clone(), + &["name", "age"], + &[&"append_user_v2", &20], + ) + .await + .unwrap(); + + database + .update_values( + database_schema.users_schema().name.clone(), + &["name", "age"], + &[&"append_user_final", &30], + ) + .await + .unwrap(); + + // And a delete that should be ignored + database + .delete_values( + database_schema.users_schema().name.clone(), + &["name"], + &["'append_user_final'"], + "", + ) + .await + .unwrap(); + + event_notify.notified().await; + pipeline.shutdown_and_wait().await.unwrap(); + + let users_table = delta_database + .load_table(&database_schema.users_schema().name) + .await + .unwrap(); + assert_table_snapshot!("append_only_ignores_updates_and_deletes", users_table); +} + +#[tokio::test(flavor = "multi_thread")] +async fn upsert_merge_validation() { + init_test_tracing(); + + let database = spawn_source_database().await; + let database_schema = setup_test_database_schema(&database, TableSelection::UsersOnly).await; + + let delta_database = setup_delta_connection().await; + + let store = NotifyingStore::new(); + let raw_destination = delta_database.build_destination(store.clone()).await; + let destination = TestDestinationWrapper::wrap(raw_destination); + + let pipeline_id: PipelineId = random(); + let mut pipeline = create_pipeline( + &database.config, + pipeline_id, + database_schema.publication_name(), + store.clone(), + destination.clone(), + ); + + let users_state_notify = store + .notify_on_table_state_type( + database_schema.users_schema().id, + TableReplicationPhaseType::SyncDone, + ) + .await; + + pipeline.start().await.unwrap(); + users_state_notify.notified().await; + + // Expect 1 insert and 2 updates to be coalesced via merge into latest state + let event_notify = destination + .wait_for_events_count(vec![(EventType::Insert, 1), (EventType::Update, 2)]) + .await; + + // Insert one user + database + .insert_values( + database_schema.users_schema().name.clone(), + &["name", "age"], + &[&"snap_user", &10], + ) + .await + .unwrap(); + + // Two subsequent updates to simulate upsert/merge collapsing + database + .update_values( + database_schema.users_schema().name.clone(), + &["name", "age"], + &[&"snap_user_v2", &20], + ) + .await + .unwrap(); + + database + .update_values( + database_schema.users_schema().name.clone(), + &["name", "age"], + &[&"snap_user_final", &30], + ) + .await + .unwrap(); + + event_notify.notified().await; + pipeline.shutdown_and_wait().await.unwrap(); + + let users_table = delta_database + .load_table(&database_schema.users_schema().name) + .await + .unwrap(); + assert_table_snapshot!("upsert_merge_validation", users_table); +} + +#[tokio::test(flavor = "multi_thread")] +async fn merge_with_delete_validation() { + init_test_tracing(); + + let database = spawn_source_database().await; + let database_schema = setup_test_database_schema(&database, TableSelection::UsersOnly).await; + + let delta_database = setup_delta_connection().await; + + let store = NotifyingStore::new(); + let raw_destination = delta_database.build_destination(store.clone()).await; + let destination = TestDestinationWrapper::wrap(raw_destination); + + let pipeline_id: PipelineId = random(); + let mut pipeline = create_pipeline( + &database.config, + pipeline_id, + database_schema.publication_name(), + store.clone(), + destination.clone(), + ); + + let users_state_notify = store + .notify_on_table_state_type( + database_schema.users_schema().id, + TableReplicationPhaseType::SyncDone, + ) + .await; + + pipeline.start().await.unwrap(); + users_state_notify.notified().await; + + // Expect 2 inserts and 1 delete + let event_notify = destination + .wait_for_events_count(vec![(EventType::Insert, 2), (EventType::Delete, 1)]) + .await; + + // Two rows + database + .insert_values( + database_schema.users_schema().name.clone(), + &["name", "age"], + &[&"d_user_a", &11], + ) + .await + .unwrap(); + database + .insert_values( + database_schema.users_schema().name.clone(), + &["name", "age"], + &[&"d_user_b", &12], + ) + .await + .unwrap(); + + // Delete one of them (by name) + database + .delete_values( + database_schema.users_schema().name.clone(), + &["name"], + &["'d_user_a'"], + "", + ) + .await + .unwrap(); + + event_notify.notified().await; + pipeline.shutdown_and_wait().await.unwrap(); + + let users_table = delta_database + .load_table(&database_schema.users_schema().name) + .await + .unwrap(); + assert_table_snapshot!("merge_with_delete_validation", users_table); +} + +#[tokio::test(flavor = "multi_thread")] +async fn table_copy_and_streaming_with_restart() { + init_test_tracing(); + + let mut database = spawn_source_database().await; + let database_schema = setup_test_database_schema(&database, TableSelection::Both).await; + + let delta_database = setup_delta_connection().await; + + // Insert initial test data. + insert_mock_data( + &mut database, + &database_schema.users_schema().name, + &database_schema.orders_schema().name, + 1..=2, + false, + ) + .await; + + let store = NotifyingStore::new(); + let raw_destination = delta_database.build_destination(store.clone()).await; + let destination = TestDestinationWrapper::wrap(raw_destination); + + // Start pipeline from scratch. + let pipeline_id: PipelineId = random(); + let mut pipeline = create_pipeline( + &database.config, + pipeline_id, + database_schema.publication_name(), + store.clone(), + destination.clone(), + ); + + // Register notifications for table copy completion. + let users_state_notify = store + .notify_on_table_state_type( + database_schema.users_schema().id, + TableReplicationPhaseType::SyncDone, + ) + .await; + let orders_state_notify = store + .notify_on_table_state_type( + database_schema.orders_schema().id, + TableReplicationPhaseType::SyncDone, + ) + .await; + + pipeline.start().await.unwrap(); + + users_state_notify.notified().await; + orders_state_notify.notified().await; + + pipeline.shutdown_and_wait().await.unwrap(); + + let mut users_table = delta_database + .load_table(&database_schema.users_schema().name) + .await + .unwrap(); + let mut orders_table = delta_database + .load_table(&database_schema.orders_schema().name) + .await + .unwrap(); + + assert_table_snapshot!( + "table_copy_and_streaming_with_restart_users_table_1", + users_table.clone() + ); + assert_table_snapshot!( + "table_copy_and_streaming_with_restart_orders_table_1", + orders_table.clone() + ); + + // We restart the pipeline and check that we can process events since we have loaded the table + // schema from the destination. + let mut pipeline = create_pipeline( + &database.config, + pipeline_id, + database_schema.publication_name(), + store.clone(), + destination.clone(), + ); + + pipeline.start().await.unwrap(); + + // We expect 2 insert events for each table (4 total). + let event_notify = destination + .wait_for_events_count(vec![(EventType::Insert, 4)]) + .await; + + // Insert additional data. + insert_mock_data( + &mut database, + &database_schema.users_schema().name, + &database_schema.orders_schema().name, + 3..=4, + false, + ) + .await; + + event_notify.notified().await; + + pipeline.shutdown_and_wait().await.unwrap(); + + users_table.load().await.unwrap(); + orders_table.load().await.unwrap(); + + assert_table_snapshot!( + "table_copy_and_streaming_with_restart_users_table_2", + users_table + ); + assert_table_snapshot!( + "table_copy_and_streaming_with_restart_orders_table_2", + orders_table + ); +} + +#[tokio::test(flavor = "multi_thread")] +async fn table_insert_update_delete() { + init_test_tracing(); + + let database = spawn_source_database().await; + let database_schema = setup_test_database_schema(&database, TableSelection::UsersOnly).await; + + let delta_database = setup_delta_connection().await; + + let store = NotifyingStore::new(); + let raw_destination = delta_database.build_destination(store.clone()).await; + let destination = TestDestinationWrapper::wrap(raw_destination); + + // Start pipeline from scratch. + let pipeline_id: PipelineId = random(); + let mut pipeline = create_pipeline( + &database.config, + pipeline_id, + database_schema.publication_name(), + store.clone(), + destination.clone(), + ); + + // Register notifications for table copy completion. + let users_state_notify = store + .notify_on_table_state_type( + database_schema.users_schema().id, + TableReplicationPhaseType::SyncDone, + ) + .await; + + pipeline.start().await.unwrap(); + + users_state_notify.notified().await; + + // Wait for the first insert. + let event_notify = destination + .wait_for_events_count(vec![(EventType::Insert, 1)]) + .await; + + // Insert a row. + database + .insert_values( + database_schema.users_schema().name.clone(), + &["name", "age"], + &[&"user_1", &1], + ) + .await + .unwrap(); + + event_notify.notified().await; + + let mut users_table = delta_database + .load_table(&database_schema.users_schema().name) + .await + .unwrap(); + + assert_table_snapshot!("table_insert_update_delete_1_insert", users_table.clone()); + + let event_notify = destination + .wait_for_events_count(vec![(EventType::Update, 1)]) + .await; + + // Update the row. + database + .update_values( + database_schema.users_schema().name.clone(), + &["name", "age"], + &[&"user_10", &10], + ) + .await + .unwrap(); + + event_notify.notified().await; + + users_table.load().await.unwrap(); + + assert_table_snapshot!("table_insert_update_delete_2_update", users_table.clone()); + + // Wait for the delete. + let event_notify = destination + .wait_for_events_count(vec![(EventType::Delete, 1)]) + .await; + + // Delete the row. + database + .delete_values( + database_schema.users_schema().name.clone(), + &["name"], + &["'user_10'"], + "", + ) + .await + .unwrap(); + + event_notify.notified().await; + + pipeline.shutdown_and_wait().await.unwrap(); + + users_table.load().await.unwrap(); + + assert_table_snapshot!("table_insert_update_delete_3_delete", users_table); +} + +#[tokio::test(flavor = "multi_thread")] +async fn table_subsequent_updates() { + init_test_tracing(); + + let mut database_1 = spawn_source_database().await; + let mut database_2 = database_1.duplicate().await; + let database_schema = setup_test_database_schema(&database_1, TableSelection::UsersOnly).await; + + let delta_database = setup_delta_connection().await; + + let store = NotifyingStore::new(); + let raw_destination = delta_database.build_destination(store.clone()).await; + let destination = TestDestinationWrapper::wrap(raw_destination); + + // Start pipeline from scratch. + let pipeline_id: PipelineId = random(); + let mut pipeline = create_pipeline( + &database_1.config, + pipeline_id, + database_schema.publication_name(), + store.clone(), + destination.clone(), + ); + + // Register notifications for table copy completion. + let users_state_notify = store + .notify_on_table_state_type( + database_schema.users_schema().id, + TableReplicationPhaseType::SyncDone, + ) + .await; + + pipeline.start().await.unwrap(); + + users_state_notify.notified().await; + + // Wait for the first insert and two updates. + let event_notify = destination + .wait_for_events_count(vec![(EventType::Insert, 1), (EventType::Update, 2)]) + .await; + + // Insert a row. + database_1 + .insert_values( + database_schema.users_schema().name.clone(), + &["name", "age"], + &[&"user_1", &1], + ) + .await + .unwrap(); + + // Create two transactions A and B on separate connections to make sure that the updates are + // ordered correctly. + let transaction_a = database_1.begin_transaction().await; + transaction_a + .update_values( + database_schema.users_schema().name.clone(), + &["name", "age"], + &[&"user_3", &3], + ) + .await + .unwrap(); + transaction_a.commit_transaction().await; + let transaction_b = database_2.begin_transaction().await; + transaction_b + .update_values( + database_schema.users_schema().name.clone(), + &["name", "age"], + &[&"user_2", &2], + ) + .await + .unwrap(); + transaction_b.commit_transaction().await; + + event_notify.notified().await; + + pipeline.shutdown_and_wait().await.unwrap(); + + let users_table = delta_database + .load_table(&database_schema.users_schema().name) + .await + .unwrap(); + + assert_table_snapshot!("table_subsequent_updates_insert", users_table.clone()); +} + +#[tokio::test(flavor = "multi_thread")] +async fn table_truncate_with_batching() { + init_test_tracing(); + + let mut database = spawn_source_database().await; + let database_schema = setup_test_database_schema(&database, TableSelection::Both).await; + + let delta_database = setup_delta_connection().await; + + let store = NotifyingStore::new(); + let raw_destination = delta_database.build_destination(store.clone()).await; + let destination = TestDestinationWrapper::wrap(raw_destination); + + // Start pipeline from scratch. + let pipeline_id: PipelineId = random(); + let mut pipeline = create_pipeline_with( + &database.config, + pipeline_id, + database_schema.publication_name(), + store.clone(), + destination.clone(), + // We use a batch size > 1, so that we can make sure that interleaved truncate statements + // work well with multiple batches of events. + Some(BatchConfig { + max_size: 10, + max_fill_ms: 1000, + }), + ); + + // Register notifications for table copy completion. + let users_state_notify = store + .notify_on_table_state_type( + database_schema.users_schema().id, + TableReplicationPhaseType::SyncDone, + ) + .await; + let orders_state_notify = store + .notify_on_table_state_type( + database_schema.orders_schema().id, + TableReplicationPhaseType::SyncDone, + ) + .await; + + pipeline.start().await.unwrap(); + + users_state_notify.notified().await; + orders_state_notify.notified().await; + + // Wait for the 8 inserts (4 per table + 4 after truncate) and 2 truncates (1 per table). + let event_notify = destination + .wait_for_events_count(vec![(EventType::Insert, 8), (EventType::Truncate, 2)]) + .await; + + // Insert 2 rows per each table. + insert_mock_data( + &mut database, + &database_schema.users_schema().name, + &database_schema.orders_schema().name, + 1..=2, + false, + ) + .await; + + // We truncate both tables. + database + .truncate_table(database_schema.users_schema().name.clone()) + .await + .unwrap(); + database + .truncate_table(database_schema.orders_schema().name.clone()) + .await + .unwrap(); + + // Insert 2 extra rows per each table. + insert_mock_data( + &mut database, + &database_schema.users_schema().name, + &database_schema.orders_schema().name, + 3..=4, + false, + ) + .await; + + event_notify.notified().await; + + pipeline.shutdown_and_wait().await.unwrap(); + + let users_table = delta_database + .load_table(&database_schema.users_schema().name) + .await + .unwrap(); + let orders_table = delta_database + .load_table(&database_schema.orders_schema().name) + .await + .unwrap(); + + assert_table_snapshot!("table_truncate_with_batching_users_table", users_table); + assert_table_snapshot!("table_truncate_with_batching_orders_table", orders_table); +} + +#[tokio::test(flavor = "multi_thread")] +async fn decimal_precision_scale_mapping() { + init_test_tracing(); + + let database = spawn_source_database().await; + let delta_database = setup_delta_connection().await; + let table_name = test_table_name("decimal_precision_test"); + + let columns = vec![ + ("id", "bigint primary key"), + ("price", "numeric(10,2)"), // NUMERIC(10,2) -> DECIMAL(10,2) + ("percentage", "numeric(5,4)"), // NUMERIC(5,4) -> DECIMAL(5,4) + ("large_number", "numeric(18,6)"), // NUMERIC(18,6) -> DECIMAL(18,6) + ("currency", "numeric(15,3)"), // NUMERIC(15,3) -> DECIMAL(15,3) + ]; + + let table_id = database + .create_table(table_name.clone(), false, &columns) + .await + .unwrap(); + + let store = NotifyingStore::new(); + let raw_destination = delta_database.build_destination(store.clone()).await; + let destination = TestDestinationWrapper::wrap(raw_destination); + + let publication_name = "test_pub_decimal".to_string(); + database + .create_publication(&publication_name, std::slice::from_ref(&table_name)) + .await + .expect("Failed to create publication"); + + let pipeline_id: PipelineId = random(); + let mut pipeline = create_pipeline( + &database.config, + pipeline_id, + publication_name, + store.clone(), + destination.clone(), + ); + + let table_sync_done_notification = store + .notify_on_table_state_type(table_id, TableReplicationPhaseType::SyncDone) + .await; + + pipeline.start().await.unwrap(); + table_sync_done_notification.notified().await; + + let event_notify = destination + .wait_for_events_count(vec![(EventType::Insert, 2)]) + .await; + + database + .insert_values( + table_name.clone(), + &["id", "price", "percentage", "large_number", "currency"], + &[ + &1i64, + &PgNumeric::from_str("123.45").unwrap(), // NUMERIC(10,2) + &PgNumeric::from_str("0.9876").unwrap(), // NUMERIC(5,4) + &PgNumeric::from_str("1234567.123456").unwrap(), // NUMERIC(18,6) + &PgNumeric::from_str("9999.999").unwrap(), // NUMERIC(15,3) + ], + ) + .await + .unwrap(); + + database + .insert_values( + table_name.clone(), + &["id", "price", "percentage", "large_number", "currency"], + &[ + &2i64, + &PgNumeric::from_str("999.99").unwrap(), // NUMERIC(10,2) + &PgNumeric::from_str("0.0001").unwrap(), // NUMERIC(5,4) + &PgNumeric::from_str("999999.999999").unwrap(), // NUMERIC(18,6) + &PgNumeric::from_str("12345.678").unwrap(), // NUMERIC(15,3) + ], + ) + .await + .unwrap(); + + event_notify.notified().await; + pipeline.shutdown_and_wait().await.unwrap(); + + let table = delta_database.load_table(&table_name).await.unwrap(); + assert_table_snapshot!("decimal_precision_scale_mapping", table); +} + +/// Test comprehensive data type mapping from Postgres to Delta Lake +#[tokio::test(flavor = "multi_thread")] +async fn data_type_mapping() { + init_test_tracing(); + + let database = spawn_source_database().await; + let delta_database = setup_delta_connection().await; + let table_name = test_table_name("comprehensive_types"); + + let columns = vec![ + ("id", "bigint primary key"), + ("bool_col", "boolean"), + ("bpchar_col", "char(5)"), + ("varchar_col", "varchar(255)"), + ("name_col", "name"), + ("text_col", "text"), + ("int2_col", "smallint"), + ("int4_col", "integer"), + ("int8_col", "bigint"), + ("float4_col", "real"), + ("float8_col", "double precision"), + ("numeric_col", "numeric(10,2)"), + ("date_col", "date"), + ("time_col", "time"), + ("timestamp_col", "timestamp"), + ("timestamptz_col", "timestamptz"), + ("uuid_col", "uuid"), + ("json_col", "json"), + ("jsonb_col", "jsonb"), + ("oid_col", "oid"), + ("bytea_col", "bytea"), + ("bool_array_col", "boolean[]"), + ("bpchar_array_col", "char(5)[]"), + ("varchar_array_col", "varchar(255)[]"), + ("name_array_col", "name[]"), + ("text_array_col", "text[]"), + ("int2_array_col", "smallint[]"), + ("int4_array_col", "integer[]"), + ("int8_array_col", "bigint[]"), + ("float4_array_col", "real[]"), + ("float8_array_col", "double precision[]"), + ("numeric_array_col", "numeric(10,2)[]"), + ("date_array_col", "date[]"), + ("time_array_col", "time[]"), + ("timestamp_array_col", "timestamp[]"), + ("timestamptz_array_col", "timestamptz[]"), + ("uuid_array_col", "uuid[]"), + ("json_array_col", "json[]"), + ("jsonb_array_col", "jsonb[]"), + ("oid_array_col", "oid[]"), + ("bytea_array_col", "bytea[]"), + ]; + + let table_id = database + .create_table( + table_name.clone(), + false, // Don't create automatic BIGSERIAL id column to avoid sequence conflicts + &columns, + ) + .await + .unwrap(); + + let store = NotifyingStore::new(); + let raw_destination = delta_database.build_destination(store.clone()).await; + let destination = TestDestinationWrapper::wrap(raw_destination); + + let publication_name = "test_pub_types".to_string(); + database + .create_publication(&publication_name, std::slice::from_ref(&table_name)) + .await + .expect("Failed to create publication"); + + let pipeline_id: PipelineId = random(); + let mut pipeline = create_pipeline( + &database.config, + pipeline_id, + publication_name, + store.clone(), + destination.clone(), + ); + + let table_sync_done_notification = store + .notify_on_table_state_type(table_id, TableReplicationPhaseType::SyncDone) + .await; + + pipeline.start().await.unwrap(); + table_sync_done_notification.notified().await; + + // Insert test data with various types + let event_notify = destination + .wait_for_events_count(vec![(EventType::Insert, 1)]) + .await; + + let id_value = 1i64; + let bool_value = true; + let bpchar_value = "fixed".to_string(); + let varchar_value = "varchar sample".to_string(); + let name_value = "pg_name_value".to_string(); + let text_value = "text field content".to_string(); + let int2_value = 42i16; + let int4_value = 4242i32; + let int8_value = 4242_4242i64; + let float4_value = 1.25f32; + let float8_value = 9.875f64; + let numeric_value = PgNumeric::from_str("12345.67").unwrap(); + let date_value = NaiveDate::from_ymd_opt(1993, 1, 15).unwrap(); + let time_value = NaiveTime::from_hms_micro_opt(10, 11, 12, 123_456).unwrap(); + let timestamp_value = NaiveDateTime::new( + NaiveDate::from_ymd_opt(2023, 1, 1).unwrap(), + NaiveTime::from_hms_opt(12, 0, 0).unwrap(), + ); + let timestamptz_value = DateTime::::from_naive_utc_and_offset(timestamp_value, Utc); + let uuid_value = Uuid::parse_str("00000000-0000-0000-0000-000000000001").unwrap(); + let json_value = json!({"kind": "json"}); + let jsonb_value = json!({"kind": "jsonb"}); + let oid_value = 424_242u32; + let bytea_value = b"Hello Delta".to_vec(); + + let bool_array = vec![true, false, true]; + let bpchar_array = vec!["one".to_string(), "two".to_string()]; + let varchar_array = vec!["alpha".to_string(), "beta".to_string()]; + let name_array = vec!["first_name".to_string(), "second_name".to_string()]; + let text_array = vec!["text one".to_string(), "text two".to_string()]; + let int2_array = vec![1i16, 2i16, 3i16]; + let int4_array = vec![10i32, 20i32]; + let int8_array = vec![100i64, 200i64]; + let float4_array = vec![1.5f32, 2.5f32]; + let float8_array = vec![3.5f64, 4.5f64]; + let numeric_array = vec![ + PgNumeric::from_str("10.10").unwrap(), + PgNumeric::from_str("20.20").unwrap(), + ]; + let date_array = vec![ + NaiveDate::from_ymd_opt(2020, 1, 1).unwrap(), + NaiveDate::from_ymd_opt(2020, 12, 31).unwrap(), + ]; + let time_array = vec![ + NaiveTime::from_hms_micro_opt(1, 2, 3, 0).unwrap(), + NaiveTime::from_hms_micro_opt(4, 5, 6, 789_000).unwrap(), + ]; + let timestamp_array = vec![ + NaiveDateTime::new( + NaiveDate::from_ymd_opt(2021, 3, 14).unwrap(), + NaiveTime::from_hms_opt(1, 59, 26).unwrap(), + ), + NaiveDateTime::new( + NaiveDate::from_ymd_opt(2022, 6, 30).unwrap(), + NaiveTime::from_hms_opt(23, 0, 0).unwrap(), + ), + ]; + let timestamptz_array: Vec> = timestamp_array + .iter() + .map(|dt| DateTime::::from_naive_utc_and_offset(*dt, Utc)) + .collect(); + let uuid_array = vec![ + Uuid::parse_str("00000000-0000-0000-0000-000000000001").unwrap(), + Uuid::parse_str("00000000-0000-0000-0000-000000000002").unwrap(), + ]; + let json_array = vec![json!({"idx": 1}), json!({"idx": 2})]; + let jsonb_array = vec![json!({"code": "a"}), json!({"code": "b"})]; + let oid_array = vec![7_000u32, 7_001u32]; + let bytea_array = vec![b"bytes1".to_vec(), b"bytes2".to_vec()]; + + let column_names: Vec<&str> = columns.iter().map(|(name, _)| *name).collect(); + let values: Vec<&(dyn ToSql + Sync)> = vec![ + &id_value, + &bool_value, + &bpchar_value, + &varchar_value, + &name_value, + &text_value, + &int2_value, + &int4_value, + &int8_value, + &float4_value, + &float8_value, + &numeric_value, + &date_value, + &time_value, + ×tamp_value, + ×tamptz_value, + &uuid_value, + &json_value, + &jsonb_value, + &oid_value, + &bytea_value, + &bool_array, + &bpchar_array, + &varchar_array, + &name_array, + &text_array, + &int2_array, + &int4_array, + &int8_array, + &float4_array, + &float8_array, + &numeric_array, + &date_array, + &time_array, + ×tamp_array, + ×tamptz_array, + &uuid_array, + &json_array, + &jsonb_array, + &oid_array, + &bytea_array, + ]; + + database + .insert_values(table_name.clone(), &column_names, &values) + .await + .unwrap(); + + event_notify.notified().await; + pipeline.shutdown_and_wait().await.unwrap(); + + let table = delta_database.load_table(&table_name).await.unwrap(); + assert_table_snapshot!("data_type_mapping", table); +} + +/// Test CDC deduplication and conflict resolution +#[tokio::test(flavor = "multi_thread")] +async fn test_cdc_deduplication_and_conflict_resolution() { + init_test_tracing(); + + let database = spawn_source_database().await; + let database_schema = setup_test_database_schema(&database, TableSelection::UsersOnly).await; + + let delta_database = setup_delta_connection().await; + + let store = NotifyingStore::new(); + let raw_destination = delta_database.build_destination(store.clone()).await; + let destination = TestDestinationWrapper::wrap(raw_destination); + + let pipeline_id: PipelineId = random(); + let mut pipeline = create_pipeline( + &database.config, + pipeline_id, + database_schema.publication_name(), + store.clone(), + destination.clone(), + ); + + let users_state_notify = store + .notify_on_table_state_type( + database_schema.users_schema().id, + TableReplicationPhaseType::SyncDone, + ) + .await; + + pipeline.start().await.unwrap(); + users_state_notify.notified().await; + + // Test scenario: Insert, multiple updates, and final delete for the same row + // This tests the last-wins deduplication logic + let event_notify = destination + .wait_for_events_count(vec![ + (EventType::Insert, 1), + (EventType::Update, 3), + (EventType::Delete, 1), + ]) + .await; + + // Insert a row + database + .insert_values( + database_schema.users_schema().name.clone(), + &["name", "age"], + &[&"test_user", &20], + ) + .await + .unwrap(); + + // Multiple rapid updates to test deduplication + database + .update_values( + database_schema.users_schema().name.clone(), + &["name", "age"], + &[&"test_user_v2", &21], + ) + .await + .unwrap(); + + database + .update_values( + database_schema.users_schema().name.clone(), + &["name", "age"], + &[&"test_user_v3", &22], + ) + .await + .unwrap(); + + database + .update_values( + database_schema.users_schema().name.clone(), + &["name", "age"], + &[&"test_user_final", &23], + ) + .await + .unwrap(); + + // Delete the row + database + .delete_values( + database_schema.users_schema().name.clone(), + &["name"], + &["'test_user_final'"], + "", + ) + .await + .unwrap(); + + event_notify.notified().await; + pipeline.shutdown_and_wait().await.unwrap(); + + let users_table = delta_database + .load_table(&database_schema.users_schema().name) + .await + .unwrap(); + + assert_table_snapshot!( + "test_cdc_deduplication_and_conflict_resolution", + users_table + ); +} + +/// Test large transaction handling and batching behavior +#[tokio::test(flavor = "multi_thread")] +async fn test_large_transaction_batching() { + init_test_tracing(); + + let mut database = spawn_source_database().await; + let database_schema = setup_test_database_schema(&database, TableSelection::UsersOnly).await; + + let delta_database = setup_delta_connection().await; + + let store = NotifyingStore::new(); + let raw_destination = delta_database.build_destination(store.clone()).await; + let destination = TestDestinationWrapper::wrap(raw_destination); + + let pipeline_id: PipelineId = random(); + let batch_size = 5; + let mut pipeline = create_pipeline_with( + &database.config, + pipeline_id, + database_schema.publication_name(), + store.clone(), + destination.clone(), + Some(BatchConfig { + max_size: batch_size, // Small batch size to force multiple batches + max_fill_ms: 1000, + }), + ); + + let users_state_notify = store + .notify_on_table_state_type( + database_schema.users_schema().id, + TableReplicationPhaseType::SyncDone, + ) + .await; + + pipeline.start().await.unwrap(); + users_state_notify.notified().await; + + // Insert many rows in a single transaction to test batching + let insert_count: usize = 20; + let event_notify = destination + .wait_for_events_count(vec![(EventType::Insert, insert_count as u64)]) + .await; + + let transaction = database.begin_transaction().await; + for i in 1..=insert_count { + transaction + .insert_values( + database_schema.users_schema().name.clone(), + &["name", "age"], + &[&format!("batch_user_{i}"), &(20 + i as i32)], + ) + .await + .unwrap(); + } + transaction.commit_transaction().await; + + event_notify.notified().await; + pipeline.shutdown_and_wait().await.unwrap(); + + let users_table = delta_database + .load_table(&database_schema.users_schema().name) + .await + .unwrap(); + assert_table_snapshot!("test_large_transaction_batching", users_table.clone()); + let commits = users_table.history(None).await.unwrap().collect::>(); + // Due to the batch timeout, in practice, there will be more commits than the batch size. + assert!(commits.len() >= (insert_count / batch_size)); +} + +#[tokio::test(flavor = "multi_thread")] +async fn compaction_minimizes_small_files() { + init_test_tracing(); + + let database = spawn_source_database().await; + let database_schema = setup_test_database_schema(&database, TableSelection::UsersOnly).await; + + let delta_database = setup_delta_connection().await; + + let store = NotifyingStore::new(); + + // Configure compaction to run after every commit for the users table. + let mut table_config: HashMap> = + HashMap::new(); + table_config.insert( + database_schema.users_schema().name.name.clone(), + Arc::new(etl_destinations::deltalake::DeltaTableConfig { + compact_after_commits: Some(NonZeroU64::new(1).unwrap()), + ..Default::default() + }), + ); + + let raw_destination = delta_database + .build_destination_with_config(store.clone(), table_config) + .await; + let destination = TestDestinationWrapper::wrap(raw_destination); + + // Use a batch size of 1 so each insert becomes a separate commit and small file. + let pipeline_id: PipelineId = random(); + let mut pipeline = create_pipeline_with( + &database.config, + pipeline_id, + database_schema.publication_name(), + store.clone(), + destination.clone(), + Some(BatchConfig { + max_size: 1, + max_fill_ms: 1000, + }), + ); + + let users_state_notify = store + .notify_on_table_state_type( + database_schema.users_schema().id, + TableReplicationPhaseType::SyncDone, + ) + .await; + + pipeline.start().await.unwrap(); + users_state_notify.notified().await; + + // Generate several inserts to create many small files (one per commit). + let insert_count: u64 = 12; + let event_notify = destination + .wait_for_events_count(vec![(EventType::Insert, insert_count)]) + .await; + + for i in 1..=insert_count { + database + .insert_values( + database_schema.users_schema().name.clone(), + &["name", "age"], + &[&format!("c_user_{i}"), &(i as i32)], + ) + .await + .unwrap(); + } + + event_notify.notified().await; + + tokio::time::sleep(std::time::Duration::from_secs(1)).await; + + pipeline.shutdown_and_wait().await.unwrap(); + + let users_table = delta_database + .load_table(&database_schema.users_schema().name) + .await + .unwrap(); + + assert_table_snapshot!("compaction_minimizes_small_files", users_table.clone()); + assert!(users_table.snapshot().unwrap().file_paths_iter().count() <= 12); +} diff --git a/etl-destinations/tests/iceberg_pipeline.rs b/etl-destinations/tests/iceberg_pipeline.rs index f552677ca..20b804576 100644 --- a/etl-destinations/tests/iceberg_pipeline.rs +++ b/etl-destinations/tests/iceberg_pipeline.rs @@ -1,4 +1,4 @@ -#![cfg(feature = "iceberg")] +#![cfg(any())] use std::collections::HashMap; diff --git a/etl-destinations/tests/snapshots/deltalake_pipeline__append_only_ignores_updates_and_deletes.snap b/etl-destinations/tests/snapshots/deltalake_pipeline__append_only_ignores_updates_and_deletes.snap new file mode 100644 index 000000000..73535e58d --- /dev/null +++ b/etl-destinations/tests/snapshots/deltalake_pipeline__append_only_ignores_updates_and_deletes.snap @@ -0,0 +1,15 @@ +--- +source: etl-destinations/tests/deltalake_pipeline.rs +expression: users_table +--- +# Schema +- id: Primitive(Long) nullable=false +- name: Primitive(String) nullable=false +- age: Primitive(Integer) nullable=false + +# Data ++----+-------------+-----+ +| id | name | age | ++----+-------------+-----+ +| 1 | append_user | 10 | ++----+-------------+-----+ diff --git a/etl-destinations/tests/snapshots/deltalake_pipeline__compaction_minimizes_small_files.snap b/etl-destinations/tests/snapshots/deltalake_pipeline__compaction_minimizes_small_files.snap new file mode 100644 index 000000000..d95b186ff --- /dev/null +++ b/etl-destinations/tests/snapshots/deltalake_pipeline__compaction_minimizes_small_files.snap @@ -0,0 +1,26 @@ +--- +source: etl-destinations/tests/deltalake_pipeline.rs +expression: users_table +--- +# Schema +- id: Primitive(Long) nullable=false +- name: Primitive(String) nullable=false +- age: Primitive(Integer) nullable=false + +# Data ++----+-----------+-----+ +| id | name | age | ++----+-----------+-----+ +| 1 | c_user_1 | 1 | +| 2 | c_user_2 | 2 | +| 3 | c_user_3 | 3 | +| 4 | c_user_4 | 4 | +| 5 | c_user_5 | 5 | +| 6 | c_user_6 | 6 | +| 7 | c_user_7 | 7 | +| 8 | c_user_8 | 8 | +| 9 | c_user_9 | 9 | +| 10 | c_user_10 | 10 | +| 11 | c_user_11 | 11 | +| 12 | c_user_12 | 12 | ++----+-----------+-----+ diff --git a/etl-destinations/tests/snapshots/deltalake_pipeline__data_type_mapping.snap b/etl-destinations/tests/snapshots/deltalake_pipeline__data_type_mapping.snap new file mode 100644 index 000000000..cc7059b0e --- /dev/null +++ b/etl-destinations/tests/snapshots/deltalake_pipeline__data_type_mapping.snap @@ -0,0 +1,53 @@ +--- +source: etl-destinations/tests/deltalake_pipeline.rs +expression: table +--- +# Schema +- id: Primitive(Long) nullable=false +- bool_col: Primitive(Boolean) nullable=true +- bpchar_col: Primitive(String) nullable=true +- varchar_col: Primitive(String) nullable=true +- name_col: Primitive(String) nullable=true +- text_col: Primitive(String) nullable=true +- int2_col: Primitive(Integer) nullable=true +- int4_col: Primitive(Integer) nullable=true +- int8_col: Primitive(Long) nullable=true +- float4_col: Primitive(Float) nullable=true +- float8_col: Primitive(Double) nullable=true +- numeric_col: Primitive(Decimal(DecimalType { precision: 10, scale: 2 })) nullable=true +- date_col: Primitive(Date) nullable=true +- time_col: Primitive(Timestamp) nullable=true +- timestamp_col: Primitive(TimestampNtz) nullable=true +- timestamptz_col: Primitive(Timestamp) nullable=true +- uuid_col: Primitive(String) nullable=true +- json_col: Primitive(String) nullable=true +- jsonb_col: Primitive(String) nullable=true +- oid_col: Primitive(Long) nullable=true +- bytea_col: Primitive(Binary) nullable=true +- bool_array_col: Array(ArrayType { type_name: "array", element_type: Primitive(Boolean), contains_null: true }) nullable=true +- bpchar_array_col: Array(ArrayType { type_name: "array", element_type: Primitive(String), contains_null: true }) nullable=true +- varchar_array_col: Array(ArrayType { type_name: "array", element_type: Primitive(String), contains_null: true }) nullable=true +- name_array_col: Array(ArrayType { type_name: "array", element_type: Primitive(String), contains_null: true }) nullable=true +- text_array_col: Array(ArrayType { type_name: "array", element_type: Primitive(String), contains_null: true }) nullable=true +- int2_array_col: Array(ArrayType { type_name: "array", element_type: Primitive(Integer), contains_null: true }) nullable=true +- int4_array_col: Array(ArrayType { type_name: "array", element_type: Primitive(Integer), contains_null: true }) nullable=true +- int8_array_col: Array(ArrayType { type_name: "array", element_type: Primitive(Long), contains_null: true }) nullable=true +- float4_array_col: Array(ArrayType { type_name: "array", element_type: Primitive(Float), contains_null: true }) nullable=true +- float8_array_col: Array(ArrayType { type_name: "array", element_type: Primitive(Double), contains_null: true }) nullable=true +- numeric_array_col: Array(ArrayType { type_name: "array", element_type: Primitive(String), contains_null: true }) nullable=true +- date_array_col: Array(ArrayType { type_name: "array", element_type: Primitive(Date), contains_null: true }) nullable=true +- time_array_col: Array(ArrayType { type_name: "array", element_type: Primitive(Timestamp), contains_null: true }) nullable=true +- timestamp_array_col: Array(ArrayType { type_name: "array", element_type: Primitive(TimestampNtz), contains_null: true }) nullable=true +- timestamptz_array_col: Array(ArrayType { type_name: "array", element_type: Primitive(Timestamp), contains_null: true }) nullable=true +- uuid_array_col: Array(ArrayType { type_name: "array", element_type: Primitive(String), contains_null: true }) nullable=true +- json_array_col: Array(ArrayType { type_name: "array", element_type: Primitive(String), contains_null: true }) nullable=true +- jsonb_array_col: Array(ArrayType { type_name: "array", element_type: Primitive(String), contains_null: true }) nullable=true +- oid_array_col: Array(ArrayType { type_name: "array", element_type: Primitive(Long), contains_null: true }) nullable=true +- bytea_array_col: Array(ArrayType { type_name: "array", element_type: Primitive(Binary), contains_null: true }) nullable=true + +# Data ++----+----------+------------+----------------+---------------+--------------------+----------+----------+----------+------------+------------+-------------+------------+-----------------------------+---------------------+----------------------+----------+-----------------+------------------+---------+------------------------+---------------------+------------------+-------------------+---------------------------+----------------------+----------------+----------------+----------------+------------------+------------------+-------------------+--------------------------+--------------------------------------------------+--------------------------------------------+----------------------------------------------+------------------------------------------------------------------------------+------------------------+------------------------------+---------------+------------------------------+ +| id | bool_col | bpchar_col | varchar_col | name_col | text_col | int2_col | int4_col | int8_col | float4_col | float8_col | numeric_col | date_col | time_col | timestamp_col | timestamptz_col | uuid_col | json_col | jsonb_col | oid_col | bytea_col | bool_array_col | bpchar_array_col | varchar_array_col | name_array_col | text_array_col | int2_array_col | int4_array_col | int8_array_col | float4_array_col | float8_array_col | numeric_array_col | date_array_col | time_array_col | timestamp_array_col | timestamptz_array_col | uuid_array_col | json_array_col | jsonb_array_col | oid_array_col | bytea_array_col | ++----+----------+------------+----------------+---------------+--------------------+----------+----------+----------+------------+------------+-------------+------------+-----------------------------+---------------------+----------------------+----------+-----------------+------------------+---------+------------------------+---------------------+------------------+-------------------+---------------------------+----------------------+----------------+----------------+----------------+------------------+------------------+-------------------+--------------------------+--------------------------------------------------+--------------------------------------------+----------------------------------------------+------------------------------------------------------------------------------+------------------------+------------------------------+---------------+------------------------------+ +| 1 | true | fixed | varchar sample | pg_name_value | text field content | 42 | 4242 | 42424242 | 1.25 | 9.875 | 12345.67 | 1993-01-15 | 1970-01-01T10:11:12.123456Z | 2023-01-01T12:00:00 | 2023-01-01T12:00:00Z | | {"kind":"json"} | {"kind":"jsonb"} | 424242 | 48656c6c6f2044656c7461 | [true, false, true] | [one , two ] | [alpha, beta] | [first_name, second_name] | [text one, text two] | [1, 2, 3] | [10, 20] | [100, 200] | [1.5, 2.5] | [3.5, 4.5] | [10.10, 20.20] | [2020-01-01, 2020-12-31] | [1970-01-01T01:02:03Z, 1970-01-01T04:05:06.789Z] | [2021-03-14T01:59:26, 2022-06-30T23:00:00] | [2021-03-14T01:59:26Z, 2022-06-30T23:00:00Z] | [00000000-0000-0000-0000-000000000001, 00000000-0000-0000-0000-000000000002] | [{"idx":1}, {"idx":2}] | [{"code":"a"}, {"code":"b"}] | [7000, 7001] | [627974657331, 627974657332] | ++----+----------+------------+----------------+---------------+--------------------+----------+----------+----------+------------+------------+-------------+------------+-----------------------------+---------------------+----------------------+----------+-----------------+------------------+---------+------------------------+---------------------+------------------+-------------------+---------------------------+----------------------+----------------+----------------+----------------+------------------+------------------+-------------------+--------------------------+--------------------------------------------------+--------------------------------------------+----------------------------------------------+------------------------------------------------------------------------------+------------------------+------------------------------+---------------+------------------------------+ diff --git a/etl-destinations/tests/snapshots/deltalake_pipeline__decimal_precision_scale_mapping.snap b/etl-destinations/tests/snapshots/deltalake_pipeline__decimal_precision_scale_mapping.snap new file mode 100644 index 000000000..469fd18e2 --- /dev/null +++ b/etl-destinations/tests/snapshots/deltalake_pipeline__decimal_precision_scale_mapping.snap @@ -0,0 +1,18 @@ +--- +source: etl-destinations/tests/deltalake_pipeline.rs +expression: table_name_ref +--- +# Schema +- id: Primitive(Long) nullable=false +- price: Primitive(Decimal(DecimalType { precision: 10, scale: 2 })) nullable=true +- percentage: Primitive(Decimal(DecimalType { precision: 5, scale: 4 })) nullable=true +- large_number: Primitive(Decimal(DecimalType { precision: 18, scale: 6 })) nullable=true +- currency: Primitive(Decimal(DecimalType { precision: 15, scale: 3 })) nullable=true + +# Data ++----+--------+------------+----------------+-----------+ +| id | price | percentage | large_number | currency | ++----+--------+------------+----------------+-----------+ +| 1 | 123.45 | 0.9876 | 1234567.123456 | 9999.999 | +| 2 | 999.99 | 0.0001 | 999999.999999 | 12345.678 | ++----+--------+------------+----------------+-----------+ diff --git a/etl-destinations/tests/snapshots/deltalake_pipeline__merge_with_delete_validation.snap b/etl-destinations/tests/snapshots/deltalake_pipeline__merge_with_delete_validation.snap new file mode 100644 index 000000000..24d3b1454 --- /dev/null +++ b/etl-destinations/tests/snapshots/deltalake_pipeline__merge_with_delete_validation.snap @@ -0,0 +1,15 @@ +--- +source: etl-destinations/tests/deltalake_pipeline.rs +expression: users_table +--- +# Schema +- id: Primitive(Long) nullable=false +- name: Primitive(String) nullable=false +- age: Primitive(Integer) nullable=false + +# Data ++----+----------+-----+ +| id | name | age | ++----+----------+-----+ +| 2 | d_user_b | 12 | ++----+----------+-----+ diff --git a/etl-destinations/tests/snapshots/deltalake_pipeline__table_copy_and_streaming_with_restart_orders_table_1.snap b/etl-destinations/tests/snapshots/deltalake_pipeline__table_copy_and_streaming_with_restart_orders_table_1.snap new file mode 100644 index 000000000..0b3025a14 --- /dev/null +++ b/etl-destinations/tests/snapshots/deltalake_pipeline__table_copy_and_streaming_with_restart_orders_table_1.snap @@ -0,0 +1,15 @@ +--- +source: etl-destinations/tests/deltalake_pipeline.rs +expression: orders_table +--- +# Schema +- id: Primitive(Long) nullable=false +- description: Primitive(String) nullable=false + +# Data ++----+---------------+ +| id | description | ++----+---------------+ +| 1 | description_1 | +| 2 | description_2 | ++----+---------------+ diff --git a/etl-destinations/tests/snapshots/deltalake_pipeline__table_copy_and_streaming_with_restart_orders_table_2.snap b/etl-destinations/tests/snapshots/deltalake_pipeline__table_copy_and_streaming_with_restart_orders_table_2.snap new file mode 100644 index 000000000..924cd9fd4 --- /dev/null +++ b/etl-destinations/tests/snapshots/deltalake_pipeline__table_copy_and_streaming_with_restart_orders_table_2.snap @@ -0,0 +1,17 @@ +--- +source: etl-destinations/tests/deltalake_pipeline.rs +expression: orders_table +--- +# Schema +- id: Primitive(Long) nullable=false +- description: Primitive(String) nullable=false + +# Data ++----+---------------+ +| id | description | ++----+---------------+ +| 1 | description_1 | +| 2 | description_2 | +| 3 | description_3 | +| 4 | description_4 | ++----+---------------+ diff --git a/etl-destinations/tests/snapshots/deltalake_pipeline__table_copy_and_streaming_with_restart_users_table_1.snap b/etl-destinations/tests/snapshots/deltalake_pipeline__table_copy_and_streaming_with_restart_users_table_1.snap new file mode 100644 index 000000000..cadfcedda --- /dev/null +++ b/etl-destinations/tests/snapshots/deltalake_pipeline__table_copy_and_streaming_with_restart_users_table_1.snap @@ -0,0 +1,16 @@ +--- +source: etl-destinations/tests/deltalake_pipeline.rs +expression: users_table +--- +# Schema +- id: Primitive(Long) nullable=false +- name: Primitive(String) nullable=false +- age: Primitive(Integer) nullable=false + +# Data ++----+--------+-----+ +| id | name | age | ++----+--------+-----+ +| 1 | user_1 | 1 | +| 2 | user_2 | 2 | ++----+--------+-----+ diff --git a/etl-destinations/tests/snapshots/deltalake_pipeline__table_copy_and_streaming_with_restart_users_table_2.snap b/etl-destinations/tests/snapshots/deltalake_pipeline__table_copy_and_streaming_with_restart_users_table_2.snap new file mode 100644 index 000000000..e1733c3f7 --- /dev/null +++ b/etl-destinations/tests/snapshots/deltalake_pipeline__table_copy_and_streaming_with_restart_users_table_2.snap @@ -0,0 +1,18 @@ +--- +source: etl-destinations/tests/deltalake_pipeline.rs +expression: users_table +--- +# Schema +- id: Primitive(Long) nullable=false +- name: Primitive(String) nullable=false +- age: Primitive(Integer) nullable=false + +# Data ++----+--------+-----+ +| id | name | age | ++----+--------+-----+ +| 1 | user_1 | 1 | +| 2 | user_2 | 2 | +| 3 | user_3 | 3 | +| 4 | user_4 | 4 | ++----+--------+-----+ diff --git a/etl-destinations/tests/snapshots/deltalake_pipeline__table_insert_update_delete_1_insert.snap b/etl-destinations/tests/snapshots/deltalake_pipeline__table_insert_update_delete_1_insert.snap new file mode 100644 index 000000000..4914e1451 --- /dev/null +++ b/etl-destinations/tests/snapshots/deltalake_pipeline__table_insert_update_delete_1_insert.snap @@ -0,0 +1,15 @@ +--- +source: etl-destinations/tests/deltalake_pipeline.rs +expression: users_table +--- +# Schema +- id: Primitive(Long) nullable=false +- name: Primitive(String) nullable=false +- age: Primitive(Integer) nullable=false + +# Data ++----+--------+-----+ +| id | name | age | ++----+--------+-----+ +| 1 | user_1 | 1 | ++----+--------+-----+ diff --git a/etl-destinations/tests/snapshots/deltalake_pipeline__table_insert_update_delete_2_update.snap b/etl-destinations/tests/snapshots/deltalake_pipeline__table_insert_update_delete_2_update.snap new file mode 100644 index 000000000..2903ffba3 --- /dev/null +++ b/etl-destinations/tests/snapshots/deltalake_pipeline__table_insert_update_delete_2_update.snap @@ -0,0 +1,15 @@ +--- +source: etl-destinations/tests/deltalake_pipeline.rs +expression: users_table +--- +# Schema +- id: Primitive(Long) nullable=false +- name: Primitive(String) nullable=false +- age: Primitive(Integer) nullable=false + +# Data ++----+---------+-----+ +| id | name | age | ++----+---------+-----+ +| 1 | user_10 | 10 | ++----+---------+-----+ diff --git a/etl-destinations/tests/snapshots/deltalake_pipeline__table_insert_update_delete_3_delete.snap b/etl-destinations/tests/snapshots/deltalake_pipeline__table_insert_update_delete_3_delete.snap new file mode 100644 index 000000000..ebb906b4e --- /dev/null +++ b/etl-destinations/tests/snapshots/deltalake_pipeline__table_insert_update_delete_3_delete.snap @@ -0,0 +1,11 @@ +--- +source: etl-destinations/tests/deltalake_pipeline.rs +expression: users_table +--- +# Schema +- id: Primitive(Long) nullable=false +- name: Primitive(String) nullable=false +- age: Primitive(Integer) nullable=false + +# Data + diff --git a/etl-destinations/tests/snapshots/deltalake_pipeline__table_subsequent_updates_insert.snap b/etl-destinations/tests/snapshots/deltalake_pipeline__table_subsequent_updates_insert.snap new file mode 100644 index 000000000..c7545e6fb --- /dev/null +++ b/etl-destinations/tests/snapshots/deltalake_pipeline__table_subsequent_updates_insert.snap @@ -0,0 +1,15 @@ +--- +source: etl-destinations/tests/deltalake_pipeline.rs +expression: users_table +--- +# Schema +- id: Primitive(Long) nullable=false +- name: Primitive(String) nullable=false +- age: Primitive(Integer) nullable=false + +# Data ++----+--------+-----+ +| id | name | age | ++----+--------+-----+ +| 1 | user_2 | 2 | ++----+--------+-----+ diff --git a/etl-destinations/tests/snapshots/deltalake_pipeline__table_truncate_with_batching_orders_table.snap b/etl-destinations/tests/snapshots/deltalake_pipeline__table_truncate_with_batching_orders_table.snap new file mode 100644 index 000000000..924cd9fd4 --- /dev/null +++ b/etl-destinations/tests/snapshots/deltalake_pipeline__table_truncate_with_batching_orders_table.snap @@ -0,0 +1,17 @@ +--- +source: etl-destinations/tests/deltalake_pipeline.rs +expression: orders_table +--- +# Schema +- id: Primitive(Long) nullable=false +- description: Primitive(String) nullable=false + +# Data ++----+---------------+ +| id | description | ++----+---------------+ +| 1 | description_1 | +| 2 | description_2 | +| 3 | description_3 | +| 4 | description_4 | ++----+---------------+ diff --git a/etl-destinations/tests/snapshots/deltalake_pipeline__table_truncate_with_batching_users_table.snap b/etl-destinations/tests/snapshots/deltalake_pipeline__table_truncate_with_batching_users_table.snap new file mode 100644 index 000000000..e1733c3f7 --- /dev/null +++ b/etl-destinations/tests/snapshots/deltalake_pipeline__table_truncate_with_batching_users_table.snap @@ -0,0 +1,18 @@ +--- +source: etl-destinations/tests/deltalake_pipeline.rs +expression: users_table +--- +# Schema +- id: Primitive(Long) nullable=false +- name: Primitive(String) nullable=false +- age: Primitive(Integer) nullable=false + +# Data ++----+--------+-----+ +| id | name | age | ++----+--------+-----+ +| 1 | user_1 | 1 | +| 2 | user_2 | 2 | +| 3 | user_3 | 3 | +| 4 | user_4 | 4 | ++----+--------+-----+ diff --git a/etl-destinations/tests/snapshots/deltalake_pipeline__test_cdc_deduplication_and_conflict_resolution.snap b/etl-destinations/tests/snapshots/deltalake_pipeline__test_cdc_deduplication_and_conflict_resolution.snap new file mode 100644 index 000000000..ebb906b4e --- /dev/null +++ b/etl-destinations/tests/snapshots/deltalake_pipeline__test_cdc_deduplication_and_conflict_resolution.snap @@ -0,0 +1,11 @@ +--- +source: etl-destinations/tests/deltalake_pipeline.rs +expression: users_table +--- +# Schema +- id: Primitive(Long) nullable=false +- name: Primitive(String) nullable=false +- age: Primitive(Integer) nullable=false + +# Data + diff --git a/etl-destinations/tests/snapshots/deltalake_pipeline__test_large_transaction_batching.snap b/etl-destinations/tests/snapshots/deltalake_pipeline__test_large_transaction_batching.snap new file mode 100644 index 000000000..d929eaf38 --- /dev/null +++ b/etl-destinations/tests/snapshots/deltalake_pipeline__test_large_transaction_batching.snap @@ -0,0 +1,34 @@ +--- +source: etl-destinations/tests/deltalake_pipeline.rs +expression: users_table +--- +# Schema +- id: Primitive(Long) nullable=false +- name: Primitive(String) nullable=false +- age: Primitive(Integer) nullable=false + +# Data ++----+---------------+-----+ +| id | name | age | ++----+---------------+-----+ +| 1 | batch_user_1 | 21 | +| 2 | batch_user_2 | 22 | +| 3 | batch_user_3 | 23 | +| 4 | batch_user_4 | 24 | +| 5 | batch_user_5 | 25 | +| 6 | batch_user_6 | 26 | +| 7 | batch_user_7 | 27 | +| 8 | batch_user_8 | 28 | +| 9 | batch_user_9 | 29 | +| 10 | batch_user_10 | 30 | +| 11 | batch_user_11 | 31 | +| 12 | batch_user_12 | 32 | +| 13 | batch_user_13 | 33 | +| 14 | batch_user_14 | 34 | +| 15 | batch_user_15 | 35 | +| 16 | batch_user_16 | 36 | +| 17 | batch_user_17 | 37 | +| 18 | batch_user_18 | 38 | +| 19 | batch_user_19 | 39 | +| 20 | batch_user_20 | 40 | ++----+---------------+-----+ diff --git a/etl-destinations/tests/snapshots/deltalake_pipeline__upsert_merge_validation.snap b/etl-destinations/tests/snapshots/deltalake_pipeline__upsert_merge_validation.snap new file mode 100644 index 000000000..d5fb75029 --- /dev/null +++ b/etl-destinations/tests/snapshots/deltalake_pipeline__upsert_merge_validation.snap @@ -0,0 +1,15 @@ +--- +source: etl-destinations/tests/deltalake_pipeline.rs +expression: users_table +--- +# Schema +- id: Primitive(Long) nullable=false +- name: Primitive(String) nullable=false +- age: Primitive(Integer) nullable=false + +# Data ++----+-----------------+-----+ +| id | name | age | ++----+-----------------+-----+ +| 1 | snap_user_final | 30 | ++----+-----------------+-----+ diff --git a/etl-destinations/tests/support/deltalake.rs b/etl-destinations/tests/support/deltalake.rs new file mode 100644 index 000000000..02def3993 --- /dev/null +++ b/etl-destinations/tests/support/deltalake.rs @@ -0,0 +1,175 @@ +#![allow(dead_code)] +#![cfg(feature = "deltalake")] + +use deltalake::table::builder::parse_table_uri; +use deltalake::{DeltaResult, DeltaTable, open_table_with_storage_options}; +use etl::store::schema::SchemaStore; +use etl::store::state::StateStore; +use etl::types::TableName; +use etl_destinations::deltalake::{DeltaDestinationConfig, DeltaLakeDestination, DeltaTableConfig}; +use std::collections::HashMap; +use std::env; +use std::sync::Arc; +use uuid::Uuid; + +/// Environment variable name for the minio endpoint URL. +const MINIO_ENDPOINT_ENV_NAME: &str = "TESTS_MINIO_ENDPOINT"; +/// Environment variable name for the minio access key. +const MINIO_ACCESS_KEY_ENV_NAME: &str = "TESTS_MINIO_ACCESS_KEY"; +/// Environment variable name for the minio secret key. +const MINIO_SECRET_KEY_ENV_NAME: &str = "TESTS_MINIO_SECRET_KEY"; +/// Environment variable name for the minio bucket name. +const MINIO_BUCKET_ENV_NAME: &str = "TESTS_MINIO_BUCKET"; + +/// Default values for local development with docker-compose setup +const DEFAULT_MINIO_ENDPOINT: &str = "http://localhost:9010"; +const DEFAULT_MINIO_ACCESS_KEY: &str = "minio-admin"; +const DEFAULT_MINIO_SECRET_KEY: &str = "minio-admin-password"; +const DEFAULT_MINIO_BUCKET: &str = "delta-dev-and-test"; + +/// Generates a unique warehouse path for test isolation. +/// +/// Creates a random warehouse path prefixed with "etl_tests_" to ensure +/// each test run uses a fresh location and avoid conflicts. +fn random_warehouse_path() -> String { + let uuid = Uuid::new_v4().simple().to_string(); + format!("etl_tests_{uuid}") +} + +/// Delta Lake database connection for testing using minio S3-compatible storage. +/// +/// Provides a unified interface for Delta Lake operations in tests, automatically +/// handling setup of test warehouse locations using minio as the object storage backend. +#[allow(unused)] +pub struct MinioDeltaLakeDatabase { + warehouse_path: String, + s3_base_uri: String, + endpoint: String, + access_key: String, + secret_key: String, + bucket: String, +} + +#[allow(unused)] +impl MinioDeltaLakeDatabase { + /// Creates a new Delta Lake database instance. + /// + /// Sets up a [`DeltaLakeDatabase`] that connects to minio S3-compatible storage + /// using either environment variables or default values for local docker-compose setup. + pub async fn new() -> Self { + // Register S3 handlers for Delta Lake + deltalake::aws::register_handlers(None); + let endpoint = env::var(MINIO_ENDPOINT_ENV_NAME) + .unwrap_or_else(|_| DEFAULT_MINIO_ENDPOINT.to_string()); + let access_key = env::var(MINIO_ACCESS_KEY_ENV_NAME) + .unwrap_or_else(|_| DEFAULT_MINIO_ACCESS_KEY.to_string()); + let secret_key = env::var(MINIO_SECRET_KEY_ENV_NAME) + .unwrap_or_else(|_| DEFAULT_MINIO_SECRET_KEY.to_string()); + let bucket = + env::var(MINIO_BUCKET_ENV_NAME).unwrap_or_else(|_| DEFAULT_MINIO_BUCKET.to_string()); + + let warehouse_path = random_warehouse_path(); + let s3_base_uri = format!("s3://{bucket}/{warehouse_path}"); + + Self { + warehouse_path, + s3_base_uri, + endpoint, + access_key, + secret_key, + bucket, + } + } + + /// Creates a [`DeltaLakeDestination`] configured for this database instance. + /// + /// Returns a destination suitable for ETL operations, configured with + /// the test warehouse location and appropriate storage options for MinIO. + pub async fn build_destination(&self, store: S) -> DeltaLakeDestination + where + S: StateStore + SchemaStore + Send + Sync, + { + self.build_destination_with_config(store, HashMap::new()) + .await + } + + /// Creates a [`DeltaLakeDestination`] with a custom per-table configuration map. + pub async fn build_destination_with_config( + &self, + store: S, + table_config: HashMap>, + ) -> DeltaLakeDestination + where + S: StateStore + SchemaStore + Send + Sync, + { + let mut storage_options = HashMap::new(); + storage_options.insert("endpoint".to_string(), self.endpoint.clone()); + storage_options.insert("access_key_id".to_string(), self.access_key.clone()); + storage_options.insert("secret_access_key".to_string(), self.secret_key.clone()); + storage_options.insert("allow_http".to_string(), "true".to_string()); + storage_options.insert( + "virtual_hosted_style_request".to_string(), + "false".to_string(), + ); + + let config = DeltaDestinationConfig { + base_uri: self.s3_base_uri.clone(), + storage_options: Some(storage_options), + table_config, + }; + + DeltaLakeDestination::new(store, config) + } + + pub async fn load_table(&self, table_name: &TableName) -> DeltaResult { + let mut storage_options = HashMap::new(); + storage_options.insert("endpoint".to_string(), self.endpoint.clone()); + storage_options.insert("access_key_id".to_string(), self.access_key.clone()); + storage_options.insert("secret_access_key".to_string(), self.secret_key.clone()); + storage_options.insert("allow_http".to_string(), "true".to_string()); + storage_options.insert( + "virtual_hosted_style_request".to_string(), + "false".to_string(), + ); + + let table = open_table_with_storage_options( + parse_table_uri(format!( + "{}/{}/{}", + self.s3_base_uri, table_name.schema, table_name.name + ))?, + storage_options, + ) + .await?; + Ok(table) + } + + /// Returns the warehouse path for this database instance. + pub fn warehouse_path(&self) -> &str { + &self.warehouse_path + } + + pub fn delete_warehouse(&self) { + // TODO(abhi): Implement cleanup of S3 objects if needed + } + + /// Returns the S3 base URI for this database instance. + pub fn s3_base_uri(&self) -> &str { + &self.s3_base_uri + } +} + +impl Drop for MinioDeltaLakeDatabase { + /// Cleans up the test warehouse when the database instance is dropped. + /// + /// Note: For now, we rely on minio's lifecycle policies or manual cleanup + /// to remove test data. In a production test environment, you might want + /// to implement explicit cleanup here. + fn drop(&mut self) { + self.delete_warehouse(); + } +} + +/// Sets up a Delta Lake database connection for testing. +pub async fn setup_delta_connection() -> MinioDeltaLakeDatabase { + MinioDeltaLakeDatabase::new().await +} diff --git a/etl-destinations/tests/support/lakekeeper.rs b/etl-destinations/tests/support/lakekeeper.rs index 7c0be6e31..dba4a3754 100644 --- a/etl-destinations/tests/support/lakekeeper.rs +++ b/etl-destinations/tests/support/lakekeeper.rs @@ -83,7 +83,7 @@ impl Default for CreateWarehouseRequest { r#type: Type::S3, }, storage_profile: StorageProfile { - bucket: "dev-and-test".to_string(), + bucket: "iceberg-dev-and-test".to_string(), region: "local-01".to_string(), sts_enabled: false, r#type: Type::S3, diff --git a/etl-destinations/tests/support/mod.rs b/etl-destinations/tests/support/mod.rs index 99cac4bcf..09575fa78 100644 --- a/etl-destinations/tests/support/mod.rs +++ b/etl-destinations/tests/support/mod.rs @@ -1,3 +1,2 @@ pub mod bigquery; -pub mod iceberg; -pub mod lakekeeper; +pub mod deltalake; diff --git a/etl-replicator/Cargo.toml b/etl-replicator/Cargo.toml index abf2c75f9..dfaaac3a9 100644 --- a/etl-replicator/Cargo.toml +++ b/etl-replicator/Cargo.toml @@ -10,7 +10,7 @@ homepage.workspace = true [dependencies] etl = { workspace = true, features = ["unknown-types-to-bytes"] } etl-config = { workspace = true } -etl-destinations = { workspace = true, features = ["bigquery", "iceberg"] } +etl-destinations = { workspace = true, features = ["bigquery", "iceberg", "deltalake"] } etl-telemetry = { workspace = true } anyhow = { workspace = true, features = ["std"] } diff --git a/etl-replicator/src/core.rs b/etl-replicator/src/core.rs index 5a1645b66..9e9619e8a 100644 --- a/etl-replicator/src/core.rs +++ b/etl-replicator/src/core.rs @@ -1,3 +1,5 @@ +use std::collections::HashMap; + use crate::migrations::migrate_state_store; use etl::destination::Destination; use etl::destination::memory::MemoryDestination; @@ -10,7 +12,10 @@ use etl::types::PipelineId; use etl_config::shared::{ BatchConfig, DestinationConfig, PgConnectionConfig, PipelineConfig, ReplicatorConfig, }; -use etl_destinations::bigquery::{BigQueryDestination, install_crypto_provider_for_bigquery}; +use etl_destinations::{ + bigquery::{BigQueryDestination, install_crypto_provider_for_bigquery}, + deltalake::{DeltaDestinationConfig, DeltaLakeDestination}, +}; use secrecy::ExposeSecret; use tokio::signal::unix::{SignalKind, signal}; use tracing::{debug, info, warn}; @@ -65,6 +70,25 @@ pub async fn start_replicator_with_config( let pipeline = Pipeline::new(replicator_config.pipeline, state_store, destination); start_pipeline(pipeline).await?; } + DestinationConfig::DeltaLake { + base_uri, + storage_options, + partition_columns: _, + optimize_after_commits: _, + } => { + let destination = DeltaLakeDestination::new( + state_store.clone(), + DeltaDestinationConfig { + base_uri: base_uri.clone(), + storage_options: storage_options.clone(), + table_config: HashMap::new(), + }, + ); + + let pipeline = Pipeline::new(replicator_config.pipeline, state_store, destination); + start_pipeline(pipeline).await?; + } + _ => unimplemented!("destination config not implemented"), } info!("replicator service completed"); @@ -97,6 +121,15 @@ fn log_destination_config(config: &DestinationConfig) { "using bigquery destination config" ) } + DestinationConfig::DeltaLake { + base_uri, + storage_options: _, + partition_columns: _, + optimize_after_commits: _, + } => { + debug!(base_uri = base_uri, "using delta lake destination config"); + } + _ => unimplemented!("destination config not implemented"), } } diff --git a/etl/src/lib.rs b/etl/src/lib.rs index 7c6db6353..150489d65 100644 --- a/etl/src/lib.rs +++ b/etl/src/lib.rs @@ -101,7 +101,7 @@ mod concurrency; pub mod config; -mod conversions; +pub mod conversions; pub mod destination; pub mod error; #[cfg(feature = "failpoints")] diff --git a/scripts/docker-compose.yaml b/scripts/docker-compose.yaml index bc4d07fc0..fe34b1546 100644 --- a/scripts/docker-compose.yaml +++ b/scripts/docker-compose.yaml @@ -73,8 +73,9 @@ services: condition: service_healthy entrypoint: > /bin/sh -c " - mc alias set iceberg http://minio:9000 minio-admin minio-admin-password; - mc mb iceberg/dev-and-test; + mc alias set minio http://minio:9000 minio-admin minio-admin-password; + mc mb minio/iceberg-dev-and-test; + mc mb minio/delta-dev-and-test; exit 0; " diff --git a/scripts/warehouse.json b/scripts/warehouse.json index 5034c44be..d933075b2 100644 --- a/scripts/warehouse.json +++ b/scripts/warehouse.json @@ -1,9 +1,9 @@ { - "warehouse-name": "dev-and-test-warehouse", + "warehouse-name": "iceberg-dev-and-test-warehouse", "project-id": "00000000-0000-0000-0000-000000000000", "storage-profile": { "type": "s3", - "bucket": "dev-and-test", + "bucket": "iceberg-dev-and-test", "key-prefix": "initial-warehouse", "assume-role-arn": null, "endpoint": "http://minio:9000",