diff --git a/.golangci.yml b/.golangci.yml index 571a303e43..1088b199a6 100644 --- a/.golangci.yml +++ b/.golangci.yml @@ -90,6 +90,12 @@ linters: - bodyclose - perfsprint path: _test.go + - linters: + - perfsprint + path: internal/impl/gcp/enterprise/changestreams/changestreamstest + - linters: + - perfsprint + path: internal/impl/gcp/enterprise/changestreams/metadata - linters: - revive text: "exported method .*\\.(Close|Connect|Read|ReadBatch|Write|WriteBatch|Process|ProcessBatch|NextBatch|Create|EndOfInput) should have comment or be unexported" diff --git a/CHANGELOG.md b/CHANGELOG.md index fc3d8b43bb..2d8203560a 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -16,6 +16,7 @@ All notable changes to this project will be documented in this file. - Field `poll_interval` added to the `redpanda_migrator_offsets` input. (@mihaitodor) - Field `consumer_group_offsets_poll_interval` added to the `redpanda_migrator_bundle` input. (@mihaitodor) - Field `input_bundle_label` added to the `redpanda_migrator_bundle` output. (@mihaitodor) +- New `gcp_spanner_cdc` input. (@mmatczuk) ### Fixed diff --git a/docs/modules/components/pages/inputs/gcp_spanner_cdc.adoc b/docs/modules/components/pages/inputs/gcp_spanner_cdc.adoc new file mode 100644 index 0000000000..8e96b3d031 --- /dev/null +++ b/docs/modules/components/pages/inputs/gcp_spanner_cdc.adoc @@ -0,0 +1,332 @@ += gcp_spanner_cdc +:type: input +:status: beta +:categories: ["Services","GCP"] + + + +//// + THIS FILE IS AUTOGENERATED! + + To make changes, edit the corresponding source file under: + + https://github.com/redpanda-data/connect/tree/main/internal/impl/. + + And: + + https://github.com/redpanda-data/connect/tree/main/cmd/tools/docs_gen/templates/plugin.adoc.tmpl +//// + +// © 2024 Redpanda Data Inc. + + +component_type_dropdown::[] + + +Creates an input that consumes from a spanner change stream. + +Introduced in version 4.56.0. + + +[tabs] +====== +Common:: ++ +-- + +```yml +# Common config fields, showing default values +input: + label: "" + gcp_spanner_cdc: + credentials_json: "" + project_id: "" # No default (required) + instance_id: "" # No default (required) + database_id: "" # No default (required) + stream_id: "" # No default (required) + start_timestamp: "" + end_timestamp: "" + batching: + count: 0 + byte_size: 0 + period: "" + check: "" + auto_replay_nacks: true +``` + +-- +Advanced:: ++ +-- + +```yml +# All config fields, showing default values +input: + label: "" + gcp_spanner_cdc: + credentials_json: "" + project_id: "" # No default (required) + instance_id: "" # No default (required) + database_id: "" # No default (required) + stream_id: "" # No default (required) + start_timestamp: "" + end_timestamp: "" + heartbeat_interval: 10s + metadata_table: "" + min_watermark_cache_ttl: 5s + allowed_mod_types: [] # No default (optional) + batching: + count: 0 + byte_size: 0 + period: "" + check: "" + processors: [] # No default (optional) + auto_replay_nacks: true +``` + +-- +====== + +Consumes change records from a Google Cloud Spanner change stream. This input allows +you to track and process database changes in real-time, making it useful for data +replication, event-driven architectures, and maintaining derived data stores. + +The input reads from a specified change stream within a Spanner database and converts +each change record into a message. The message payload contains the change records in +JSON format, and metadata is added with details about the Spanner instance, database, +and stream. + +Change streams provide a way to track mutations to your Spanner database tables. For +more information about Spanner change streams, refer to the Google Cloud documentation: +https://cloud.google.com/spanner/docs/change-streams + + +== Fields + +=== `credentials_json` + +Base64 encoded GCP service account JSON credentials file for authentication. If not provided, Application Default Credentials (ADC) will be used. + + +*Type*: `string` + +*Default*: `""` + +=== `project_id` + +GCP project ID containing the Spanner instance + + +*Type*: `string` + + +=== `instance_id` + +Spanner instance ID + + +*Type*: `string` + + +=== `database_id` + +Spanner database ID + + +*Type*: `string` + + +=== `stream_id` + +The name of the change stream to track, the stream must exist in the database. To create a change stream, see https://cloud.google.com/spanner/docs/change-streams/manage. + + +*Type*: `string` + + +=== `start_timestamp` + +RFC3339 formatted inclusive timestamp to start reading from the change stream (default: current time) + + +*Type*: `string` + +*Default*: `""` + +```yml +# Examples + +start_timestamp: "2022-01-01T00:00:00Z" +``` + +=== `end_timestamp` + +RFC3339 formatted exclusive timestamp to stop reading at (default: no end time) + + +*Type*: `string` + +*Default*: `""` + +```yml +# Examples + +end_timestamp: "2022-01-01T00:00:00Z" +``` + +=== `heartbeat_interval` + +Duration string for heartbeat interval + + +*Type*: `string` + +*Default*: `"10s"` + +=== `metadata_table` + +The table to store metadata in (default: cdc_metadata_) + + +*Type*: `string` + +*Default*: `""` + +=== `min_watermark_cache_ttl` + +Duration string for frequency of querying Spanner for minimum watermark. + + +*Type*: `string` + +*Default*: `"5s"` + +=== `allowed_mod_types` + +List of modification types to process. If not specified, all modification types are processed. Allowed values: INSERT, UPDATE, DELETE + + +*Type*: `array` + + +```yml +# Examples + +allowed_mod_types: + - INSERT + - UPDATE + - DELETE +``` + +=== `batching` + +Allows you to configure a xref:configuration:batching.adoc[batching policy]. + + +*Type*: `object` + + +```yml +# Examples + +batching: + byte_size: 5000 + count: 0 + period: 1s + +batching: + count: 10 + period: 1s + +batching: + check: this.contains("END BATCH") + count: 0 + period: 1m +``` + +=== `batching.count` + +A number of messages at which the batch should be flushed. If `0` disables count based batching. + + +*Type*: `int` + +*Default*: `0` + +=== `batching.byte_size` + +An amount of bytes at which the batch should be flushed. If `0` disables size based batching. + + +*Type*: `int` + +*Default*: `0` + +=== `batching.period` + +A period in which an incomplete batch should be flushed regardless of its size. + + +*Type*: `string` + +*Default*: `""` + +```yml +# Examples + +period: 1s + +period: 1m + +period: 500ms +``` + +=== `batching.check` + +A xref:guides:bloblang/about.adoc[Bloblang query] that should return a boolean value indicating whether a message should end a batch. + + +*Type*: `string` + +*Default*: `""` + +```yml +# Examples + +check: this.type == "end_of_transaction" +``` + +=== `batching.processors` + +A list of xref:components:processors/about.adoc[processors] to apply to a batch as it is flushed. This allows you to aggregate and archive the batch however you see fit. Please note that all resulting messages are flushed as a single batch, therefore splitting the batch into smaller batches using these processors is a no-op. + + +*Type*: `array` + + +```yml +# Examples + +processors: + - archive: + format: concatenate + +processors: + - archive: + format: lines + +processors: + - archive: + format: json_array +``` + +=== `auto_replay_nacks` + +Whether messages that are rejected (nacked) at the output level should be automatically replayed indefinitely, eventually resulting in back pressure if the cause of the rejections is persistent. If set to `false` these messages will instead be deleted. Disabling auto replays can greatly improve memory efficiency of high throughput streams as the original shape of the data can be discarded immediately upon consumption and mutation. + + +*Type*: `bool` + +*Default*: `true` + + diff --git a/go.mod b/go.mod index 6fcc5ceb9b..7a49e04bb5 100644 --- a/go.mod +++ b/go.mod @@ -7,10 +7,11 @@ toolchain go1.24.2 replace github.com/99designs/keyring => github.com/Jeffail/keyring v1.2.3 require ( - cloud.google.com/go/aiplatform v1.74.0 - cloud.google.com/go/bigquery v1.66.2 - cloud.google.com/go/pubsub v1.47.0 - cloud.google.com/go/storage v1.51.0 + cloud.google.com/go/aiplatform v1.85.0 + cloud.google.com/go/bigquery v1.67.0 + cloud.google.com/go/pubsub v1.49.0 + cloud.google.com/go/spanner v1.82.0 + cloud.google.com/go/storage v1.53.0 cloud.google.com/go/vertexai v0.12.0 github.com/Azure/azure-sdk-for-go/sdk/azcore v1.18.0 github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.9.0 @@ -72,7 +73,7 @@ require ( github.com/getsentry/sentry-go v0.31.1 github.com/go-faker/faker/v4 v4.4.2 github.com/go-git/go-git/v5 v5.14.0 - github.com/go-jose/go-jose/v4 v4.0.5 + github.com/go-jose/go-jose/v4 v4.1.0 github.com/go-mysql-org/go-mysql v1.10.0 github.com/go-resty/resty/v2 v2.15.3 github.com/go-sql-driver/mysql v1.8.1 @@ -80,7 +81,7 @@ require ( github.com/gocql/gocql v1.6.0 github.com/gofrs/uuid/v5 v5.3.2 github.com/golang-jwt/jwt/v5 v5.2.2 - github.com/googleapis/go-sql-spanner v1.8.0 + github.com/googleapis/go-sql-spanner v1.13.2 github.com/gosimple/slug v1.14.0 github.com/hamba/avro/v2 v2.28.0 github.com/influxdata/influxdb1-client v0.0.0-20220302092344-a9ab5670611c @@ -161,17 +162,16 @@ require ( golang.org/x/net v0.40.0 golang.org/x/sync v0.14.0 golang.org/x/text v0.25.0 - google.golang.org/api v0.227.0 + google.golang.org/api v0.233.0 google.golang.org/protobuf v1.36.6 modernc.org/sqlite v1.32.0 ) require ( - cel.dev/expr v0.22.1 // indirect - cloud.google.com/go/longrunning v0.6.5 // indirect - cloud.google.com/go/monitoring v1.24.1 // indirect - cloud.google.com/go/secretmanager v1.14.5 // indirect - cloud.google.com/go/spanner v1.76.1 // indirect + cel.dev/expr v0.23.1 // indirect + cloud.google.com/go/longrunning v0.6.7 // indirect + cloud.google.com/go/monitoring v1.24.2 // indirect + cloud.google.com/go/secretmanager v1.14.7 // indirect github.com/Azure/azure-sdk-for-go/sdk/keyvault/azsecrets v0.12.0 // indirect github.com/Azure/azure-sdk-for-go/sdk/keyvault/internal v0.7.1 // indirect github.com/BurntSushi/toml v1.5.0 // indirect @@ -183,7 +183,7 @@ require ( github.com/aws/aws-sdk-go-v2/service/secretsmanager v1.34.3 // indirect github.com/certifi/gocertifi v0.0.0-20210507211836-431795d63e8d // indirect github.com/cloudflare/circl v1.6.0 // indirect - github.com/cncf/xds/go v0.0.0-20250121191232-2f005788dc42 // indirect + github.com/cncf/xds/go v0.0.0-20250326154945-ae57f3c0d45f // indirect github.com/containerd/platforms v1.0.0-rc.1 // indirect github.com/cyphar/filepath-securejoin v0.4.1 // indirect github.com/ebitengine/purego v0.8.4 // indirect @@ -241,12 +241,12 @@ require ( ) require ( - cloud.google.com/go v0.120.0 // indirect - cloud.google.com/go/auth v0.15.0 + cloud.google.com/go v0.121.1 // indirect + cloud.google.com/go/auth v0.16.1 cloud.google.com/go/auth/oauth2adapt v0.2.8 // indirect cloud.google.com/go/compute/metadata v0.6.0 // indirect - cloud.google.com/go/iam v1.4.2 // indirect - cloud.google.com/go/trace v1.11.3 // indirect + cloud.google.com/go/iam v1.5.2 // indirect + cloud.google.com/go/trace v1.11.6 // indirect cuelang.org/go v0.13.0 // indirect dario.cat/mergo v1.0.2 // indirect filippo.io/edwards25519 v1.1.0 // indirect @@ -344,7 +344,7 @@ require ( github.com/google/shlex v0.0.0-20191202100458-e7afc7fbc510 // indirect github.com/google/uuid v1.6.0 github.com/googleapis/enterprise-certificate-proxy v0.3.6 // indirect - github.com/googleapis/gax-go/v2 v2.14.1 // indirect + github.com/googleapis/gax-go/v2 v2.14.2 // indirect github.com/gorilla/css v1.0.1 // indirect github.com/gorilla/handlers v1.5.2 github.com/gorilla/mux v1.8.1 @@ -361,7 +361,7 @@ require ( github.com/hashicorp/go-uuid v1.0.3 // indirect github.com/hashicorp/golang-lru v0.5.4 // indirect github.com/hashicorp/golang-lru/arc/v2 v2.0.7 // indirect - github.com/hashicorp/golang-lru/v2 v2.0.7 // indirect + github.com/hashicorp/golang-lru/v2 v2.0.7 github.com/influxdata/go-syslog/v3 v3.0.0 // indirect github.com/itchyny/gojq v0.12.17 // indirect github.com/itchyny/timefmt-go v0.1.6 // indirect @@ -462,9 +462,9 @@ require ( golang.org/x/tools v0.33.0 // indirect golang.org/x/xerrors v0.0.0-20240903120638-7835f813f4da // indirect google.golang.org/genai v1.7.0 - google.golang.org/genproto v0.0.0-20250324211829-b45e905df463 // indirect - google.golang.org/genproto/googleapis/api v0.0.0-20250324211829-b45e905df463 // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20250324211829-b45e905df463 // indirect + google.golang.org/genproto v0.0.0-20250505200425-f936aa4a68b2 // indirect + google.golang.org/genproto/googleapis/api v0.0.0-20250505200425-f936aa4a68b2 // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20250512202823-5a2f75b736a9 // indirect google.golang.org/grpc v1.72.0 gopkg.in/inf.v0 v0.9.1 // indirect gopkg.in/jcmturner/aescts.v1 v1.0.1 // indirect diff --git a/go.sum b/go.sum index 223334d90e..9b4c51b8f1 100644 --- a/go.sum +++ b/go.sum @@ -1,5 +1,5 @@ -cel.dev/expr v0.22.1 h1:xoFEsNh972Yzey8N9TCPx2nDvMN7TMhQEzxLuj/iRrI= -cel.dev/expr v0.22.1/go.mod h1:MrpN08Q+lEBs+bGYdLxxHkZoUSsCp0nSKTs0nTymJgw= +cel.dev/expr v0.23.1 h1:K4KOtPCJQjVggkARsjG9RWXP6O4R73aHeJMa/dmCQQg= +cel.dev/expr v0.23.1/go.mod h1:hLPLo1W4QUmuYdA72RBX06QTs6MXw941piREPl3Yfiw= cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= cloud.google.com/go v0.34.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= cloud.google.com/go v0.38.0/go.mod h1:990N+gfupTy94rShfmMCWGDn0LpTmnzTp2qbd1dvSRU= @@ -38,8 +38,8 @@ cloud.google.com/go v0.104.0/go.mod h1:OO6xxXdJyvuJPcEPBLN9BJPD+jep5G1+2U5B5gkRY cloud.google.com/go v0.105.0/go.mod h1:PrLgOJNe5nfE9UMxKxgXj4mD3voiP+YQ6gdt6KMFOKM= cloud.google.com/go v0.107.0/go.mod h1:wpc2eNrD7hXUTy8EKS10jkxpZBjASrORK7goS+3YX2I= cloud.google.com/go v0.110.0/go.mod h1:SJnCLqQ0FCFGSZMUNUf84MV3Aia54kn7pi8st7tMzaY= -cloud.google.com/go v0.120.0 h1:wc6bgG9DHyKqF5/vQvX1CiZrtHnxJjBlKUyF9nP6meA= -cloud.google.com/go v0.120.0/go.mod h1:/beW32s8/pGRuj4IILWQNd4uuebeT4dkOhKmkfit64Q= +cloud.google.com/go v0.121.1 h1:S3kTQSydxmu1JfLRLpKtxRPA7rSrYPRPEUmL/PavVUw= +cloud.google.com/go v0.121.1/go.mod h1:nRFlrHq39MNVWu+zESP2PosMWA0ryJw8KUBZ2iZpxbw= cloud.google.com/go/accessapproval v1.4.0/go.mod h1:zybIuC3KpDOvotz59lFe5qxRZx6C75OtwbisN56xYB4= cloud.google.com/go/accessapproval v1.5.0/go.mod h1:HFy3tuiGvMdcd/u+Cu5b9NkO1pEICJ46IR82PoUdplw= cloud.google.com/go/accessapproval v1.6.0/go.mod h1:R0EiYnwV5fsRFiKZkPHr6mwyk2wxUJ30nL4j2pcFY2E= @@ -53,8 +53,8 @@ cloud.google.com/go/aiplatform v1.27.0/go.mod h1:Bvxqtl40l0WImSb04d0hXFU7gDOiq9j cloud.google.com/go/aiplatform v1.35.0/go.mod h1:7MFT/vCaOyZT/4IIFfxH4ErVg/4ku6lKv3w0+tFTgXQ= cloud.google.com/go/aiplatform v1.36.1/go.mod h1:WTm12vJRPARNvJ+v6P52RDHCNe4AhvjcIZ/9/RRHy/k= cloud.google.com/go/aiplatform v1.37.0/go.mod h1:IU2Cv29Lv9oCn/9LkFiiuKfwrRTq+QQMbW+hPCxJGZw= -cloud.google.com/go/aiplatform v1.74.0 h1:rE2P5H7FOAFISAZilmdkapbk4CVgwfVs6FDWlhGfuy0= -cloud.google.com/go/aiplatform v1.74.0/go.mod h1:hVEw30CetNut5FrblYd1AJUWRVSIjoyIvp0EVUh51HA= +cloud.google.com/go/aiplatform v1.85.0 h1:80/GqdP8Tovaaw9Qr6fYZNDvwJeA9rLk8mYkqBJNIJQ= +cloud.google.com/go/aiplatform v1.85.0/go.mod h1:S4DIKz3TFLSt7ooF2aCRdAqsUR4v/YDXUoHqn5P0EFc= cloud.google.com/go/analytics v0.11.0/go.mod h1:DjEWCu41bVbYcKyvlws9Er60YE4a//bK6mnhWvQeFNI= cloud.google.com/go/analytics v0.12.0/go.mod h1:gkfj9h6XRf9+TS4bmuhPEShsh3hH8PAZzm/41OOhQd4= cloud.google.com/go/analytics v0.17.0/go.mod h1:WXFa3WSym4IZ+JiKmavYdJwGG/CvpqiqczmL59bTD9M= @@ -103,8 +103,8 @@ cloud.google.com/go/assuredworkloads v1.7.0/go.mod h1:z/736/oNmtGAyU47reJgGN+KVo cloud.google.com/go/assuredworkloads v1.8.0/go.mod h1:AsX2cqyNCOvEQC8RMPnoc0yEarXQk6WEKkxYfL6kGIo= cloud.google.com/go/assuredworkloads v1.9.0/go.mod h1:kFuI1P78bplYtT77Tb1hi0FMxM0vVpRC7VVoJC3ZoT0= cloud.google.com/go/assuredworkloads v1.10.0/go.mod h1:kwdUQuXcedVdsIaKgKTp9t0UJkE5+PAVNhdQm4ZVq2E= -cloud.google.com/go/auth v0.15.0 h1:Ly0u4aA5vG/fsSsxu98qCQBemXtAtJf+95z9HK+cxps= -cloud.google.com/go/auth v0.15.0/go.mod h1:WJDGqZ1o9E9wKIL+IwStfyn/+s59zl4Bi+1KQNVXLZ8= +cloud.google.com/go/auth v0.16.1 h1:XrXauHMd30LhQYVRHLGvJiYeczweKQXZxsTbV9TiguU= +cloud.google.com/go/auth v0.16.1/go.mod h1:1howDHJ5IETh/LwYs3ZxvlkXF48aSqqJUM+5o02dNOI= cloud.google.com/go/auth/oauth2adapt v0.2.8 h1:keo8NaayQZ6wimpNSmW5OPc283g65QNIiLpZnkHRbnc= cloud.google.com/go/auth/oauth2adapt v0.2.8/go.mod h1:XQ9y31RkqZCcwJWNSx2Xvric3RrU88hAYYbjDWYDL+c= cloud.google.com/go/automl v1.5.0/go.mod h1:34EjfoFGMZ5sgJ9EoLsRtdPSNZLcfflJR39VbVNS2M0= @@ -135,8 +135,8 @@ cloud.google.com/go/bigquery v1.47.0/go.mod h1:sA9XOgy0A8vQK9+MWhEQTY6Tix87M/Zur cloud.google.com/go/bigquery v1.48.0/go.mod h1:QAwSz+ipNgfL5jxiaK7weyOhzdoAy1zFm0Nf1fysJac= cloud.google.com/go/bigquery v1.49.0/go.mod h1:Sv8hMmTFFYBlt/ftw2uN6dFdQPzBlREY9yBh7Oy7/4Q= cloud.google.com/go/bigquery v1.50.0/go.mod h1:YrleYEh2pSEbgTBZYMJ5SuSr0ML3ypjRB1zgf7pvQLU= -cloud.google.com/go/bigquery v1.66.2 h1:EKOSqjtO7jPpJoEzDmRctGea3c2EOGoexy8VyY9dNro= -cloud.google.com/go/bigquery v1.66.2/go.mod h1:+Yd6dRyW8D/FYEjUGodIbu0QaoEmgav7Lwhotup6njo= +cloud.google.com/go/bigquery v1.67.0 h1:GXleMyn/cu5+DPLy9Rz5f5IULWTLrepwbQnP/5qrVbY= +cloud.google.com/go/bigquery v1.67.0/go.mod h1:HQeP1AHFuAz0Y55heDSb0cjZIhnEkuwFRBGo6EEKHug= cloud.google.com/go/billing v1.4.0/go.mod h1:g9IdKBEFlItS8bTtlrZdVLWSSdSyFUZKXNS02zKMOZY= cloud.google.com/go/billing v1.5.0/go.mod h1:mztb1tBc3QekhjSgmpf/CV4LzWXLzCArwpLmP2Gm88s= cloud.google.com/go/billing v1.6.0/go.mod h1:WoXzguj+BeHXPbKfNWkqVtDdzORazmCjraY+vrxcyvI= @@ -210,8 +210,8 @@ cloud.google.com/go/datacatalog v1.8.0/go.mod h1:KYuoVOv9BM8EYz/4eMFxrr4DUKhGIOX cloud.google.com/go/datacatalog v1.8.1/go.mod h1:RJ58z4rMp3gvETA465Vg+ag8BGgBdnRPEMMSTr5Uv+M= cloud.google.com/go/datacatalog v1.12.0/go.mod h1:CWae8rFkfp6LzLumKOnmVh4+Zle4A3NXLzVJ1d1mRm0= cloud.google.com/go/datacatalog v1.13.0/go.mod h1:E4Rj9a5ZtAxcQJlEBTLgMTphfP11/lNaAshpoBgemX8= -cloud.google.com/go/datacatalog v1.24.3 h1:3bAfstDB6rlHyK0TvqxEwaeOvoN9UgCs2bn03+VXmss= -cloud.google.com/go/datacatalog v1.24.3/go.mod h1:Z4g33XblDxWGHngDzcpfeOU0b1ERlDPTuQoYG6NkF1s= +cloud.google.com/go/datacatalog v1.26.0 h1:eFgygb3DTufTWWUB8ARk+dSuXz+aefNJXTlkWlQcWwE= +cloud.google.com/go/datacatalog v1.26.0/go.mod h1:bLN2HLBAwB3kLTFT5ZKLHVPj/weNz6bR0c7nYp0LE14= cloud.google.com/go/dataflow v0.6.0/go.mod h1:9QwV89cGoxjjSR9/r7eFDqqjtvbKxAK2BaYU6PVk9UM= cloud.google.com/go/dataflow v0.7.0/go.mod h1:PX526vb4ijFMesO1o202EaUmouZKBpjHsTlCtB4parQ= cloud.google.com/go/dataflow v0.8.0/go.mod h1:Rcf5YgTKPtQyYz8bLYhFoIV/vP39eL7fWNcSOyFfLJE= @@ -325,8 +325,8 @@ cloud.google.com/go/iam v0.8.0/go.mod h1:lga0/y3iH6CX7sYqypWJ33hf7kkfXJag67naqGE cloud.google.com/go/iam v0.11.0/go.mod h1:9PiLDanza5D+oWFZiH1uG+RnRCfEGKoyl6yo4cgWZGY= cloud.google.com/go/iam v0.12.0/go.mod h1:knyHGviacl11zrtZUoDuYpDgLjvr28sLQaG0YB2GYAY= cloud.google.com/go/iam v0.13.0/go.mod h1:ljOg+rcNfzZ5d6f1nAUJ8ZIxOaZUVoS14bKCtaLZ/D0= -cloud.google.com/go/iam v1.4.2 h1:4AckGYAYsowXeHzsn/LCKWIwSWLkdb0eGjH8wWkd27Q= -cloud.google.com/go/iam v1.4.2/go.mod h1:REGlrt8vSlh4dfCJfSEcNjLGq75wW75c5aU3FLOYq34= +cloud.google.com/go/iam v1.5.2 h1:qgFRAGEmd8z6dJ/qyEchAuL9jpswyODjA2lS+w234g8= +cloud.google.com/go/iam v1.5.2/go.mod h1:SE1vg0N81zQqLzQEwxL2WI6yhetBdbNQuTvIKCSkUHE= cloud.google.com/go/iap v1.4.0/go.mod h1:RGFwRJdihTINIe4wZ2iCP0zF/qu18ZwyKxrhMhygBEc= cloud.google.com/go/iap v1.5.0/go.mod h1:UH/CGgKd4KyohZL5Pt0jSKE4m3FR51qg6FKQ/z/Ix9A= cloud.google.com/go/iap v1.6.0/go.mod h1:NSuvI9C/j7UdjGjIde7t7HBz+QTwBcapPE07+sSRcLk= @@ -346,8 +346,8 @@ cloud.google.com/go/kms v1.8.0/go.mod h1:4xFEhYFqvW+4VMELtZyxomGSYtSQKzM178ylFW4 cloud.google.com/go/kms v1.9.0/go.mod h1:qb1tPTgfF9RQP8e1wq4cLFErVuTJv7UsSC915J8dh3w= cloud.google.com/go/kms v1.10.0/go.mod h1:ng3KTUtQQU9bPX3+QGLsflZIHlkbn8amFAMY63m8d24= cloud.google.com/go/kms v1.10.1/go.mod h1:rIWk/TryCkR59GMC3YtHtXeLzd634lBbKenvyySAyYI= -cloud.google.com/go/kms v1.21.0 h1:x3EeWKuYwdlo2HLse/876ZrKjk2L5r7Uexfm8+p6mSI= -cloud.google.com/go/kms v1.21.0/go.mod h1:zoFXMhVVK7lQ3JC9xmhHMoQhnjEDZFoLAr5YMwzBLtk= +cloud.google.com/go/kms v1.21.2 h1:c/PRUSMNQ8zXrc1sdAUnsenWWaNXN+PzTXfXOcSFdoE= +cloud.google.com/go/kms v1.21.2/go.mod h1:8wkMtHV/9Z8mLXEXr1GK7xPSBdi6knuLXIhqjuWcI6w= cloud.google.com/go/language v1.4.0/go.mod h1:F9dRpNFQmJbkaop6g0JhSBXCNlO90e1KWx5iDdxbWic= cloud.google.com/go/language v1.6.0/go.mod h1:6dJ8t3B+lUYfStgls25GusK04NLh3eDLQnWM3mdEbhI= cloud.google.com/go/language v1.7.0/go.mod h1:DJ6dYN/W+SQOjF8e1hLQXMF21AkH2w9wiPzPCJa2MIE= @@ -363,8 +363,8 @@ cloud.google.com/go/logging v1.13.0/go.mod h1:36CoKh6KA/M0PbhPKMq6/qety2DCAErbhX cloud.google.com/go/longrunning v0.1.1/go.mod h1:UUFxuDWkv22EuY93jjmDMFT5GPQKeFVJBIF6QlTqdsE= cloud.google.com/go/longrunning v0.3.0/go.mod h1:qth9Y41RRSUE69rDcOn6DdK3HfQfsUI0YSmW3iIlLJc= cloud.google.com/go/longrunning v0.4.1/go.mod h1:4iWDqhBZ70CvZ6BfETbvam3T8FMvLK+eFj0E6AaRQTo= -cloud.google.com/go/longrunning v0.6.5 h1:sD+t8DO8j4HKW4QfouCklg7ZC1qC4uzVZt8iz3uTW+Q= -cloud.google.com/go/longrunning v0.6.5/go.mod h1:Et04XK+0TTLKa5IPYryKf5DkpwImy6TluQ1QTLwlKmY= +cloud.google.com/go/longrunning v0.6.7 h1:IGtfDWHhQCgCjwQjV9iiLnUta9LBCo8R9QmAFsS/PrE= +cloud.google.com/go/longrunning v0.6.7/go.mod h1:EAFV3IZAKmM56TyiE6VAP3VoTzhZzySwI/YI1s/nRsY= cloud.google.com/go/managedidentities v1.3.0/go.mod h1:UzlW3cBOiPrzucO5qWkNkh0w33KFtBJU281hacNvsdE= cloud.google.com/go/managedidentities v1.4.0/go.mod h1:NWSBYbEMgqmbZsLIyKvxrYbtqOsxY1ZrGM+9RgDqInM= cloud.google.com/go/managedidentities v1.5.0/go.mod h1:+dWcZ0JlUmpuxpIDfyP5pP5y0bLdRwOS4Lp7gMni/LA= @@ -388,8 +388,8 @@ cloud.google.com/go/monitoring v1.7.0/go.mod h1:HpYse6kkGo//7p6sT0wsIC6IBDET0RhI cloud.google.com/go/monitoring v1.8.0/go.mod h1:E7PtoMJ1kQXWxPjB6mv2fhC5/15jInuulFdYYtlcvT4= cloud.google.com/go/monitoring v1.12.0/go.mod h1:yx8Jj2fZNEkL/GYZyTLS4ZtZEZN8WtDEiEqG4kLK50w= cloud.google.com/go/monitoring v1.13.0/go.mod h1:k2yMBAB1H9JT/QETjNkgdCGD9bPF712XiLTVr+cBrpw= -cloud.google.com/go/monitoring v1.24.1 h1:vKiypZVFD/5a3BbQMvI4gZdl8445ITzXFh257XBgrS0= -cloud.google.com/go/monitoring v1.24.1/go.mod h1:Z05d1/vn9NaujqY2voG6pVQXoJGbp+r3laV+LySt9K0= +cloud.google.com/go/monitoring v1.24.2 h1:5OTsoJ1dXYIiMiuL+sYscLc9BumrL3CarVLL7dd7lHM= +cloud.google.com/go/monitoring v1.24.2/go.mod h1:x7yzPWcgDRnPEv3sI+jJGBkwl5qINf+6qY4eq0I9B4U= cloud.google.com/go/networkconnectivity v1.4.0/go.mod h1:nOl7YL8odKyAOtzNX73/M5/mGZgqqMeryi6UPZTk/rA= cloud.google.com/go/networkconnectivity v1.5.0/go.mod h1:3GzqJx7uhtlM3kln0+x5wyFvuVH1pIBJjhCpjzSt75o= cloud.google.com/go/networkconnectivity v1.6.0/go.mod h1:OJOoEXW+0LAxHh89nXd64uGG+FbQoeH8DtxCHVOMlaM= @@ -447,8 +447,8 @@ cloud.google.com/go/pubsub v1.26.0/go.mod h1:QgBH3U/jdJy/ftjPhTkyXNj543Tin1pRYcd cloud.google.com/go/pubsub v1.27.1/go.mod h1:hQN39ymbV9geqBnfQq6Xf63yNhUAhv9CZhzp5O6qsW0= cloud.google.com/go/pubsub v1.28.0/go.mod h1:vuXFpwaVoIPQMGXqRyUQigu/AX1S3IWugR9xznmcXX8= cloud.google.com/go/pubsub v1.30.0/go.mod h1:qWi1OPS0B+b5L+Sg6Gmc9zD1Y+HaM0MdUr7LsupY1P4= -cloud.google.com/go/pubsub v1.47.0 h1:Ou2Qu4INnf7ykrFjGv2ntFOjVo8Nloh/+OffF4mUu9w= -cloud.google.com/go/pubsub v1.47.0/go.mod h1:LaENesmga+2u0nDtLkIOILskxsfvn/BXX9Ak1NFxOs8= +cloud.google.com/go/pubsub v1.49.0 h1:5054IkbslnrMCgA2MAEPcsN3Ky+AyMpEZcii/DoySPo= +cloud.google.com/go/pubsub v1.49.0/go.mod h1:K1FswTWP+C1tI/nfi3HQecoVeFvL4HUOB1tdaNXKhUY= cloud.google.com/go/pubsublite v1.5.0/go.mod h1:xapqNQ1CuLfGi23Yda/9l4bBCKz/wC3KIJ5gKcxveZg= cloud.google.com/go/pubsublite v1.6.0/go.mod h1:1eFCS0U11xlOuMFV/0iBqw3zP12kddMeCbj/F3FSj9k= cloud.google.com/go/pubsublite v1.7.0/go.mod h1:8hVMwRXfDfvGm3fahVbtDbiLePT3gpoiJYJY+vxWxVM= @@ -500,8 +500,8 @@ cloud.google.com/go/secretmanager v1.6.0/go.mod h1:awVa/OXF6IiyaU1wQ34inzQNc4ISI cloud.google.com/go/secretmanager v1.8.0/go.mod h1:hnVgi/bN5MYHd3Gt0SPuTPPp5ENina1/LxM+2W9U9J4= cloud.google.com/go/secretmanager v1.9.0/go.mod h1:b71qH2l1yHmWQHt9LC80akm86mX8AL6X1MA01dW8ht4= cloud.google.com/go/secretmanager v1.10.0/go.mod h1:MfnrdvKMPNra9aZtQFvBcvRU54hbPD8/HayQdlUgJpU= -cloud.google.com/go/secretmanager v1.14.5 h1:W++V0EL9iL6T2+ec24Dm++bIti0tI6Gx6sCosDBters= -cloud.google.com/go/secretmanager v1.14.5/go.mod h1:GXznZF3qqPZDGZQqETZwZqHw4R6KCaYVvcGiRBA+aqY= +cloud.google.com/go/secretmanager v1.14.7 h1:VkscIRzj7GcmZyO4z9y1EH7Xf81PcoiAo7MtlD+0O80= +cloud.google.com/go/secretmanager v1.14.7/go.mod h1:uRuB4F6NTFbg0vLQ6HsT7PSsfbY7FqHbtJP1J94qxGc= cloud.google.com/go/security v1.5.0/go.mod h1:lgxGdyOKKjHL4YG3/YwIL2zLqMFCKs0UbQwgyZmfJl4= cloud.google.com/go/security v1.7.0/go.mod h1:mZklORHl6Bg7CNnnjLH//0UlAlaXqiG7Lb9PsPXLfD0= cloud.google.com/go/security v1.8.0/go.mod h1:hAQOwgmaHhztFhiQ41CjDODdWP0+AE1B3sX4OFlq+GU= @@ -540,8 +540,8 @@ cloud.google.com/go/shell v1.6.0/go.mod h1:oHO8QACS90luWgxP3N9iZVuEiSF84zNyLytb+ cloud.google.com/go/spanner v1.41.0/go.mod h1:MLYDBJR/dY4Wt7ZaMIQ7rXOTLjYrmxLE/5ve9vFfWos= cloud.google.com/go/spanner v1.44.0/go.mod h1:G8XIgYdOK+Fbcpbs7p2fiprDw4CaZX63whnSMLVBxjk= cloud.google.com/go/spanner v1.45.0/go.mod h1:FIws5LowYz8YAE1J8fOS7DJup8ff7xJeetWEo5REA2M= -cloud.google.com/go/spanner v1.76.1 h1:vYbVZuXfnFwvNcvH3lhI2PeUA+kHyqKmLC7mJWaC4Ok= -cloud.google.com/go/spanner v1.76.1/go.mod h1:YtwoE+zObKY7+ZeDCBtZ2ukM+1/iPaMfUM+KnTh/sx0= +cloud.google.com/go/spanner v1.82.0 h1:w9uO8RqEoBooBLX4nqV1RtgudyU2ZX780KTLRgeVg60= +cloud.google.com/go/spanner v1.82.0/go.mod h1:BzybQHFQ/NqGxvE/M+/iU29xgutJf7Q85/4U9RWMto0= cloud.google.com/go/speech v1.6.0/go.mod h1:79tcr4FHCimOp56lwC01xnt/WPJZc4v3gzyT7FoBkCM= cloud.google.com/go/speech v1.7.0/go.mod h1:KptqL+BAQIhMsj1kOP2la5DSEEerPDuOP/2mmkhHhZQ= cloud.google.com/go/speech v1.8.0/go.mod h1:9bYIl1/tjsAnMgKGHKmBZzXKEkGgtU+MpdDPTE9f7y0= @@ -559,8 +559,8 @@ cloud.google.com/go/storage v1.23.0/go.mod h1:vOEEDNFnciUMhBeT6hsJIn3ieU5cFRmzeL cloud.google.com/go/storage v1.27.0/go.mod h1:x9DOL8TK/ygDUMieqwfhdpQryTeEkhGKMi80i/iqR2s= cloud.google.com/go/storage v1.28.1/go.mod h1:Qnisd4CqDdo6BGs2AD5LLnEsmSQ80wQ5ogcBBKhU86Y= cloud.google.com/go/storage v1.29.0/go.mod h1:4puEjyTKnku6gfKoTfNOU/W+a9JyuVNxjpS5GBrB8h4= -cloud.google.com/go/storage v1.51.0 h1:ZVZ11zCiD7b3k+cH5lQs/qcNaoSz3U9I0jgwVzqDlCw= -cloud.google.com/go/storage v1.51.0/go.mod h1:YEJfu/Ki3i5oHC/7jyTgsGZwdQ8P9hqMqvpi5kRKGgc= +cloud.google.com/go/storage v1.53.0 h1:gg0ERZwL17pJ+Cz3cD2qS60w1WMDnwcm5YPAIQBHUAw= +cloud.google.com/go/storage v1.53.0/go.mod h1:7/eO2a/srr9ImZW9k5uufcNahT2+fPb8w5it1i5boaA= cloud.google.com/go/storagetransfer v1.5.0/go.mod h1:dxNzUopWy7RQevYFHewchb29POFv3/AaBgnhqzqiK0w= cloud.google.com/go/storagetransfer v1.6.0/go.mod h1:y77xm4CQV/ZhFZH75PLEXY0ROiS7Gh6pSKrM8dJyg6I= cloud.google.com/go/storagetransfer v1.7.0/go.mod h1:8Giuj1QNb1kfLAiWM1bN6dHzfdlDAVC9rv9abHot2W4= @@ -580,8 +580,8 @@ cloud.google.com/go/trace v1.3.0/go.mod h1:FFUE83d9Ca57C+K8rDl/Ih8LwOzWIV1krKgxg cloud.google.com/go/trace v1.4.0/go.mod h1:UG0v8UBqzusp+z63o7FK74SdFE+AXpCLdFb1rshXG+Y= cloud.google.com/go/trace v1.8.0/go.mod h1:zH7vcsbAhklH8hWFig58HvxcxyQbaIqMarMg9hn5ECA= cloud.google.com/go/trace v1.9.0/go.mod h1:lOQqpE5IaWY0Ixg7/r2SjixMuc6lfTFeO4QGM4dQWOk= -cloud.google.com/go/trace v1.11.3 h1:c+I4YFjxRQjvAhRmSsmjpASUKq88chOX854ied0K/pE= -cloud.google.com/go/trace v1.11.3/go.mod h1:pt7zCYiDSQjC9Y2oqCsh9jF4GStB/hmjrYLsxRR27q8= +cloud.google.com/go/trace v1.11.6 h1:2O2zjPzqPYAHrn3OKl029qlqG6W8ZdYaOWRyr8NgMT4= +cloud.google.com/go/trace v1.11.6/go.mod h1:GA855OeDEBiBMzcckLPE2kDunIpC72N+Pq8WFieFjnI= cloud.google.com/go/translate v1.3.0/go.mod h1:gzMUwRjvOqj5i69y/LYLd8RrNQk+hOmIXTi9+nb3Djs= cloud.google.com/go/translate v1.4.0/go.mod h1:06Dn/ppvLD6WvA5Rhdp029IX2Mi3Mn7fpMRLPvXT5Wg= cloud.google.com/go/translate v1.5.0/go.mod h1:29YDSYveqqpA1CQFD7NQuP49xymq17RXNaUDdc0mNu0= @@ -970,8 +970,8 @@ github.com/cncf/xds/go v0.0.0-20211130200136-a8f946100490/go.mod h1:eXthEFrGJvWH github.com/cncf/xds/go v0.0.0-20220314180256-7f1daf1720fc/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= github.com/cncf/xds/go v0.0.0-20230105202645-06c439db220b/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= github.com/cncf/xds/go v0.0.0-20230607035331-e9ce68804cb4/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= -github.com/cncf/xds/go v0.0.0-20250121191232-2f005788dc42 h1:Om6kYQYDUk5wWbT0t0q6pvyM49i9XZAv9dDrkDA7gjk= -github.com/cncf/xds/go v0.0.0-20250121191232-2f005788dc42/go.mod h1:W+zGtBO5Y1IgJhy4+A9GOqVhqLpfZi+vwmdNXUehLA8= +github.com/cncf/xds/go v0.0.0-20250326154945-ae57f3c0d45f h1:C5bqEmzEPLsHm9Mv73lSE9e9bKV23aB1vxOsmZrkl3k= +github.com/cncf/xds/go v0.0.0-20250326154945-ae57f3c0d45f/go.mod h1:W+zGtBO5Y1IgJhy4+A9GOqVhqLpfZi+vwmdNXUehLA8= github.com/cockroachdb/apd v1.1.0 h1:3LFP3629v+1aKXU5Q37mxmRxX/pIu1nijXydLShEq5I= github.com/cockroachdb/apd v1.1.0/go.mod h1:8Sl8LxpKi29FqWXR16WEFZRNSz3SoPzUzeMeY4+DwBQ= github.com/cockroachdb/apd/v3 v3.2.1 h1:U+8j7t0axsIgvQUqthuNm82HIrYXodOV2iWLWtEaIwg= @@ -1141,8 +1141,8 @@ github.com/go-git/go-git/v5 v5.14.0/go.mod h1:Z5Xhoia5PcWA3NF8vRLURn9E5FRhSl7dGj github.com/go-gl/glfw v0.0.0-20190409004039-e6da0acd62b1/go.mod h1:vR7hzQXu2zJy9AVAgeJqvqgH9Q5CA+iKCZ2gyEVpxRU= github.com/go-gl/glfw/v3.3/glfw v0.0.0-20191125211704-12ad95a8df72/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= github.com/go-gl/glfw/v3.3/glfw v0.0.0-20200222043503-6f7a984d4dc4/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= -github.com/go-jose/go-jose/v4 v4.0.5 h1:M6T8+mKZl/+fNNuFHvGIzDz7BTLQPIounk/b9dw3AaE= -github.com/go-jose/go-jose/v4 v4.0.5/go.mod h1:s3P1lRrkT8igV8D9OjyL4WRyHvjB6a4JSllnOrmmBOA= +github.com/go-jose/go-jose/v4 v4.1.0 h1:cYSYxd3pw5zd2FSXk2vGdn9igQU2PS8MuxrCOCl0FdY= +github.com/go-jose/go-jose/v4 v4.1.0/go.mod h1:GG/vqmYm3Von2nYiB2vGTXzdoNKE5tix5tuc6iAd+sw= github.com/go-kit/kit v0.8.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= github.com/go-kit/kit v0.9.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= github.com/go-kit/log v0.1.0/go.mod h1:zbhenjAZHb184qTLMA9ZjW7ThYL0H2mk7Q6pNt4vbaY= @@ -1331,10 +1331,10 @@ github.com/googleapis/gax-go/v2 v2.5.1/go.mod h1:h6B0KMMFNtI2ddbGJn3T3ZbwkeT6yqE github.com/googleapis/gax-go/v2 v2.6.0/go.mod h1:1mjbznJAPHFpesgE5ucqfYEscaz5kMdcIDwU/6+DDoY= github.com/googleapis/gax-go/v2 v2.7.0/go.mod h1:TEop28CZZQ2y+c0VxMUmu1lV+fQx57QpBWsYpwqHJx8= github.com/googleapis/gax-go/v2 v2.7.1/go.mod h1:4orTrqY6hXxxaUL4LHIPl6lGo8vAE38/qKbhSAKP6QI= -github.com/googleapis/gax-go/v2 v2.14.1 h1:hb0FFeiPaQskmvakKu5EbCbpntQn48jyHuvrkurSS/Q= -github.com/googleapis/gax-go/v2 v2.14.1/go.mod h1:Hb/NubMaVM88SrNkvl8X/o8XWwDJEPqouaLeN2IUxoA= -github.com/googleapis/go-sql-spanner v1.8.0 h1:qAokLb3dhmPA66hGrMRrr3AGFak/6QyjdPwsgJHbfy8= -github.com/googleapis/go-sql-spanner v1.8.0/go.mod h1:Jz+J6AfJsHzfC7SDKxAZH1oe7i0vMrU/ryAFBTaM1jw= +github.com/googleapis/gax-go/v2 v2.14.2 h1:eBLnkZ9635krYIPD+ag1USrOAI0Nr0QYF3+/3GqO0k0= +github.com/googleapis/gax-go/v2 v2.14.2/go.mod h1:ON64QhlJkhVtSqp4v1uaK92VyZ2gmvDQsweuyLV+8+w= +github.com/googleapis/go-sql-spanner v1.13.2 h1:iZ0tSXG4Uad1mDHc3Pz87S2qcLlDFzhrJY+YsKvPhfQ= +github.com/googleapis/go-sql-spanner v1.13.2/go.mod h1:39INzBaBeBGdjUupHk1Be4utiSTXbfiY8lF/dymE8Uc= github.com/googleapis/go-type-adapters v1.0.0/go.mod h1:zHW75FOG2aur7gAO2B+MLby+cLsWGBF62rFAi7WjWO4= github.com/googleapis/google-cloud-go-testing v0.0.0-20200911160855-bcd43fbb19e8/go.mod h1:dvDLG8qkwmyD9a/MJJN3XJcT3xFxOKAvTZGvuZmac9g= github.com/gorilla/css v1.0.1 h1:ntNaBIghp6JmvWnxbZKANoLyuXTPZ4cAMlo6RyhlbO8= @@ -2083,8 +2083,8 @@ go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracegrpc v1.28.0 h1:R3X6Z go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracegrpc v1.28.0/go.mod h1:QWFXnDavXWwMx2EEcZsf3yxgEKAqsxQ+Syjp+seyInw= go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracehttp v1.31.0 h1:lUsI2TYsQw2r1IASwoROaCnjdj2cvC2+Jbxvk6nHnWU= go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracehttp v1.31.0/go.mod h1:2HpZxxQurfGxJlJDblybejHB6RX6pmExPNe517hREw4= -go.opentelemetry.io/otel/exporters/stdout/stdoutmetric v1.29.0 h1:WDdP9acbMYjbKIyJUhTvtzj601sVJOqgWdUxSdR/Ysc= -go.opentelemetry.io/otel/exporters/stdout/stdoutmetric v1.29.0/go.mod h1:BLbf7zbNIONBLPwvFnwNHGj4zge8uTCM/UPIVW1Mq2I= +go.opentelemetry.io/otel/exporters/stdout/stdoutmetric v1.35.0 h1:PB3Zrjs1sG1GBX51SXyTSoOTqcDglmsk7nT6tkKPb/k= +go.opentelemetry.io/otel/exporters/stdout/stdoutmetric v1.35.0/go.mod h1:U2R3XyVPzn0WX7wOIypPuptulsMcPDPs/oiSVOMVnHY= go.opentelemetry.io/otel/metric v1.36.0 h1:MoWPKVhQvJ+eeXWHFBOPoBOi20jh6Iq2CcCREuTYufE= go.opentelemetry.io/otel/metric v1.36.0/go.mod h1:zC7Ks+yeyJt4xig9DEw9kuUFe5C3zLbVjV2PzT6qzbs= go.opentelemetry.io/otel/sdk v1.36.0 h1:b6SYIuLRs88ztox4EyrvRti80uXIFy+Sqzoh9kFULbs= @@ -2629,8 +2629,8 @@ google.golang.org/api v0.108.0/go.mod h1:2Ts0XTHNVWxypznxWOYUeI4g3WdP9Pk2Qk58+a/ google.golang.org/api v0.110.0/go.mod h1:7FC4Vvx1Mooxh8C5HWjzZHcavuS2f6pmJpZx60ca7iI= google.golang.org/api v0.111.0/go.mod h1:qtFHvU9mhgTJegR31csQ+rwxyUTHOKFqCKWp1J0fdw0= google.golang.org/api v0.114.0/go.mod h1:ifYI2ZsFK6/uGddGfAD5BMxlnkBqCmqHSDUVi45N5Yg= -google.golang.org/api v0.227.0 h1:QvIHF9IuyG6d6ReE+BNd11kIB8hZvjN8Z5xY5t21zYc= -google.golang.org/api v0.227.0/go.mod h1:EIpaG6MbTgQarWF5xJvX0eOJPK9n/5D4Bynb9j2HXvQ= +google.golang.org/api v0.233.0 h1:iGZfjXAJiUFSSaekVB7LzXl6tRfEKhUN7FkZN++07tI= +google.golang.org/api v0.233.0/go.mod h1:TCIVLLlcwunlMpZIhIp7Ltk77W+vUSdUKAAIlbxY44c= google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= google.golang.org/appengine v1.5.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= @@ -2774,12 +2774,12 @@ google.golang.org/genproto v0.0.0-20230323212658-478b75c54725/go.mod h1:UUQDJDOl google.golang.org/genproto v0.0.0-20230330154414-c0448cd141ea/go.mod h1:UUQDJDOlWu4KYeJZffbWgBkS1YFobzKbLVfK69pe0Ak= google.golang.org/genproto v0.0.0-20230331144136-dcfb400f0633/go.mod h1:UUQDJDOlWu4KYeJZffbWgBkS1YFobzKbLVfK69pe0Ak= google.golang.org/genproto v0.0.0-20230410155749-daa745c078e1/go.mod h1:nKE/iIaLqn2bQwXBg8f1g2Ylh6r5MN5CmZvuzZCgsCU= -google.golang.org/genproto v0.0.0-20250324211829-b45e905df463 h1:qEFnJI6AnfZk0NNe8YTyXQh5i//Zxi4gBHwRgp76qpw= -google.golang.org/genproto v0.0.0-20250324211829-b45e905df463/go.mod h1:SqIx1NV9hcvqdLHo7uNZDS5lrUJybQ3evo3+z/WBfA0= -google.golang.org/genproto/googleapis/api v0.0.0-20250324211829-b45e905df463 h1:hE3bRWtU6uceqlh4fhrSnUyjKHMKB9KrTLLG+bc0ddM= -google.golang.org/genproto/googleapis/api v0.0.0-20250324211829-b45e905df463/go.mod h1:U90ffi8eUL9MwPcrJylN5+Mk2v3vuPDptd5yyNUiRR8= -google.golang.org/genproto/googleapis/rpc v0.0.0-20250324211829-b45e905df463 h1:e0AIkUUhxyBKh6ssZNrAMeqhA7RKUj42346d1y02i2g= -google.golang.org/genproto/googleapis/rpc v0.0.0-20250324211829-b45e905df463/go.mod h1:qQ0YXyHHx3XkvlzUtpXDkS29lDSafHMZBAZDc03LQ3A= +google.golang.org/genproto v0.0.0-20250505200425-f936aa4a68b2 h1:1tXaIXCracvtsRxSBsYDiSBN0cuJvM7QYW+MrpIRY78= +google.golang.org/genproto v0.0.0-20250505200425-f936aa4a68b2/go.mod h1:49MsLSx0oWMOZqcpB3uL8ZOkAh1+TndpJ8ONoCBWiZk= +google.golang.org/genproto/googleapis/api v0.0.0-20250505200425-f936aa4a68b2 h1:vPV0tzlsK6EzEDHNNH5sa7Hs9bd7iXR7B1tSiPepkV0= +google.golang.org/genproto/googleapis/api v0.0.0-20250505200425-f936aa4a68b2/go.mod h1:pKLAc5OolXC3ViWGI62vvC0n10CpwAtRcTNCFwTKBEw= +google.golang.org/genproto/googleapis/rpc v0.0.0-20250512202823-5a2f75b736a9 h1:IkAfh6J/yllPtpYFU0zZN1hUPYdT0ogkBT/9hMxHjvg= +google.golang.org/genproto/googleapis/rpc v0.0.0-20250512202823-5a2f75b736a9/go.mod h1:qQ0YXyHHx3XkvlzUtpXDkS29lDSafHMZBAZDc03LQ3A= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= google.golang.org/grpc v1.20.1/go.mod h1:10oTOabMzJvdu6/UiuZezV6QK5dSlG84ov/aaiqXj38= google.golang.org/grpc v1.21.1/go.mod h1:oYelfM1adQP15Ek0mdvEgi9Df8B9CZIaU1084ijfRaM= diff --git a/internal/ack/once.go b/internal/ack/once.go new file mode 100644 index 0000000000..a1b92b220f --- /dev/null +++ b/internal/ack/once.go @@ -0,0 +1,75 @@ +// Copyright 2025 Redpanda Data, Inc. +// +// Licensed as a Redpanda Enterprise file under the Redpanda Community +// License (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// https://github.com/redpanda-data/connect/blob/main/licenses/rcl.md + +package ack + +import ( + "context" + "sync" +) + +// Once wraps an ack function and ensures that it is called at most once. Ack +// will return the same result every time. Wait can be called once. If Ack is +// called with error the ack is not called and error is propagated to Wait. +// Otherwise, Ack returns ack result and the result is also propagated to Wait. +type Once struct { + ack func(ctx context.Context) error + once sync.Once + ackErr error + waitErr error + done chan struct{} +} + +// NewOnce creates new Once. +func NewOnce(ack func(ctx context.Context) error) *Once { + return &Once{ + ack: ack, + done: make(chan struct{}), + once: sync.Once{}, + ackErr: nil, + } +} + +// Ack is service.AckFunc that ensures that ack is called at most once. +// See Once for details. +func (a *Once) Ack(ctx context.Context, err error) error { + a.once.Do(func() { + if err != nil { + a.waitErr = err + } else { + a.ackErr = a.ack(ctx) + a.waitErr = a.ackErr + } + close(a.done) + }) + + return a.ackErr +} + +// Wait waits for Ack call and returns the Ack error. See Once for details. +// Wait can be called multiple times and will always return the same result +// if Ack was called. +func (a *Once) Wait(ctx context.Context) error { + select { + case <-ctx.Done(): + return ctx.Err() + case <-a.done: + return a.waitErr + } +} + +// TryWait returns true if Ack was called and false otherwise. If Ack was called +// the Ack error is returned. +func (a *Once) TryWait() (bool, error) { + select { + case <-a.done: + return true, a.waitErr + default: + return false, nil + } +} diff --git a/internal/ack/once_test.go b/internal/ack/once_test.go new file mode 100644 index 0000000000..059b0cc14a --- /dev/null +++ b/internal/ack/once_test.go @@ -0,0 +1,67 @@ +// Copyright 2025 Redpanda Data, Inc. +// +// Licensed as a Redpanda Enterprise file under the Redpanda Community +// License (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// https://github.com/redpanda-data/connect/blob/main/licenses/rcl.md + +package ack + +import ( + "context" + "errors" + "testing" + + "github.com/stretchr/testify/assert" +) + +func TestOnceArgError(t *testing.T) { + a := NewOnce(func(_ context.Context) error { + t.Fatalf("Ack called") + return nil + }) + + assert.NoError(t, a.Ack(t.Context(), errors.New("arg error"))) + assert.NoError(t, a.Ack(t.Context(), errors.New("arg error"))) + assert.EqualError(t, a.Wait(t.Context()), "arg error") + assert.EqualError(t, a.Wait(t.Context()), "arg error") +} + +func TestOnceAckError(t *testing.T) { + a := NewOnce(func(_ context.Context) error { + return errors.New("ack error") + }) + + assert.EqualError(t, a.Ack(t.Context(), nil), "ack error") + assert.EqualError(t, a.Ack(t.Context(), nil), "ack error") + assert.EqualError(t, a.Wait(t.Context()), "ack error") + assert.EqualError(t, a.Wait(t.Context()), "ack error") +} + +func TestOnceWaitContextCanceled(t *testing.T) { + t.Parallel() + + ctx, cancel := context.WithCancel(t.Context()) + cancel() + + am := NewOnce(func(_ context.Context) error { + return nil + }) + + assert.ErrorIs(t, am.Wait(ctx), context.Canceled) +} + +func TestOnceAckOnce(t *testing.T) { + ackCount := 0 + a := NewOnce(func(_ context.Context) error { + ackCount++ + return nil + }) + + assert.NoError(t, a.Ack(t.Context(), nil)) + assert.NoError(t, a.Ack(t.Context(), nil)) + assert.NoError(t, a.Ack(t.Context(), nil)) + + assert.Equal(t, 1, ackCount, "Ack should be called exactly once") +} diff --git a/internal/impl/gcp/enterprise/changestreams/callback.go b/internal/impl/gcp/enterprise/changestreams/callback.go new file mode 100644 index 0000000000..8ee3dbc404 --- /dev/null +++ b/internal/impl/gcp/enterprise/changestreams/callback.go @@ -0,0 +1,42 @@ +// Copyright 2025 Redpanda Data, Inc. +// +// Licensed as a Redpanda Enterprise file under the Redpanda Community +// License (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// https://github.com/redpanda-data/connect/blob/main/licenses/rcl.md + +package changestreams + +import ( + "context" + "time" +) + +// CallbackFunc is a function that is called for each change record. +// If error is returned the processing will be stopped. Implementations should +// update the partition watermark by calling Subscriber.UpdatePartitionWatermark +// when data is processed. +// +// When partition ends, the callback will be called with a nil DataChangeRecord. +// If batch processing is enabled, the batch shall be flushed when the last +// record is received to avoid mixing records from different partitions in +// the same batch. +type CallbackFunc func(ctx context.Context, partitionToken string, dcr *DataChangeRecord) error + +// UpdatePartitionWatermark updates the watermark for a partition. It's intended +// for use by Callback function to update progress. If commitTimestamp is zero +// value, the watermark is not updated. +func (s *Subscriber) UpdatePartitionWatermark( + ctx context.Context, + partitionToken string, + commitTimestamp time.Time, +) error { + if commitTimestamp.IsZero() { + return nil + } + + s.log.Debugf("%s: updating watermark to %s", partitionToken, commitTimestamp) + + return s.store.UpdateWatermark(ctx, partitionToken, commitTimestamp) +} diff --git a/internal/impl/gcp/enterprise/changestreams/changestreamstest/emulator.go b/internal/impl/gcp/enterprise/changestreams/changestreamstest/emulator.go new file mode 100644 index 0000000000..23c0be6c7f --- /dev/null +++ b/internal/impl/gcp/enterprise/changestreams/changestreamstest/emulator.go @@ -0,0 +1,226 @@ +// Copyright 2025 Redpanda Data, Inc. +// +// Licensed as a Redpanda Enterprise file under the Redpanda Community +// License (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// https://github.com/redpanda-data/connect/blob/main/licenses/rcl.md + +package changestreamstest + +import ( + "context" + "errors" + "fmt" + "testing" + + "cloud.google.com/go/spanner" + adminapi "cloud.google.com/go/spanner/admin/database/apiv1" + adminpb "cloud.google.com/go/spanner/admin/database/apiv1/databasepb" + instance "cloud.google.com/go/spanner/admin/instance/apiv1" + "cloud.google.com/go/spanner/admin/instance/apiv1/instancepb" + "github.com/ory/dockertest/v3" + "github.com/ory/dockertest/v3/docker" + "google.golang.org/api/option" + "google.golang.org/grpc" + "google.golang.org/grpc/connectivity" + "google.golang.org/grpc/credentials/insecure" +) + +func startSpannerEmulator(t *testing.T) (addr string) { + pool, err := dockertest.NewPool("") + if err != nil { + t.Fatal(err) + } + + t.Log("Starting emulator") + res, err := pool.RunWithOptions(&dockertest.RunOptions{ + Repository: "gcr.io/cloud-spanner-emulator/emulator", + Tag: "latest", + Env: []string{ + "SPANNER_EMULATOR_HOST=0.0.0.0:9010", + }, + ExposedPorts: []string{"9010/tcp"}, + }, func(cfg *docker.HostConfig) { + cfg.AutoRemove = true + cfg.RestartPolicy = docker.RestartPolicy{ + Name: "no", + } + }) + if err != nil { + t.Fatal(err) + } + + closeFn := func() { + if err := pool.Purge(res); err != nil { + t.Errorf("Failed to purge resource: %v", err) + } + t.Log("Emulator stopped") + } + + addr = "localhost:" + res.GetPort("9010/tcp") + + if err := pool.Retry(func() error { + t.Logf("Waiting for emulator to be ready at %s", addr) + conn, err := grpc.NewClient(addr, grpc.WithTransportCredentials(insecure.NewCredentials())) + if err != nil { + return err + } + defer conn.Close() + + if s := conn.GetState(); s != connectivity.Ready { + return fmt.Errorf("not ready: %v", s) + } + return nil + }); err != nil { + closeFn() + t.Fatal(err) + } + + t.Cleanup(closeFn) + return +} + +const ( + // EmulatorProjectID is the project ID used for testing with the emulator. + EmulatorProjectID = "test-project" + // EmulatorInstanceID is the instance ID used for testing with the emulator + EmulatorInstanceID = "test-instance" +) + +func createInstance(ctx context.Context, conn *grpc.ClientConn) (string, error) { + adm, err := instance.NewInstanceAdminClient(ctx, + option.WithGRPCConn(conn), + option.WithoutAuthentication(), + ) + if err != nil { + return "", err + } + // Do not close as it will close the grpc connection + + op, err := adm.CreateInstance(ctx, &instancepb.CreateInstanceRequest{ + Parent: "projects/" + EmulatorProjectID, + InstanceId: EmulatorInstanceID, + Instance: &instancepb.Instance{ + Config: "projects/" + EmulatorProjectID + "/instanceConfigs/regional-europe-central2", + DisplayName: EmulatorInstanceID, + ProcessingUnits: 100, + }, + }) + if err != nil { + return "", err + } + + resp, err := op.Wait(ctx) + if err != nil { + return "", err + } + + return resp.Name, nil +} + +// EmulatorHelper provides utilities for working with the Spanner emulator in tests. +type EmulatorHelper struct { + *adminapi.DatabaseAdminClient + instanceName string + + t *testing.T + conn *grpc.ClientConn +} + +// MakeEmulatorHelper creates a new helper for interacting with the Spanner emulator in tests. +func MakeEmulatorHelper(t *testing.T) EmulatorHelper { + t.Helper() + + // Create a gRPC connection to the emulator + conn, err := grpc.NewClient(startSpannerEmulator(t), + grpc.WithTransportCredentials(insecure.NewCredentials())) + if err != nil { + t.Fatal(err) + } + + ctx := t.Context() + + // Create an instance + instanceName, err := createInstance(ctx, conn) + if err != nil { + t.Fatal(err) + } + + // Create the database admin client with the gRPC connection + adm, err := adminapi.NewDatabaseAdminClient(ctx, + option.WithGRPCConn(conn), + option.WithoutAuthentication()) + if err != nil { + t.Fatal(err) + } + + return EmulatorHelper{ + DatabaseAdminClient: adm, + instanceName: instanceName, + + t: t, + conn: conn, + } +} + +// CreateTestDatabase creates a new test database with the given name and returns a client connected to it. +func (e EmulatorHelper) CreateTestDatabase(dbName string, opts ...func(*adminpb.CreateDatabaseRequest)) *spanner.Client { + c, err := e.createTestDatabase(dbName, opts...) + if err != nil { + e.t.Fatal(err) + } + return c +} + +// CreateTestDatabaseWithDialect creates a new test database with the given name and dialect, and returns a client connected to it. +func (e EmulatorHelper) CreateTestDatabaseWithDialect(dbName string, dialect adminpb.DatabaseDialect, opts ...func(*adminpb.CreateDatabaseRequest)) *spanner.Client { + opts = append(opts, func(req *adminpb.CreateDatabaseRequest) { + req.DatabaseDialect = dialect + }) + + c, err := e.createTestDatabase(dbName, opts...) + if err != nil { + e.t.Fatal(err) + } + return c +} + +func (e EmulatorHelper) createTestDatabase(dbName string, opts ...func(*adminpb.CreateDatabaseRequest)) (*spanner.Client, error) { + req := &adminpb.CreateDatabaseRequest{ + Parent: e.instanceName, + CreateStatement: "CREATE DATABASE " + dbName, + } + for _, o := range opts { + o(req) + } + + e.t.Logf("Creating test database %q", dbName) + ctx := e.t.Context() + op, err := e.CreateDatabase(ctx, req) + if err != nil { + return nil, err + } + if _, err := op.Wait(ctx); err != nil { + return nil, err + } + c, err := spanner.NewClient(ctx, e.fullDatabaseName(dbName), option.WithGRPCConn(e.conn)) + if err != nil { + return nil, err + } + + return c, nil +} + +func (e EmulatorHelper) fullDatabaseName(dbName string) string { + return fmt.Sprintf("%s/databases/%s", e.instanceName, dbName) +} + +// Conn returns the gRPC client connection to the emulator. +func (e EmulatorHelper) Conn() *grpc.ClientConn { + return e.conn +} + +func (e EmulatorHelper) Close() error { + return errors.Join(e.DatabaseAdminClient.Close(), e.conn.Close()) +} diff --git a/internal/impl/gcp/enterprise/changestreams/changestreamstest/real.go b/internal/impl/gcp/enterprise/changestreams/changestreamstest/real.go new file mode 100644 index 0000000000..a6dbe0336a --- /dev/null +++ b/internal/impl/gcp/enterprise/changestreams/changestreamstest/real.go @@ -0,0 +1,231 @@ +// Copyright 2025 Redpanda Data, Inc. +// +// Licensed as a Redpanda Enterprise file under the Redpanda Community +// License (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// https://github.com/redpanda-data/connect/blob/main/licenses/rcl.md + +package changestreamstest + +import ( + "context" + "flag" + "fmt" + "math/rand" + "strings" + "testing" + "time" + + "cloud.google.com/go/spanner" + adminapi "cloud.google.com/go/spanner/admin/database/apiv1" + "cloud.google.com/go/spanner/admin/database/apiv1/databasepb" +) + +var ( + realSpannerProjectID = flag.String("spanner.project_id", "", "GCP project ID for Spanner tests") + realSpannerInstanceID = flag.String("spanner.instance_id", "", "Spanner instance ID for tests") + realSpannerDatabaseID = flag.String("spanner.database_id", "", "Spanner database ID for tests") +) + +// CheckSkipReal skips the test if the real Spanner environment is not configured. +// It checks if the required environment variables for real Spanner tests are set. +func CheckSkipReal(t *testing.T) { + if *realSpannerProjectID == "" || *realSpannerInstanceID == "" || *realSpannerDatabaseID == "" { + t.Skip("skipping real tests") + } +} + +func realSpannerFullDatabaseName() string { + return fmt.Sprintf("projects/%s/instances/%s/databases/%s", *realSpannerProjectID, *realSpannerInstanceID, *realSpannerDatabaseID) +} + +// MaybeDropOrphanedStreams finds all change streams with the pattern +// "rpcn_test_stream_%d" and deletes both the streams and their associated +// tables. +// +// Spanner has a limit of 10 streams per database. In some cases when tests fail +// the database may be left in a bad state. This function is used to clean up +// those bad states 10% of the time. +func MaybeDropOrphanedStreams(ctx context.Context) error { + if rand.Intn(100) > 10 { + return nil + } + return dropOrphanedStreams(ctx) +} + +func dropOrphanedStreams(ctx context.Context) error { + client, err := spanner.NewClient(ctx, realSpannerFullDatabaseName()) + if err != nil { + return err + } + + stmt := spanner.Statement{ + SQL: `SELECT change_stream_name FROM information_schema.change_streams WHERE change_stream_name LIKE 'rpcn_test_stream_%'`, + } + iter := client.Single().Query(ctx, stmt) + defer iter.Stop() + + // Collect all stream names + streamNames := make([]string, 0) + if err := iter.Do(func(row *spanner.Row) error { + var sn string + if err := row.Columns(&sn); err != nil { + return err + } + streamNames = append(streamNames, sn) + return nil + }); err != nil { + return err + } + + if len(streamNames) == 0 { + return nil + } + + dropSQLs := make([]string, 0, len(streamNames)*2) + for _, sn := range streamNames { + dropSQLs = append(dropSQLs, + fmt.Sprintf(`DROP CHANGE STREAM %s`, sn), + fmt.Sprintf(`DROP TABLE %s`, strings.Replace(sn, "stream", "table", 1))) + } + adm, err := adminapi.NewDatabaseAdminClient(ctx) + if err != nil { + return fmt.Errorf("failed to create admin client: %w", err) + } + + op, err := adm.UpdateDatabaseDdl(ctx, &databasepb.UpdateDatabaseDdlRequest{ + Database: realSpannerFullDatabaseName(), + Statements: dropSQLs, + }) + if err != nil { + return fmt.Errorf("failed to execute drop statements: %w", err) + } + return op.Wait(ctx) +} + +// RealHelper provides utilities for testing with a real Spanner instance. +// It manages the lifecycle of Spanner client and admin connections. +type RealHelper struct { + t *testing.T + admin *adminapi.DatabaseAdminClient + client *spanner.Client + table string + stream string +} + +// MakeRealHelper creates a RealHelper for the real spanner test environment. +func MakeRealHelper(t *testing.T) RealHelper { + client, err := spanner.NewClient(t.Context(), realSpannerFullDatabaseName()) + if err != nil { + t.Fatal(err) + } + + admin, err := adminapi.NewDatabaseAdminClient(t.Context()) + if err != nil { + t.Fatal(err) + } + + ts := time.Now().UnixNano() + return RealHelper{ + t: t, + admin: admin, + client: client, + table: fmt.Sprintf("rpcn_test_table_%d", ts), + stream: fmt.Sprintf("rpcn_test_stream_%d", ts), + } +} + +// ProjectID returns the project ID for the real Spanner instance. +func (RealHelper) ProjectID() string { + return *realSpannerProjectID +} + +// InstanceID returns the instance ID for the real Spanner instance. +func (RealHelper) InstanceID() string { + return *realSpannerInstanceID +} + +// DatabaseID returns the database ID for the real Spanner instance. +func (RealHelper) DatabaseID() string { + return *realSpannerDatabaseID +} + +// Table returns the table name generated for the test. +func (h RealHelper) Table() string { + return h.table +} + +// Stream returns the stream name generated for the test. +func (h RealHelper) Stream() string { + return h.stream +} + +// DatabaseAdminClient returns the database admin client. +func (h RealHelper) DatabaseAdminClient() *adminapi.DatabaseAdminClient { + return h.admin +} + +// Client returns the Spanner client. +func (h RealHelper) Client() *spanner.Client { + return h.client +} + +// CreateTableAndStream creates a table and a change stream for the current +// test. The table name and stream name are pre-generated and are available +// via Table() and Stream(). +func (h RealHelper) CreateTableAndStream(sql string) { + b := time.Now() + h.t.Logf("Creating table %q and stream %q", h.table, h.stream) + if err := h.createTableAndStream(sql); err != nil { + h.t.Fatal(err) + } + h.t.Logf("Table %q and stream %q created in %s", h.table, h.stream, time.Since(b)) + + h.t.Cleanup(func() { + if err := h.dropTableAndStream(); err != nil { + h.t.Logf("drop failed: %v", err) + } + }) +} + +func (h RealHelper) createTableAndStream(sql string) error { + ctx := h.t.Context() + + op, err := h.admin.UpdateDatabaseDdl(ctx, &databasepb.UpdateDatabaseDdlRequest{ + Database: realSpannerFullDatabaseName(), + Statements: []string{ + fmt.Sprintf(sql, h.table), + fmt.Sprintf(`CREATE CHANGE STREAM %s FOR %s`, h.stream, h.table), + }, + }) + if err != nil { + return fmt.Errorf("creating singers table: %w", err) + } + return op.Wait(ctx) +} + +func (h RealHelper) dropTableAndStream() error { + ctx := context.Background() + op, err := h.admin.UpdateDatabaseDdl(ctx, &databasepb.UpdateDatabaseDdlRequest{ + Database: realSpannerFullDatabaseName(), + Statements: []string{ + fmt.Sprintf(`DROP CHANGE STREAM %s`, h.stream), + fmt.Sprintf(`DROP TABLE %s`, h.table), + }, + }) + if err != nil { + return err + } + return op.Wait(ctx) +} + +func (h RealHelper) Close() error { + if err := h.admin.Close(); err != nil { + return err + } + + h.client.Close() + + return nil +} diff --git a/internal/impl/gcp/enterprise/changestreams/dialect.go b/internal/impl/gcp/enterprise/changestreams/dialect.go new file mode 100644 index 0000000000..fd0e5b3a9e --- /dev/null +++ b/internal/impl/gcp/enterprise/changestreams/dialect.go @@ -0,0 +1,57 @@ +// Copyright 2025 Redpanda Data, Inc. +// +// Licensed as a Redpanda Enterprise file under the Redpanda Community +// License (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// https://github.com/redpanda-data/connect/blob/main/licenses/rcl.md +// +// Copyright 2022 Google LLC +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. +// + +package changestreams + +import ( + "context" + + "cloud.google.com/go/spanner" + "cloud.google.com/go/spanner/admin/database/apiv1/databasepb" +) + +type dialect = databasepb.DatabaseDialect + +var ( + dialectGoogleSQL = databasepb.DatabaseDialect_GOOGLE_STANDARD_SQL + dialectPostgreSQL = databasepb.DatabaseDialect_POSTGRESQL +) + +func detectDialect(ctx context.Context, client *spanner.Client) (dialect, error) { + const stmt = `SELECT option_value FROM information_schema.database_options WHERE option_name = 'database_dialect'` + var v string + if err := client.Single().Query(ctx, spanner.NewStatement(stmt)).Do(func(r *spanner.Row) error { + return r.ColumnByName("option_value", &v) + }); err != nil { + return databasepb.DatabaseDialect_DATABASE_DIALECT_UNSPECIFIED, err + } + + switch v { + case dialectGoogleSQL.String(), "": + return dialectGoogleSQL, nil + case dialectPostgreSQL.String(): + return dialectPostgreSQL, nil + default: + return databasepb.DatabaseDialect_DATABASE_DIALECT_UNSPECIFIED, nil + } +} diff --git a/internal/impl/gcp/enterprise/changestreams/dialect_test.go b/internal/impl/gcp/enterprise/changestreams/dialect_test.go new file mode 100644 index 0000000000..96ce600eab --- /dev/null +++ b/internal/impl/gcp/enterprise/changestreams/dialect_test.go @@ -0,0 +1,58 @@ +// Copyright 2025 Redpanda Data, Inc. +// +// Licensed as a Redpanda Enterprise file under the Redpanda Community +// License (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// https://github.com/redpanda-data/connect/blob/main/licenses/rcl.md + +package changestreams + +import ( + "fmt" + "testing" + + adminpb "cloud.google.com/go/spanner/admin/database/apiv1/databasepb" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + + "github.com/redpanda-data/benthos/v4/public/service/integration" + + "github.com/redpanda-data/connect/v4/internal/impl/gcp/enterprise/changestreams/changestreamstest" +) + +func TestIntegrationDetectDialect(t *testing.T) { + integration.CheckSkip(t) + + e := changestreamstest.MakeEmulatorHelper(t) + + testCases := []struct { + dialect dialect + fn func(*adminpb.CreateDatabaseRequest) + }{ + { + dialect: dialectGoogleSQL, + }, + { + dialect: dialectPostgreSQL, + fn: func(req *adminpb.CreateDatabaseRequest) { + req.DatabaseDialect = dialectPostgreSQL + }, + }, + } + + for i, tc := range testCases { + t.Run(tc.dialect.String(), func(t *testing.T) { + dbName := fmt.Sprintf("dialect%d", i) + + var opts []func(*adminpb.CreateDatabaseRequest) + if tc.fn != nil { + opts = append(opts, tc.fn) + } + dd, err := detectDialect(t.Context(), e.CreateTestDatabase(dbName, opts...)) + require.NoError(t, err) + assert.Equal(t, tc.dialect, dd) + }) + } +} diff --git a/internal/impl/gcp/enterprise/changestreams/filter.go b/internal/impl/gcp/enterprise/changestreams/filter.go new file mode 100644 index 0000000000..d9f0ac8f96 --- /dev/null +++ b/internal/impl/gcp/enterprise/changestreams/filter.go @@ -0,0 +1,35 @@ +// Copyright 2025 Redpanda Data, Inc. +// +// Licensed as a Redpanda Enterprise file under the Redpanda Community +// License (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// https://github.com/redpanda-data/connect/blob/main/licenses/rcl.md + +package changestreams + +import ( + "context" +) + +// filteredCallback returns a CallbackFunc that filters out DataChangeRecords +// that don't match the provided filter. +func filteredCallback(cb CallbackFunc, filter func(dcr *DataChangeRecord) bool) CallbackFunc { + return func(ctx context.Context, partitionToken string, dcr *DataChangeRecord) error { + if dcr != nil && !filter(dcr) { + return nil + } + return cb(ctx, partitionToken, dcr) + } +} + +func modTypeFilter(allowedModTypes []string) func(dcr *DataChangeRecord) bool { + m := map[string]struct{}{} + for _, modType := range allowedModTypes { + m[modType] = struct{}{} + } + return func(dcr *DataChangeRecord) bool { + _, ok := m[dcr.ModType] + return ok + } +} diff --git a/internal/impl/gcp/enterprise/changestreams/handler.go b/internal/impl/gcp/enterprise/changestreams/handler.go new file mode 100644 index 0000000000..0b91a8e352 --- /dev/null +++ b/internal/impl/gcp/enterprise/changestreams/handler.go @@ -0,0 +1,120 @@ +// Copyright 2025 Redpanda Data, Inc. +// +// Licensed as a Redpanda Enterprise file under the Redpanda Community +// License (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// https://github.com/redpanda-data/connect/blob/main/licenses/rcl.md + +package changestreams + +import ( + "context" + "fmt" + "time" + + "cloud.google.com/go/spanner" + "google.golang.org/grpc/codes" + + "github.com/redpanda-data/benthos/v4/public/service" + + "github.com/redpanda-data/connect/v4/internal/impl/gcp/enterprise/changestreams/metadata" +) + +type handler struct { + pm metadata.PartitionMetadata + tr timeRange + cb CallbackFunc + store *metadata.Store + log *service.Logger + metrics *Metrics +} + +func (s *Subscriber) partitionMetadataHandler(pm metadata.PartitionMetadata) *handler { + return &handler{ + pm: pm, + cb: s.cb, + tr: timeRange{ + cur: pm.StartTimestamp, + end: pm.EndTimestamp, + }, + store: s.store, + log: s.log, + metrics: s.metrics, + } +} + +func (h *handler) handleChangeRecord(ctx context.Context, cr ChangeRecord) error { + if err := h.handleDataChangeRecords(ctx, cr); err != nil { + return err + } + for _, hr := range cr.HeartbeatRecords { + h.metrics.IncHeartbeatRecordCount() + h.tr.tryClaim(hr.Timestamp) + } + if err := h.handleChildPartitionsRecords(ctx, cr); err != nil { + return err + } + + return nil +} + +func (h *handler) handleDataChangeRecords(ctx context.Context, cr ChangeRecord) error { + for _, dcr := range cr.DataChangeRecords { + h.metrics.IncDataChangeRecordCount() + if !h.tr.tryClaim(dcr.CommitTimestamp) { + h.log.Errorf("%s: failed to claim data change record timestamp: %v, current: %v", + h.pm.PartitionToken, dcr.CommitTimestamp, h.tr.now()) + continue + } + + h.log.Debugf("%s: data change record: table: %s, modification type: %s, commit timestamp: %v", + h.pm.PartitionToken, dcr.TableName, dcr.ModType, dcr.CommitTimestamp) + + if err := h.cb(ctx, h.pm.PartitionToken, dcr); err != nil { + return fmt.Errorf("data change record handler failed: %w", err) + } + h.metrics.UpdateDataChangeRecordCommittedToEmitted(time.Since(dcr.CommitTimestamp)) + + // Updating watermark is delegated to Callback. + } + return nil +} + +func (h *handler) handleChildPartitionsRecords(ctx context.Context, cr ChangeRecord) error { + for _, cpr := range cr.ChildPartitionsRecords { + if !h.tr.tryClaim(cpr.StartTimestamp) { + h.log.Errorf("%s: failed to claim child partition record timestamp: %v, current: %v", + h.pm.PartitionToken, cpr.StartTimestamp, h.tr.now()) + continue + } + + var childPartitions []metadata.PartitionMetadata + for _, cp := range cpr.ChildPartitions { + h.log.Debugf("%s: child partition: token: %s, parent partition tokens: %+v", + h.pm.PartitionToken, cp.Token, cp.ParentPartitionTokens) + childPartitions = append(childPartitions, + cp.toPartitionMetadata(cpr.StartTimestamp, h.pm.EndTimestamp, h.pm.HeartbeatMillis)) + } + + if err := h.store.Create(ctx, childPartitions); err != nil { + if spanner.ErrCode(err) != codes.AlreadyExists { + return fmt.Errorf("create partitions: %w", err) + } + } + h.metrics.IncPartitionRecordCreatedCount(len(childPartitions)) + + for _, cp := range cpr.ChildPartitions { + if cp.isSplit() { + h.metrics.IncPartitionRecordSplitCount() + } else { + h.metrics.IncPartitionRecordMergeCount() + } + } + } + return nil +} + +func (h *handler) watermark() time.Time { + return h.tr.now() +} diff --git a/internal/impl/gcp/enterprise/changestreams/metadata/metadata.go b/internal/impl/gcp/enterprise/changestreams/metadata/metadata.go new file mode 100644 index 0000000000..54405a737e --- /dev/null +++ b/internal/impl/gcp/enterprise/changestreams/metadata/metadata.go @@ -0,0 +1,680 @@ +// Copyright 2025 Redpanda Data, Inc. +// +// Licensed as a Redpanda Enterprise file under the Redpanda Community +// License (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// https://github.com/redpanda-data/connect/blob/main/licenses/rcl.md + +package metadata + +import ( + "context" + "errors" + "fmt" + "strings" + "time" + + "cloud.google.com/go/spanner" + adminapi "cloud.google.com/go/spanner/admin/database/apiv1" + "cloud.google.com/go/spanner/admin/database/apiv1/databasepb" + lru "github.com/hashicorp/golang-lru/v2" + "google.golang.org/api/iterator" +) + +// State represents the current status of a partition in the change stream. +type State string + +// Possible states for a partition in the change stream. +const ( + StateCreated State = "CREATED" + StateScheduled State = "SCHEDULED" + StateRunning State = "RUNNING" + StateFinished State = "FINISHED" +) + +// PartitionMetadata contains information about a change stream partition. +// +// To support reading change stream records in near real-time as database +// writes scale, the Spanner API is designed for a change stream to be queried +// concurrently using change stream partitions. Change stream partitions +// map to change stream data splits that contain the change stream records. +// A change stream's partitions change dynamically over time and are correlated +// to how Spanner dynamically splits and merges the database data. +// +// A change stream partition contains records for an immutable key range for +// a specific time range. Any change stream partition can split into one or more +// change stream partitions, or be merged with other change stream partitions. +// When these split or merge events happen, child partitions are created to +// capture the changes for their respective immutable key ranges for the next +// time range. In addition to data change records, a change stream query returns +// child partition records to notify readers of new change stream partitions +// that need to be queried, as well as heartbeat records to indicate forward +// progress when no writes have occurred recently. +// +// The StartTimestamp is taken from ChildPartitionsRecord.StartTimestamp, +// and represents the earliest DataChangeRecord.CommitTimestamp in this +// partition or in the sibling partitions. +// +// The Watermark is set to the last processed DataChangeRecord.CommitTimestamp +// in this partition. +// +// The order of timestamps monotonically increases, starting with: +// - StartTimestamp, +// - Watermark, +// - CreatedAt, +// - ScheduledAt, +// - RunningAt, +// - FinishedAt. +// +// The last four timestamps are set to the Spanner commit timestamp when +// the PartitionMetadata record is created, scheduled, started, or finished. +type PartitionMetadata struct { + PartitionToken string `spanner:"PartitionToken" json:"partition_token"` + ParentTokens []string `spanner:"ParentTokens" json:"parent_tokens"` + StartTimestamp time.Time `spanner:"StartTimestamp" json:"start_timestamp"` + EndTimestamp time.Time `spanner:"EndTimestamp" json:"end_timestamp"` + HeartbeatMillis int64 `spanner:"HeartbeatMillis" json:"heartbeat_millis"` + State State `spanner:"State" json:"state"` + Watermark time.Time `spanner:"Watermark" json:"watermark"` + CreatedAt time.Time `spanner:"CreatedAt" json:"created_at"` + ScheduledAt *time.Time `spanner:"ScheduledAt" json:"scheduled_at,omitempty"` + RunningAt *time.Time `spanner:"RunningAt" json:"running_at,omitempty"` + FinishedAt *time.Time `spanner:"FinishedAt" json:"finished_at,omitempty"` +} + +// Column names for the partition metadata table +const ( + columnPartitionToken = "PartitionToken" + columnParentTokens = "ParentTokens" + columnStartTimestamp = "StartTimestamp" + columnEndTimestamp = "EndTimestamp" + columnHeartbeatMillis = "HeartbeatMillis" + columnState = "State" + columnWatermark = "Watermark" + columnCreatedAt = "CreatedAt" + columnScheduledAt = "ScheduledAt" + columnRunningAt = "RunningAt" + columnFinishedAt = "FinishedAt" +) + +// StoreConfig contains configuration for the metadata store. +type StoreConfig struct { + ProjectID string + InstanceID string + DatabaseID string + Dialect databasepb.DatabaseDialect + TableNames +} + +func (c StoreConfig) fullDatabaseName() string { + return fmt.Sprintf("projects/%s/instances/%s/databases/%s", c.ProjectID, c.InstanceID, c.DatabaseID) +} + +func (c StoreConfig) isPostgres() bool { + return c.Dialect == databasepb.DatabaseDialect_POSTGRESQL +} + +// CreatePartitionMetadataTableWithDatabaseAdminClient creates a table for +// storing partition metadata if it doesn't exist. +func CreatePartitionMetadataTableWithDatabaseAdminClient( + ctx context.Context, + conf StoreConfig, + adm *adminapi.DatabaseAdminClient, +) error { + const TTLAfterPartitionFinishedDays = 1 + + var ddl []string + + if conf.isPostgres() { + // PostgreSQL requires quotes around identifiers to preserve casing + ddl = append(ddl, fmt.Sprintf(`CREATE TABLE IF NOT EXISTS "%s"("%s" text NOT NULL,"%s" text[] NOT NULL,"%s" timestamptz NOT NULL,"%s" timestamptz NOT NULL,"%s" BIGINT NOT NULL,"%s" text NOT NULL,"%s" timestamptz NOT NULL,"%s" SPANNER.COMMIT_TIMESTAMP NOT NULL,"%s" SPANNER.COMMIT_TIMESTAMP,"%s" SPANNER.COMMIT_TIMESTAMP,"%s" SPANNER.COMMIT_TIMESTAMP, PRIMARY KEY ("%s")) TTL INTERVAL '%d days' ON "%s"`, + conf.TableName, + columnPartitionToken, + columnParentTokens, + columnStartTimestamp, + columnEndTimestamp, + columnHeartbeatMillis, + columnState, + columnWatermark, + columnCreatedAt, + columnScheduledAt, + columnRunningAt, + columnFinishedAt, + columnPartitionToken, + TTLAfterPartitionFinishedDays, + columnFinishedAt)) + + ddl = append(ddl, fmt.Sprintf(`CREATE INDEX IF NOT EXISTS "%s" on "%s" ("%s") INCLUDE ("%s")`, + conf.WatermarkIndexName, + conf.TableName, + columnWatermark, + columnState)) + + ddl = append(ddl, fmt.Sprintf(`CREATE INDEX IF NOT EXISTS "%s" ON "%s" ("%s","%s")`, + conf.CreatedAtIndexName, + conf.TableName, + columnCreatedAt, + columnStartTimestamp)) + } else { + ddl = append(ddl, fmt.Sprintf(`CREATE TABLE IF NOT EXISTS %s (%s STRING(MAX) NOT NULL,%s ARRAY NOT NULL,%s TIMESTAMP NOT NULL,%s TIMESTAMP NOT NULL,%s INT64 NOT NULL,%s STRING(MAX) NOT NULL,%s TIMESTAMP NOT NULL,%s TIMESTAMP NOT NULL OPTIONS (allow_commit_timestamp=true),%s TIMESTAMP OPTIONS (allow_commit_timestamp=true),%s TIMESTAMP OPTIONS (allow_commit_timestamp=true),%s TIMESTAMP OPTIONS (allow_commit_timestamp=true)) PRIMARY KEY (%s), ROW DELETION POLICY (OLDER_THAN(%s, INTERVAL %d DAY))`, + conf.TableName, + columnPartitionToken, + columnParentTokens, + columnStartTimestamp, + columnEndTimestamp, + columnHeartbeatMillis, + columnState, + columnWatermark, + columnCreatedAt, + columnScheduledAt, + columnRunningAt, + columnFinishedAt, + columnPartitionToken, + columnFinishedAt, + TTLAfterPartitionFinishedDays)) + + ddl = append(ddl, fmt.Sprintf(`CREATE INDEX IF NOT EXISTS %s on %s (%s) STORING (%s)`, + conf.WatermarkIndexName, + conf.TableName, + columnWatermark, + columnState)) + + ddl = append(ddl, fmt.Sprintf(`CREATE INDEX IF NOT EXISTS %s ON %s (%s,%s)`, + conf.CreatedAtIndexName, + conf.TableName, + columnCreatedAt, + columnStartTimestamp)) + } + + op, err := adm.UpdateDatabaseDdl(ctx, &databasepb.UpdateDatabaseDdlRequest{ + Database: conf.fullDatabaseName(), + Statements: ddl, + }) + if err != nil { + return fmt.Errorf("create partition metadata table: %w", err) + } + + if err := op.Wait(ctx); err != nil { + return fmt.Errorf("wait for partition metadata table creation: %w", err) + } + + return nil +} + +// DeletePartitionMetadataTableWithDatabaseAdminClient deletes the partition +// metadata table. +func DeletePartitionMetadataTableWithDatabaseAdminClient( + ctx context.Context, + conf StoreConfig, + adm *adminapi.DatabaseAdminClient, +) error { + var ddl []string + + if conf.isPostgres() { + for _, index := range []string{conf.WatermarkIndexName, conf.CreatedAtIndexName} { + ddl = append(ddl, fmt.Sprintf(`DROP INDEX "%s"`, index)) + } + ddl = append(ddl, fmt.Sprintf(`DROP TABLE "%s"`, conf.TableName)) + } else { + for _, index := range []string{conf.WatermarkIndexName, conf.CreatedAtIndexName} { + ddl = append(ddl, fmt.Sprintf(`DROP INDEX %s`, index)) + } + ddl = append(ddl, fmt.Sprintf(`DROP TABLE %s`, conf.TableName)) + } + + op, err := adm.UpdateDatabaseDdl(ctx, &databasepb.UpdateDatabaseDdlRequest{ + Database: conf.fullDatabaseName(), + Statements: ddl, + }) + if err != nil { + return fmt.Errorf("delete partition metadata table: %w", err) + } + + if err := op.Wait(ctx); err != nil { + return fmt.Errorf("wait for partition metadata table deletion: %w", err) + } + + return nil +} + +// Store manages the persistence of partition metadata. +type Store struct { + conf StoreConfig + client *spanner.Client + + finishedTokensCache *lru.Cache[string, struct{}] +} + +const defaultFinishedTokensCacheSize = 10_000 + +// NewStore returns a Store instance with the given configuration and Spanner +// client. The client must be connected to the same database as the configuration. +func NewStore(conf StoreConfig, client *spanner.Client) (*Store, error) { + cache, err := lru.New[string, struct{}](defaultFinishedTokensCacheSize) + if err != nil { + return nil, fmt.Errorf("create LRU cache: %w", err) + } + + return &Store{ + conf: conf, + client: client, + finishedTokensCache: cache, + }, nil +} + +// Config returns the store configuration. +func (s *Store) Config() StoreConfig { + return s.conf +} + +// GetPartition fetches the partition metadata row data for the given partition token. +func (s *Store) GetPartition(ctx context.Context, partitionToken string) (PartitionMetadata, error) { + var stmt spanner.Statement + if s.conf.isPostgres() { + stmt = spanner.Statement{ + SQL: fmt.Sprintf(`SELECT * FROM "%s" WHERE "%s" = $1`, + s.conf.TableName, columnPartitionToken), + Params: map[string]any{"p1": partitionToken}, + } + } else { + stmt = spanner.Statement{ + SQL: fmt.Sprintf(`SELECT * FROM %s WHERE %s = @partition`, + s.conf.TableName, columnPartitionToken), + Params: map[string]any{"partition": partitionToken}, + } + } + + iter := s.client.Single().QueryWithOptions(ctx, stmt, queryTag("GetPartition")) + defer iter.Stop() + + row, err := iter.Next() + if errors.Is(err, iterator.Done) { + return PartitionMetadata{}, nil + } + if err != nil { + return PartitionMetadata{}, fmt.Errorf("get partition: %w", err) + } + + var pm PartitionMetadata + if err := row.ToStruct(&pm); err != nil { + return PartitionMetadata{}, fmt.Errorf("parse partition: %w", err) + } + + return pm, nil +} + +// GetUnfinishedMinWatermark fetches the earliest partition watermark from +// the partition metadata table that is not in a FINISHED state. +func (s *Store) GetUnfinishedMinWatermark(ctx context.Context) (time.Time, error) { + var stmt spanner.Statement + if s.conf.isPostgres() { + stmt = spanner.Statement{ + SQL: fmt.Sprintf(`SELECT "%s" FROM "%s" WHERE "%s" != $1 ORDER BY "%s" ASC LIMIT 1`, + columnWatermark, s.conf.TableName, columnState, columnWatermark), + Params: map[string]any{"p1": StateFinished}, + } + } else { + stmt = spanner.Statement{ + SQL: fmt.Sprintf(`SELECT %s FROM %s WHERE %s != @state ORDER BY %s ASC LIMIT 1`, + columnWatermark, s.conf.TableName, columnState, columnWatermark), + Params: map[string]any{"state": StateFinished}, + } + } + + iter := s.client.Single().QueryWithOptions(ctx, stmt, queryTag("GetUnfinishedMinWatermark")) + defer iter.Stop() + + row, err := iter.Next() + if errors.Is(err, iterator.Done) { + return time.Time{}, nil + } + if err != nil { + return time.Time{}, fmt.Errorf("get unfinished min watermark: %w", err) + } + + var watermark time.Time + if err := row.Columns(&watermark); err != nil { + return time.Time{}, fmt.Errorf("parse watermark: %w", err) + } + + return watermark, nil +} + +// GetPartitionsCreatedAfter fetches all partitions created after the +// specified timestamp that are in the CREATED state. Results are ordered by +// creation time and start timestamp in ascending order. +func (s *Store) GetPartitionsCreatedAfter(ctx context.Context, timestamp time.Time) ([]PartitionMetadata, error) { + var stmt spanner.Statement + if s.conf.isPostgres() { + stmt = spanner.Statement{ + SQL: fmt.Sprintf(`SELECT * FROM "%s" WHERE "%s" > $1 AND "%s" = $2 ORDER BY "%s" ASC, "%s" ASC`, + s.conf.TableName, columnCreatedAt, columnState, columnCreatedAt, columnStartTimestamp), + Params: map[string]any{ + "p1": timestamp, + "p2": StateCreated, + }, + } + } else { + stmt = spanner.Statement{ + SQL: fmt.Sprintf(`SELECT * FROM %s WHERE %s > @timestamp AND %s = @state ORDER BY %s ASC, %s ASC`, + s.conf.TableName, columnCreatedAt, columnState, columnCreatedAt, columnStartTimestamp), + Params: map[string]any{ + "timestamp": timestamp, + "state": StateCreated, + }, + } + } + + iter := s.client.Single().QueryWithOptions(ctx, stmt, queryTag("GetPartitionsCreatedAfter")) + defer iter.Stop() + + var pms []PartitionMetadata + if err := iter.Do(func(row *spanner.Row) error { + var p PartitionMetadata + if err := row.ToStruct(&p); err != nil { + return err + } + pms = append(pms, p) + return nil + }); err != nil { + return nil, fmt.Errorf("get all partitions created after: %w", err) + } + + return pms, nil +} + +// GetInterruptedPartitions fetches all partitions that are in SCHEDULED or +// RUNNING state. These partitions are considered "interrupted" as they were +// being processed but didn't reach the FINISHED state. Results are ordered +// by creation time and start timestamp in ascending order. +func (s *Store) GetInterruptedPartitions(ctx context.Context) ([]PartitionMetadata, error) { + var ( + sql string + params map[string]any + ) + + states := []State{StateScheduled, StateRunning} + + if s.conf.isPostgres() { + sql = fmt.Sprintf(`SELECT * FROM "%s" WHERE "%s" = ANY($1) ORDER BY "%s" ASC, "%s" ASC`, + s.conf.TableName, + columnState, + columnCreatedAt, + columnStartTimestamp) + params = map[string]any{ + "p1": states, + } + } else { + sql = fmt.Sprintf("SELECT * FROM %s WHERE %s IN UNNEST(@states) ORDER BY %s ASC, %s ASC", + s.conf.TableName, + columnState, + columnCreatedAt, + columnStartTimestamp) + params = map[string]any{ + "states": states, + } + } + + stmt := spanner.Statement{ + SQL: sql, + Params: params, + } + + iter := s.client.Single().QueryWithOptions(ctx, stmt, queryTag("GetInterruptedPartitions")) + + var pms []PartitionMetadata + if err := iter.Do(func(r *spanner.Row) error { + var pm PartitionMetadata + if err := r.ToStruct(&pm); err != nil { + return err + } + pms = append(pms, pm) + return nil + }); err != nil { + return nil, fmt.Errorf("get interrupted partitions: %w", err) + } + + return pms, nil +} + +// Create creates a new partition metadata row in state CREATED. +func (s *Store) Create(ctx context.Context, pms []PartitionMetadata) error { + ms := make([]*spanner.Mutation, len(pms)) + + for i, p := range pms { + ms[i] = spanner.Insert(s.conf.TableName, + []string{ + columnPartitionToken, + columnParentTokens, + columnStartTimestamp, + columnEndTimestamp, + columnHeartbeatMillis, + columnState, + columnWatermark, + columnCreatedAt, + }, + []any{ + p.PartitionToken, + p.ParentTokens, + p.StartTimestamp, + p.EndTimestamp, + p.HeartbeatMillis, + StateCreated, + p.Watermark, + spanner.CommitTimestamp, + }) + } + + return s.applyWithTag(ctx, "Create", ms...) +} + +func (s *Store) insert(ctx context.Context, partitions []PartitionMetadata) error { + ms := make([]*spanner.Mutation, len(partitions)) + + var err error + for i := range partitions { + ms[i], err = spanner.InsertStruct(s.conf.TableName, &partitions[i]) + if err != nil { + return err + } + } + + return s.applyWithTag(ctx, "Insert", ms...) +} + +// UpdateToScheduled updates multiple partition rows to SCHEDULED state. It only +// updates partitions that are currently in CREATED state. Returns the commit +// timestamp of the transaction. +func (s *Store) UpdateToScheduled(ctx context.Context, partitionTokens []string) (time.Time, error) { + return s.updatePartitionStatus(ctx, partitionTokens, StateCreated, StateScheduled, columnScheduledAt) +} + +// UpdateToRunning updates partition row to RUNNING state. It only updates +// partitions that are currently in SCHEDULED state. Returns the commit +// timestamp of the transaction. +func (s *Store) UpdateToRunning(ctx context.Context, partitionToken string) (time.Time, error) { + return s.updatePartitionStatus(ctx, []string{partitionToken}, StateScheduled, StateRunning, columnRunningAt) +} + +// UpdateToFinished updates partition row to FINISHED state. It only updates +// partitions that are currently in RUNNING state. Returns the commit +// timestamp of the transaction. +func (s *Store) UpdateToFinished(ctx context.Context, partitionToken string) (time.Time, error) { + ts, err := s.updatePartitionStatus(ctx, []string{partitionToken}, StateRunning, StateFinished, columnFinishedAt) + if err == nil { + s.finishedTokensCache.Add(partitionToken, struct{}{}) + } + return ts, err +} + +// updatePartitionStatus updates partition rows from fromState to toState and +// sets the specified timestamp column to the commit timestamp. It only updates +// partitions that are currently in fromState. Returns the commit timestamp +// of the transaction. +func (s *Store) updatePartitionStatus( + ctx context.Context, + partitionTokens []string, + fromState State, + toState State, + timestampColumn string, +) (time.Time, error) { + resp, err := s.client.ReadWriteTransactionWithOptions(ctx, func(ctx context.Context, txn *spanner.ReadWriteTransaction) error { + matchingTokens, err := s.getPartitionsMatchingStateInTransaction(ctx, txn, partitionTokens, fromState) + if err != nil { + return fmt.Errorf("get partitions matching state: %w", err) + } + + var ms []*spanner.Mutation + for _, token := range matchingTokens { + m := spanner.Update( + s.conf.TableName, + []string{ + columnPartitionToken, + columnState, + timestampColumn, + }, + []any{ + token, + toState, + spanner.CommitTimestamp, + }) + ms = append(ms, m) + } + return txn.BufferWrite(ms) + }, spanner.TransactionOptions{TransactionTag: "UpdateTo" + strings.ToTitle(string(toState))}) + + return resp.CommitTs.UTC(), err +} + +// CheckPartitionsFinished checks if all parent tokens in the given list +// are in FINISHED state. +func (s *Store) CheckPartitionsFinished(ctx context.Context, partitionTokens []string) (bool, error) { + if len(partitionTokens) == 0 { + return true, nil + } + + uncachedTokens := make([]string, 0, len(partitionTokens)) + for _, token := range partitionTokens { + if _, ok := s.finishedTokensCache.Get(token); !ok { + uncachedTokens = append(uncachedTokens, token) + } + } + if len(uncachedTokens) == 0 { + return true, nil + } + + var ok bool + + if _, err := s.client.ReadWriteTransactionWithOptions(ctx, func(ctx context.Context, txn *spanner.ReadWriteTransaction) error { + matchingTokens, err := s.getPartitionsMatchingStateInTransaction(ctx, txn, uncachedTokens, StateFinished) + if err != nil { + return fmt.Errorf("get partitions matching state: %w", err) + } + + for _, token := range matchingTokens { + s.finishedTokensCache.Add(token, struct{}{}) + } + + ok = len(uncachedTokens) == len(matchingTokens) + return nil + }, spanner.TransactionOptions{TransactionTag: "CheckPartitionsFinished"}); err != nil { + return false, err + } + + return ok, nil +} + +func (s *Store) getPartitionsMatchingStateInTransaction( + ctx context.Context, + txn *spanner.ReadWriteTransaction, + partitionTokens []string, + state State, +) ([]string, error) { + var stmt spanner.Statement + if s.conf.isPostgres() { + var sb strings.Builder + for i, tok := range partitionTokens { + if i > 0 { + sb.WriteByte(',') + } + sb.WriteByte('\'') + sb.WriteString(tok) + sb.WriteByte('\'') + } + + stmt = spanner.Statement{ + SQL: fmt.Sprintf(`SELECT "%s" FROM "%s" WHERE "%s" = ANY (Array[%s]) AND "%s" = '%s'`, + columnPartitionToken, + s.conf.TableName, + columnPartitionToken, + sb.String(), + columnState, + state), + } + } else { + stmt = spanner.Statement{ + SQL: fmt.Sprintf(`SELECT %s FROM %s WHERE %s IN UNNEST(@partitionTokens) AND %s = @state`, + columnPartitionToken, + s.conf.TableName, + columnPartitionToken, + columnState), + Params: map[string]any{ + "partitionTokens": partitionTokens, + "state": state, + }, + } + } + + iter := txn.QueryWithOptions(ctx, stmt, queryTag(fmt.Sprintf("getPartitionsMatchingState=%s", state))) + defer iter.Stop() + + var matchingTokens []string + for { + row, err := iter.Next() + if errors.Is(err, iterator.Done) { + break + } + if err != nil { + return nil, fmt.Errorf("query partitions: %w", err) + } + + var token string + if err := row.Column(0, &token); err != nil { + return nil, fmt.Errorf("get partition token: %w", err) + } + matchingTokens = append(matchingTokens, token) + } + + return matchingTokens, nil +} + +// UpdateWatermark updates the partition watermark to the given timestamp. +func (s *Store) UpdateWatermark(ctx context.Context, partitionToken string, watermark time.Time) error { + m := spanner.Update( + s.conf.TableName, + []string{ + columnPartitionToken, + columnWatermark, + }, + []any{ + partitionToken, + watermark, + }, + ) + + return s.applyWithTag(ctx, "updateWatermark", m) +} + +func queryTag(tag string) spanner.QueryOptions { + return spanner.QueryOptions{RequestTag: "query=" + tag} +} + +func (s *Store) applyWithTag(ctx context.Context, tag string, ms ...*spanner.Mutation) error { + _, err := s.client.Apply(ctx, ms, spanner.TransactionTag(tag)) + if err != nil { + return fmt.Errorf("%s: %w", tag, err) + } + + return nil +} diff --git a/internal/impl/gcp/enterprise/changestreams/metadata/metadata_integration_test.go b/internal/impl/gcp/enterprise/changestreams/metadata/metadata_integration_test.go new file mode 100644 index 0000000000..56bc0f5eff --- /dev/null +++ b/internal/impl/gcp/enterprise/changestreams/metadata/metadata_integration_test.go @@ -0,0 +1,378 @@ +// Copyright 2025 Redpanda Data, Inc. +// +// Licensed as a Redpanda Enterprise file under the Redpanda Community +// License (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// https://github.com/redpanda-data/connect/blob/main/licenses/rcl.md + +package metadata + +import ( + "context" + "sync" + "testing" + "time" + + "cloud.google.com/go/spanner" + "cloud.google.com/go/spanner/admin/database/apiv1/databasepb" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "google.golang.org/grpc/codes" + + "github.com/redpanda-data/benthos/v4/public/service/integration" + + "github.com/redpanda-data/connect/v4/internal/impl/gcp/enterprise/changestreams/changestreamstest" +) + +func testStores(t *testing.T, e changestreamstest.EmulatorHelper) (*Store, *Store) { + const ( + googleSQLDatabaseName = "google_sql_db" + postgresDatabaseName = "postgres_db" + ) + + g, err := NewStore(StoreConfig{ + ProjectID: changestreamstest.EmulatorProjectID, + InstanceID: changestreamstest.EmulatorInstanceID, + DatabaseID: googleSQLDatabaseName, + TableNames: RandomTableNames(googleSQLDatabaseName), + Dialect: databasepb.DatabaseDialect_GOOGLE_STANDARD_SQL, + }, e.CreateTestDatabase(googleSQLDatabaseName)) + require.NoError(t, err) + + p, err := NewStore(StoreConfig{ + ProjectID: changestreamstest.EmulatorProjectID, + InstanceID: changestreamstest.EmulatorInstanceID, + DatabaseID: postgresDatabaseName, + TableNames: RandomTableNames(postgresDatabaseName), + Dialect: databasepb.DatabaseDialect_POSTGRESQL, + }, e.CreateTestDatabaseWithDialect(postgresDatabaseName, databasepb.DatabaseDialect_POSTGRESQL)) + require.NoError(t, err) + + return g, p +} + +func TestIntegrationStore(t *testing.T) { + integration.CheckSkip(t) + + e := changestreamstest.MakeEmulatorHelper(t) + g, p := testStores(t, e) + tests := []struct { + name string + s *Store + }{ + {name: "GoogleSQL", s: g}, + {name: "Postgres", s: p}, + } + + t.Run("CreatePartitionMetadataTableWithDatabaseAdminClient", func(t *testing.T) { + for _, tc := range tests { + t.Run(tc.name, func(t *testing.T) { + require.NoError(t, + CreatePartitionMetadataTableWithDatabaseAdminClient(t.Context(), tc.s.conf, e.DatabaseAdminClient)) + }) + } + }) + + t.Run("GetUnfinishedMinWatermarkEmpty", func(t *testing.T) { + for _, tc := range tests { + t.Run(tc.name, func(t *testing.T) { + require.NoError(t, + CreatePartitionMetadataTableWithDatabaseAdminClient(t.Context(), tc.s.conf, e.DatabaseAdminClient)) + + // Test with empty table + got, err := tc.s.GetUnfinishedMinWatermark(t.Context()) + require.NoError(t, err) + + // Should return zero time when no data exists + want := time.Time{} + assert.Equal(t, want, got) + }) + } + }) + + ts := time.Date(2025, 1, 1, 0, 0, 0, 0, time.UTC) + pm := func(token string, start time.Time, state State) PartitionMetadata { + return PartitionMetadata{ + PartitionToken: token, + ParentTokens: []string{}, + StartTimestamp: start, + State: state, + Watermark: start, + CreatedAt: start, + } + } + + t.Run("InsertTestData", func(t *testing.T) { + for _, tc := range tests { + t.Run(tc.name, func(t *testing.T) { + require.NoError(t, tc.s.insert(t.Context(), []PartitionMetadata{ + pm("created1", ts, StateCreated), + pm("created2", ts.Add(-2*time.Second), StateCreated), + pm("scheduled", ts.Add(time.Second), StateScheduled), + pm("running", ts.Add(2*time.Second), StateRunning), + pm("finished", ts.Add(-time.Second), StateFinished), + })) + }) + } + }) + + t.Run("GetPartition", func(t *testing.T) { + for _, tc := range tests { + t.Run(tc.name, func(t *testing.T) { + got, err := tc.s.GetPartition(t.Context(), "created1") + require.NoError(t, err) + want := pm("created1", ts, StateCreated) + assert.Equal(t, want, got) + }) + } + }) + + t.Run("GetUnfinishedMinWatermark", func(t *testing.T) { + for _, tc := range tests { + t.Run(tc.name, func(t *testing.T) { + got, err := tc.s.GetUnfinishedMinWatermark(t.Context()) + require.NoError(t, err) + + want := ts.Add(-2 * time.Second) + assert.Equal(t, want, got) + }) + } + }) + + t.Run("GetPartitionsCreatedAfter", func(t *testing.T) { + cutoff := ts.Add(-1 * time.Second) + + for _, tc := range tests { + t.Run(tc.name, func(t *testing.T) { + got, err := tc.s.GetPartitionsCreatedAfter(t.Context(), cutoff) + require.NoError(t, err) + + want := []PartitionMetadata{ + pm("created1", ts, StateCreated), + } + + assert.Equal(t, want, got) + }) + } + }) + + t.Run("GetInterruptedPartitions", func(t *testing.T) { + for _, tc := range tests { + t.Run(tc.name, func(t *testing.T) { + got, err := tc.s.GetInterruptedPartitions(t.Context()) + require.NoError(t, err) + + // Should return partitions in SCHEDULED or RUNNING state + // ordered by creation time and start timestamp ascending + want := []PartitionMetadata{ + pm("scheduled", ts.Add(time.Second), StateScheduled), + pm("running", ts.Add(2*time.Second), StateRunning), + } + + assert.Equal(t, want, got) + }) + } + }) + + t.Run("Create", func(t *testing.T) { + for _, tc := range tests { + t.Run(tc.name, func(t *testing.T) { + err := tc.s.Create(t.Context(), []PartitionMetadata{ + pm("created3", ts, StateCreated), + }) + require.NoError(t, err) + + err = tc.s.Create(t.Context(), []PartitionMetadata{ + pm("created3", ts.Add(time.Second), StateCreated), + pm("created4", ts.Add(time.Second), StateCreated), + }) + assert.Equal(t, codes.AlreadyExists, spanner.ErrCode(err)) + }) + } + }) + + t.Run("UpdateToScheduled", func(t *testing.T) { + for _, tc := range tests { + t.Run(tc.name, func(t *testing.T) { + partitionForToken := func(token string) PartitionMetadata { + t.Helper() + pm, err := tc.s.GetPartition(t.Context(), token) + require.NoError(t, err) + return pm + } + + // Before UpdateToScheduled: + pms := partitionForToken("scheduled") + pmr := partitionForToken("running") + + commitTs, err := tc.s.UpdateToScheduled(t.Context(), []string{"created1", "scheduled", "running"}) + require.NoError(t, err) + assert.False(t, commitTs.IsZero()) + + // created1 + { + pm, err := tc.s.GetPartition(t.Context(), "created1") + require.NoError(t, err) + assert.Equal(t, StateScheduled, pm.State) + assert.NotNil(t, pm.ScheduledAt) + assert.Equal(t, commitTs, *pm.ScheduledAt) + } + + // scheduled + { + pm, err := tc.s.GetPartition(t.Context(), "scheduled") + require.NoError(t, err) + assert.Equal(t, pms, pm) + } + + // running + { + pm, err := tc.s.GetPartition(t.Context(), "running") + require.NoError(t, err) + assert.Equal(t, pmr, pm) + } + }) + } + }) + + t.Run("CheckPartitionsFinished", func(t *testing.T) { + for _, tc := range tests { + t.Run(tc.name, func(t *testing.T) { + subtests := []struct { + name string + partitions []string + expectResult bool + errorContains string + }{ + { + name: "all finished", + partitions: []string{"finished"}, + expectResult: true, + }, + { + name: "mixed states", + partitions: []string{"finished", "running"}, + expectResult: false, + }, + { + name: "empty list", + partitions: []string{}, + expectResult: true, + }, + { + name: "non-existent", + partitions: []string{"nonexistent"}, + expectResult: false, + }, + } + + for _, st := range subtests { + t.Run(st.name, func(t *testing.T) { + result, err := tc.s.CheckPartitionsFinished(t.Context(), st.partitions) + require.NoError(t, err) + assert.Equal(t, st.expectResult, result) + }) + } + }) + } + }) + + t.Run("UpdateWatermark", func(t *testing.T) { + for _, tc := range tests { + t.Run(tc.name, func(t *testing.T) { + want := ts.Add(5 * time.Minute) + err := tc.s.UpdateWatermark(t.Context(), "created1", want) + require.NoError(t, err) + + got, err := tc.s.GetPartition(t.Context(), "created1") + require.NoError(t, err) + + assert.Equal(t, want, got.Watermark) + }) + } + }) + + t.Run("DeletePartitionMetadataTableWithDatabaseAdminClient", func(t *testing.T) { + for _, tc := range tests { + t.Run(tc.name, func(t *testing.T) { + require.NoError(t, CreatePartitionMetadataTableWithDatabaseAdminClient(t.Context(), tc.s.conf, e.DatabaseAdminClient)) + }) + } + }) +} + +func realTestSore(t *testing.T, r changestreamstest.RealHelper) *Store { + s, err := NewStore(StoreConfig{ + ProjectID: r.ProjectID(), + InstanceID: r.InstanceID(), + DatabaseID: r.DatabaseID(), + TableNames: RandomTableNames(r.DatabaseID()), + Dialect: databasepb.DatabaseDialect_GOOGLE_STANDARD_SQL, + }, r.Client()) + require.NoError(t, err) + return s +} + +func TestIntegrationRealStore(t *testing.T) { + integration.CheckSkip(t) + + changestreamstest.CheckSkipReal(t) + + r := changestreamstest.MakeRealHelper(t) + defer r.Close() + s := realTestSore(t, r) + + require.NoError(t, + CreatePartitionMetadataTableWithDatabaseAdminClient(t.Context(), s.conf, r.DatabaseAdminClient())) + + defer func() { + if err := DeletePartitionMetadataTableWithDatabaseAdminClient( + context.Background(), s.conf, r.DatabaseAdminClient()); err != nil { //nolint:usetesting // use context.Background + t.Log(err) + } + }() + + t.Run("UpdateToScheduledInParallel", func(t *testing.T) { + require.NoError(t, s.Create(t.Context(), []PartitionMetadata{{ + PartitionToken: "created", + ParentTokens: []string{}, + }})) + + // Run 10 workers in parallel, all trying to update the same partition + const numWorkers = 10 + workerCommitTs := make([]time.Time, numWorkers) + + var wg sync.WaitGroup + wg.Add(numWorkers) + for i := 0; i < numWorkers; i++ { + go func(workerID int) { + defer wg.Done() + + // Each worker tries to update the same partition + commitTs, err := s.UpdateToScheduled(t.Context(), []string{"created"}) + if err != nil { + t.Errorf("Worker %d: %v", workerID, err) + return + } + workerCommitTs[workerID] = commitTs + }(i) + } + wg.Wait() + + // Verify that the partition is now in SCHEDULED state + pm, err := s.GetPartition(t.Context(), "created") + require.NoError(t, err) + assert.Equal(t, StateScheduled, pm.State) + assert.NotNil(t, pm.ScheduledAt) + + // Verify only one commit timestamp was set + var matchCount int + for i := 0; i < numWorkers; i++ { + if workerCommitTs[i].Equal(*pm.ScheduledAt) { + matchCount++ + } + } + assert.Equal(t, 1, matchCount) + }) +} diff --git a/internal/impl/gcp/enterprise/changestreams/metadata/name.go b/internal/impl/gcp/enterprise/changestreams/metadata/name.go new file mode 100644 index 0000000000..d56433a407 --- /dev/null +++ b/internal/impl/gcp/enterprise/changestreams/metadata/name.go @@ -0,0 +1,67 @@ +// Copyright 2025 Redpanda Data, Inc. +// +// Licensed as a Redpanda Enterprise file under the Redpanda Community +// License (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// https://github.com/redpanda-data/connect/blob/main/licenses/rcl.md + +package metadata + +import ( + "fmt" + "strings" + + "github.com/google/uuid" +) + +const ( + tableNameFormat = "Metadata_%s_%s" + watermarkIndexFormat = "WatermarkIdx_%s_%s" + metadataCreatedAtIndexFormat = "CreatedAtIdx_%s_%s" +) + +func genName(template, databaseID string, id uuid.UUID) string { + // maxNameLength is the maximum length for table and index names in PostgreSQL (63 bytes) + const maxNameLength = 63 + + name := fmt.Sprintf(template, databaseID, id) + name = strings.ReplaceAll(name, "-", "_") + if len(name) > maxNameLength { + return name[:maxNameLength] + } + return name +} + +// TableNames specifies table and index names to be used for metadata storage. +type TableNames struct { + TableName string + WatermarkIndexName string + CreatedAtIndexName string +} + +// RandomTableNames generates a unique name for the partition metadata table and its indexes. +// The table name will be in the form of "Metadata__". +// The watermark index will be in the form of "WatermarkIdx__". +// The createdAt / start timestamp index will be in the form of "CreatedAtIdx__". +func RandomTableNames(databaseID string) TableNames { + id := uuid.New() + return TableNames{ + TableName: genName(tableNameFormat, databaseID, id), + WatermarkIndexName: genName(watermarkIndexFormat, databaseID, id), + CreatedAtIndexName: genName(metadataCreatedAtIndexFormat, databaseID, id), + } +} + +// TableNamesFromExistingTable encapsulates a selected table name. +// Index names are generated, but will only be used if the given table does not exist. +// The watermark index will be in the form of "WatermarkIdx__". +// The createdAt / start timestamp index will be in the form of "CreatedAtIdx__". +func TableNamesFromExistingTable(databaseID, table string) TableNames { + id := uuid.New() + return TableNames{ + TableName: table, + WatermarkIndexName: genName(watermarkIndexFormat, databaseID, id), + CreatedAtIndexName: genName(metadataCreatedAtIndexFormat, databaseID, id), + } +} diff --git a/internal/impl/gcp/enterprise/changestreams/metadata/name_test.go b/internal/impl/gcp/enterprise/changestreams/metadata/name_test.go new file mode 100644 index 0000000000..c9509e282a --- /dev/null +++ b/internal/impl/gcp/enterprise/changestreams/metadata/name_test.go @@ -0,0 +1,61 @@ +// Copyright 2025 Redpanda Data, Inc. +// +// Licensed as a Redpanda Enterprise file under the Redpanda Community +// License (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// https://github.com/redpanda-data/connect/blob/main/licenses/rcl.md + +package metadata + +import ( + "testing" + + "github.com/stretchr/testify/assert" +) + +func TestRandomTableNamesRemovesHyphens(t *testing.T) { + databaseID := "my-database-id-12345" + + names1 := RandomTableNames(databaseID) + assert.NotContains(t, names1.TableName, "-") + assert.NotContains(t, names1.WatermarkIndexName, "-") + assert.NotContains(t, names1.CreatedAtIndexName, "-") + + names2 := RandomTableNames(databaseID) + assert.NotEqual(t, names1.TableName, names2.TableName) + assert.NotEqual(t, names1.WatermarkIndexName, names2.WatermarkIndexName) + assert.NotEqual(t, names1.CreatedAtIndexName, names2.CreatedAtIndexName) +} + +func TestRandomTableNamesIsShorterThanMaxLength(t *testing.T) { + // maxNameLength is the maximum length for table and index names in PostgreSQL (63 bytes) + const maxNameLength = 63 + + longDatabaseID := "my-database-id-larger-than-maximum-length-1234567890-1234567890-1234567890" + names := RandomTableNames(longDatabaseID) + assert.LessOrEqual(t, len(names.TableName), maxNameLength) + assert.LessOrEqual(t, len(names.WatermarkIndexName), maxNameLength) + assert.LessOrEqual(t, len(names.CreatedAtIndexName), maxNameLength) + + shortDatabaseID := "d" + names = RandomTableNames(shortDatabaseID) + assert.LessOrEqual(t, len(names.TableName), maxNameLength) + assert.LessOrEqual(t, len(names.WatermarkIndexName), maxNameLength) + assert.LessOrEqual(t, len(names.CreatedAtIndexName), maxNameLength) +} + +func TestTableNamesFromExistingTable(t *testing.T) { + databaseID := "databaseid" + tableName := "mytable" + + names1 := TableNamesFromExistingTable(databaseID, tableName) + assert.Equal(t, tableName, names1.TableName) + assert.NotContains(t, names1.WatermarkIndexName, "-") + assert.NotContains(t, names1.CreatedAtIndexName, "-") + + names2 := TableNamesFromExistingTable(databaseID, tableName) + assert.Equal(t, tableName, names2.TableName) + assert.NotEqual(t, names1.WatermarkIndexName, names2.WatermarkIndexName) + assert.NotEqual(t, names1.CreatedAtIndexName, names2.CreatedAtIndexName) +} diff --git a/internal/impl/gcp/enterprise/changestreams/metrics.go b/internal/impl/gcp/enterprise/changestreams/metrics.go new file mode 100644 index 0000000000..124ecc1b52 --- /dev/null +++ b/internal/impl/gcp/enterprise/changestreams/metrics.go @@ -0,0 +1,128 @@ +// Copyright 2025 Redpanda Data, Inc. +// +// Licensed as a Redpanda Enterprise file under the Redpanda Community +// License (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// https://github.com/redpanda-data/connect/blob/main/licenses/rcl.md + +package changestreams + +import ( + "time" + + "github.com/redpanda-data/benthos/v4/public/service" +) + +// Metrics contains counters and timers for tracking Spanner CDC operations. +type Metrics struct { + // partitionRecordCreatedCount tracks the total number of partitions created + // during connector execution. + partitionRecordCreatedCount *service.MetricCounter + // partitionRecordRunningCount tracks the total number of partitions that + // have started processing. + partitionRecordRunningCount *service.MetricCounter + // partitionRecordFinishedCount tracks the total number of partitions that + // have completed processing. + partitionRecordFinishedCount *service.MetricCounter + // partitionRecordSplitCount tracks the total number of partition splits + // identified during execution. + partitionRecordSplitCount *service.MetricCounter + // partitionRecordMergeCount tracks the total number of partition merges + // identified during execution. + partitionRecordMergeCount *service.MetricCounter + // partitionCreatedToScheduled measures time (ns) for partitions to + // transition from CREATED to SCHEDULED state. + partitionCreatedToScheduled *service.MetricTimer + // partitionScheduledToRunning measures time (ns) for partitions to + // transition from SCHEDULED to RUNNING state. + partitionScheduledToRunning *service.MetricTimer + // queryCount tracks the total number of queries issued to Spanner during + // connector execution. + queryCount *service.MetricCounter + // dataChangeRecordCount tracks the total number of data change records processed. + dataChangeRecordCount *service.MetricCounter + // dataChangeRecordCommittedToEmitted counts records processing latency. + dataChangeRecordCommittedToEmitted *service.MetricTimer + // heartbeatRecordCount tracks the total number of heartbeat records received. + heartbeatRecordCount *service.MetricCounter + + streamID string +} + +const metricsStreamIDLabel = "stream" + +// NewMetrics creates a new Metrics instance using the provided service Metrics. +func NewMetrics(m *service.Metrics, streamID string) *Metrics { + return &Metrics{ + partitionRecordCreatedCount: m.NewCounter("spanner_cdc_partition_record_created_count", metricsStreamIDLabel), + partitionRecordRunningCount: m.NewCounter("spanner_cdc_partition_record_running_count", metricsStreamIDLabel), + partitionRecordFinishedCount: m.NewCounter("spanner_cdc_partition_record_finished_count", metricsStreamIDLabel), + partitionRecordSplitCount: m.NewCounter("spanner_cdc_partition_record_split_count", metricsStreamIDLabel), + partitionRecordMergeCount: m.NewCounter("spanner_cdc_partition_record_merge_count", metricsStreamIDLabel), + partitionCreatedToScheduled: m.NewTimer("spanner_cdc_partition_created_to_scheduled_ns", metricsStreamIDLabel), + partitionScheduledToRunning: m.NewTimer("spanner_cdc_partition_scheduled_to_running_ns", metricsStreamIDLabel), + queryCount: m.NewCounter("spanner_cdc_query_count", metricsStreamIDLabel), + dataChangeRecordCount: m.NewCounter("spanner_cdc_data_change_record_count", metricsStreamIDLabel), + dataChangeRecordCommittedToEmitted: m.NewTimer("spanner_cdc_data_change_record_committed_to_emitted_ns", metricsStreamIDLabel), + heartbeatRecordCount: m.NewCounter("spanner_cdc_heartbeat_record_count", metricsStreamIDLabel), + + streamID: streamID, + } +} + +// IncPartitionRecordCreatedCount increments the partition record created counter. +func (m *Metrics) IncPartitionRecordCreatedCount(n int) { + m.partitionRecordCreatedCount.Incr(int64(n), m.streamID) +} + +// IncPartitionRecordRunningCount increments the partition record running counter. +func (m *Metrics) IncPartitionRecordRunningCount() { + m.partitionRecordRunningCount.Incr(1, m.streamID) +} + +// IncPartitionRecordFinishedCount increments the partition record finished counter. +func (m *Metrics) IncPartitionRecordFinishedCount() { + m.partitionRecordFinishedCount.Incr(1, m.streamID) +} + +// IncPartitionRecordSplitCount increments the partition record split counter. +func (m *Metrics) IncPartitionRecordSplitCount() { + m.partitionRecordSplitCount.Incr(1, m.streamID) +} + +// IncPartitionRecordMergeCount increments the partition record merge counter. +func (m *Metrics) IncPartitionRecordMergeCount() { + m.partitionRecordMergeCount.Incr(1, m.streamID) +} + +// UpdatePartitionCreatedToScheduled records the time taken for a partition to transition from created to scheduled state. +func (m *Metrics) UpdatePartitionCreatedToScheduled(d time.Duration) { + m.partitionCreatedToScheduled.Timing(d.Nanoseconds(), m.streamID) +} + +// UpdatePartitionScheduledToRunning records the time taken for a partition to transition from scheduled to running state. +func (m *Metrics) UpdatePartitionScheduledToRunning(d time.Duration) { + m.partitionScheduledToRunning.Timing(d.Nanoseconds(), m.streamID) +} + +// IncQueryCount increments the query counter. +func (m *Metrics) IncQueryCount() { + m.queryCount.Incr(1, m.streamID) +} + +// IncDataChangeRecordCount increments the data change record counter. +func (m *Metrics) IncDataChangeRecordCount() { + m.dataChangeRecordCount.Incr(1, m.streamID) +} + +// UpdateDataChangeRecordCommittedToEmitted records the latency of a data change +// record in the appropriate bucket. +func (m *Metrics) UpdateDataChangeRecordCommittedToEmitted(d time.Duration) { + m.dataChangeRecordCommittedToEmitted.Timing(d.Nanoseconds(), m.streamID) +} + +// IncHeartbeatRecordCount increments the heartbeat record counter. +func (m *Metrics) IncHeartbeatRecordCount() { + m.heartbeatRecordCount.Incr(1, m.streamID) +} diff --git a/internal/impl/gcp/enterprise/changestreams/model.go b/internal/impl/gcp/enterprise/changestreams/model.go new file mode 100644 index 0000000000..9cdac12f39 --- /dev/null +++ b/internal/impl/gcp/enterprise/changestreams/model.go @@ -0,0 +1,217 @@ +// Copyright 2025 Redpanda Data, Inc. +// +// Licensed as a Redpanda Enterprise file under the Redpanda Community +// License (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// https://github.com/redpanda-data/connect/blob/main/licenses/rcl.md + +package changestreams + +import ( + "fmt" + "strings" + "time" + + "cloud.google.com/go/spanner" + + "github.com/redpanda-data/connect/v4/internal/impl/gcp/enterprise/changestreams/metadata" +) + +// ChangeRecord is the single unit of the records from the change stream. +// See https://cloud.google.com/spanner/docs/change-streams/details#change_streams_record_format +type ChangeRecord struct { + DataChangeRecords []*DataChangeRecord `spanner:"data_change_record" json:"data_change_record"` + HeartbeatRecords []*HeartbeatRecord `spanner:"heartbeat_record" json:"heartbeat_record"` + ChildPartitionsRecords []*ChildPartitionsRecord `spanner:"child_partitions_record" json:"child_partitions_record"` +} + +func (cr *ChangeRecord) String() string { + var ( + b strings.Builder + c = false + ) + b.WriteString("ChangeRecord{") + if len(cr.DataChangeRecords) > 0 { + b.WriteString("DataChangeRecords: ") + b.WriteString(fmt.Sprintf("%+v", cr.DataChangeRecords)) + c = true + } + if len(cr.HeartbeatRecords) > 0 { + if c { + b.WriteString(", ") + } + b.WriteString("HeartbeatRecords: ") + b.WriteString(fmt.Sprintf("%+v", cr.HeartbeatRecords)) + c = true + } + if len(cr.ChildPartitionsRecords) > 0 { + if c { + b.WriteString(", ") + } + b.WriteString("ChildPartitionsRecords: ") + b.WriteString(fmt.Sprintf("%+v", cr.ChildPartitionsRecords)) + } + b.WriteString("}") + return b.String() +} + +// DataChangeRecord contains a set of changes to the table with the same +// modification type (insert, update, or delete) committed at the same +// CommitTimestamp in one change stream partition for the same transaction. +// Multiple data change records can be returned for the same transaction across +// multiple change stream partitions. +// +// All data change records have CommitTimestamp, ServerTransactionID, +// and RecordSequence fields, which together determine the order in the change +// stream for a stream record. These three fields are sufficient to derive +// the ordering of changes and provide external consistency. +// +// Note that multiple transactions can have the same commit timestamp +// if they touch non-overlapping data. The ServerTransactionID field offers +// the ability to distinguish which set of changes (potentially across change +// stream partitions) were issued within the same transaction. Pairing it with +// the RecordSequence and NumberOfRecordsInTransaction fields allows you to +// buffer and order all the records from a particular transaction, as well. +// +// See https://cloud.google.com/spanner/docs/change-streams/details#data-change-records +type DataChangeRecord struct { + CommitTimestamp time.Time `spanner:"commit_timestamp" json:"commit_timestamp"` + RecordSequence string `spanner:"record_sequence" json:"record_sequence"` + ServerTransactionID string `spanner:"server_transaction_id" json:"server_transaction_id"` + IsLastRecordInTransactionInPartition bool `spanner:"is_last_record_in_transaction_in_partition" json:"is_last_record_in_transaction_in_partition"` + TableName string `spanner:"table_name" json:"table_name"` + ColumnTypes []*ColumnType `spanner:"column_types" json:"column_types"` + Mods []*Mod `spanner:"mods" json:"mods"` + ModType string `spanner:"mod_type" json:"mod_type"` + ValueCaptureType string `spanner:"value_capture_type" json:"value_capture_type"` + NumberOfRecordsInTransaction int64 `spanner:"number_of_records_in_transaction" json:"number_of_records_in_transaction"` + NumberOfPartitionsInTransaction int64 `spanner:"number_of_partitions_in_transaction" json:"number_of_partitions_in_transaction"` + TransactionTag string `spanner:"transaction_tag" json:"transaction_tag"` + IsSystemTransaction bool `spanner:"is_system_transaction" json:"is_system_transaction"` +} + +// String implements the fmt.Stringer interface for DataChangeRecord. +func (dcr *DataChangeRecord) String() string { + return fmt.Sprintf("DataChangeRecord{CommitTimestamp: %v, RecordSequence: %s, ServerTransactionID: %s, "+ + "IsLastRecordInTransactionInPartition: %v, TableName: %s, ColumnTypes: %+v, Mods: %+v, ModType: %s, "+ + "ValueCaptureType: %s, NumberOfRecordsInTransaction: %d, NumberOfPartitionsInTransaction: %d, "+ + "TransactionTag: %s, IsSystemTransaction: %v}", + dcr.CommitTimestamp, dcr.RecordSequence, dcr.ServerTransactionID, + dcr.IsLastRecordInTransactionInPartition, dcr.TableName, dcr.ColumnTypes, dcr.Mods, dcr.ModType, + dcr.ValueCaptureType, dcr.NumberOfRecordsInTransaction, dcr.NumberOfPartitionsInTransaction, + dcr.TransactionTag, dcr.IsSystemTransaction) +} + +// ColumnType is the metadata of the column. +type ColumnType struct { + Name string `spanner:"name" json:"name"` + Type spanner.NullJSON `spanner:"type" json:"type"` + IsPrimaryKey bool `spanner:"is_primary_key" json:"is_primary_key"` + OrdinalPosition int64 `spanner:"ordinal_position" json:"ordinal_position"` +} + +// String implements the fmt.Stringer interface for ColumnType. +func (ct *ColumnType) String() string { + return fmt.Sprintf("ColumnType{Name: %s, Type: %+v, IsPrimaryKey: %v, OrdinalPosition: %d}", + ct.Name, ct.Type, ct.IsPrimaryKey, ct.OrdinalPosition) +} + +// Mod is the changes that were made on the table. +// See https://cloud.google.com/spanner/docs/change-streams/details#heartbeat-records +type Mod struct { + Keys spanner.NullJSON `spanner:"keys" json:"keys"` + NewValues spanner.NullJSON `spanner:"new_values" json:"new_values"` + OldValues spanner.NullJSON `spanner:"old_values" json:"old_values"` +} + +// String implements the fmt.Stringer interface for Mod. +func (m *Mod) String() string { + return fmt.Sprintf("Mod{Keys: %+v, NewValues: %+v, OldValues: %+v}", + m.Keys, m.NewValues, m.OldValues) +} + +// HeartbeatRecord is the heartbeat record returned from Cloud Spanner. +// +// When a heartbeat record is returned, it indicates that all changes with +// CommitTimestamp less than or equal to the heartbeat record's Timestamp have +// been returned, and future data records in this partition must have higher +// commit timestamps than that returned by the heartbeat record. +// +// Heartbeat records are returned when there are no data changes written to +// a partition. When there are data changes written to the partition, +// DataChangeRecord.CommitTimestamp can be used instead of +// HeartbeatRecord.Timestamp to tell that the reader is making forward +// progress in reading the partition. +// +// You can use heartbeat records returned on partitions to synchronize readers +// across all partitions. Once all readers have received either a heartbeat +// greater than or equal to some timestamp A or have received data or child +// partition records greater than or equal to timestamp A, the readers know they +// have received all records committed at or before that timestamp A and can +// start processing the buffered records—for example, sorting the +// cross-partition records by timestamp and grouping them by ServerTransactionID. +// +// See https://cloud.google.com/spanner/docs/change-streams/details#heartbeat-records +type HeartbeatRecord struct { + Timestamp time.Time `spanner:"timestamp" json:"timestamp"` +} + +// String implements the fmt.Stringer interface for HeartbeatRecord. +func (hr *HeartbeatRecord) String() string { + return fmt.Sprintf("HeartbeatRecord{Timestamp: %v}", hr.Timestamp) +} + +// ChildPartitionsRecord contains information about child partitions: +// their partition tokens, the tokens of their parent partitions, +// and the StartTimestamp that represents the earliest timestamp that the child +// partitions contain change records for. Records whose commit timestamps are +// immediately prior to the StartTimestamp are returned in the current partition. +// +// See https://cloud.google.com/spanner/docs/change-streams/details#child-partitions-records +type ChildPartitionsRecord struct { + StartTimestamp time.Time `spanner:"start_timestamp" json:"start_timestamp"` + RecordSequence string `spanner:"record_sequence" json:"record_sequence"` + ChildPartitions []*ChildPartition `spanner:"child_partitions" json:"child_partitions"` +} + +func (cpr *ChildPartitionsRecord) String() string { + return fmt.Sprintf("ChildPartitionsRecord{StartTimestamp: %v, RecordSequence: %s, ChildPartitions: %+v}", + cpr.StartTimestamp, cpr.RecordSequence, cpr.ChildPartitions) +} + +// ChildPartition contains the child partition token. +type ChildPartition struct { + Token string `spanner:"token" json:"token"` + ParentPartitionTokens []string `spanner:"parent_partition_tokens" json:"parent_partition_tokens"` +} + +func (cp *ChildPartition) String() string { + return fmt.Sprintf("ChildPartition{Token: %s, ParentPartitionTokens: %+v}", + cp.Token, cp.ParentPartitionTokens) +} + +// toPartitionMetadata converts a ChildPartition to a PartitionMetadata. +// The startTimestamp is taken from the ChildPartitionsRecord.StartTimestamp, +// and represents the earliest timestamp that the child partitions contain +// change records for. The endTimestamp and heartbeatMillis are inherited +// from the parent partition. +func (cp *ChildPartition) toPartitionMetadata( + startTimestamp, + endTimestamp time.Time, + heartbeatMillis int64, +) metadata.PartitionMetadata { + return metadata.PartitionMetadata{ + PartitionToken: cp.Token, + ParentTokens: cp.ParentPartitionTokens, + StartTimestamp: startTimestamp, + EndTimestamp: endTimestamp, + HeartbeatMillis: heartbeatMillis, + State: metadata.StateCreated, + Watermark: startTimestamp, + } +} + +func (cp *ChildPartition) isSplit() bool { + return len(cp.ParentPartitionTokens) == 1 +} diff --git a/internal/impl/gcp/enterprise/changestreams/model_pg.go b/internal/impl/gcp/enterprise/changestreams/model_pg.go new file mode 100644 index 0000000000..06cbfa151c --- /dev/null +++ b/internal/impl/gcp/enterprise/changestreams/model_pg.go @@ -0,0 +1,51 @@ +// Copyright 2025 Redpanda Data, Inc. +// +// Licensed as a Redpanda Enterprise file under the Redpanda Community +// License (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// https://github.com/redpanda-data/connect/blob/main/licenses/rcl.md + +package changestreams + +import ( + "encoding/json" + "fmt" + + "cloud.google.com/go/spanner" +) + +var emptyChangeRecord = ChangeRecord{} + +func decodePostgresRow(row *spanner.Row) (ChangeRecord, error) { + var col spanner.NullJSON + if err := row.Column(0, &col); err != nil { + return emptyChangeRecord, fmt.Errorf("extract column from row: %w", err) + } + + b, err := col.MarshalJSON() + if err != nil { + return emptyChangeRecord, fmt.Errorf("marshal JSON column: %w", err) + } + + var pgcr struct { + DataChangeRecord *DataChangeRecord `json:"data_change_record"` + HeartbeatRecord *HeartbeatRecord `json:"heartbeat_record"` + ChildPartitionsRecord *ChildPartitionsRecord `json:"child_partitions_record"` + } + if err := json.Unmarshal(b, &pgcr); err != nil { + return emptyChangeRecord, fmt.Errorf("unmarshal JSON data: %w", err) + } + + var cr ChangeRecord + if pgcr.DataChangeRecord != nil { + cr.DataChangeRecords = []*DataChangeRecord{pgcr.DataChangeRecord} + } + if pgcr.HeartbeatRecord != nil { + cr.HeartbeatRecords = []*HeartbeatRecord{pgcr.HeartbeatRecord} + } + if pgcr.ChildPartitionsRecord != nil { + cr.ChildPartitionsRecords = []*ChildPartitionsRecord{pgcr.ChildPartitionsRecord} + } + return cr, nil +} diff --git a/internal/impl/gcp/enterprise/changestreams/model_pg_test.go b/internal/impl/gcp/enterprise/changestreams/model_pg_test.go new file mode 100644 index 0000000000..af6d908856 --- /dev/null +++ b/internal/impl/gcp/enterprise/changestreams/model_pg_test.go @@ -0,0 +1,215 @@ +// Copyright 2025 Redpanda Data, Inc. +// +// Licensed as a Redpanda Enterprise file under the Redpanda Community +// License (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// https://github.com/redpanda-data/connect/blob/main/licenses/rcl.md +// +// Copyright 2022 Google LLC +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. +// + +package changestreams + +import ( + "encoding/json" + "fmt" + "testing" + "time" + + "cloud.google.com/go/spanner" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +func TestDecodePostgresRow(t *testing.T) { + tests := []struct { + desc string + changeRecordJSON string + want ChangeRecord + }{ + { + desc: "child partitions record", + changeRecordJSON: ` +{ + "child_partitions_record": { + "start_timestamp": "2023-02-24T01:06:48.000000-08:00", + "record_sequence": "00000001", + "child_partitions": [ + { + "token": "__8BAYEG0qQD8AABgsBDg3BsYXllcnNzdHJlYW0AAYSBAIKAgwjDZAAAAAAAbYQEHbHBFIVnMF8wAAH__4X_BfVuWWHW8ob_BfVu9ITI-Yf_BfVuWWHW8sBkAQH__w", + "parent_partition_tokens": [] + } + ] + } +}`, + want: ChangeRecord{ + ChildPartitionsRecords: []*ChildPartitionsRecord{ + { + StartTimestamp: mustParseTime("2023-02-24T01:06:48.000000-08:00"), + RecordSequence: "00000001", + ChildPartitions: []*ChildPartition{ + { + Token: "__8BAYEG0qQD8AABgsBDg3BsYXllcnNzdHJlYW0AAYSBAIKAgwjDZAAAAAAAbYQEHbHBFIVnMF8wAAH__4X_BfVuWWHW8ob_BfVu9ITI-Yf_BfVuWWHW8sBkAQH__w", + ParentPartitionTokens: []string{}, + }, + }, + }, + }, + }, + }, + { + desc: "data change record", + changeRecordJSON: ` +{ + "data_change_record": { + "column_types": [ + { + "is_primary_key": true, + "name": "playerid", + "ordinal_position": 1, + "type": { + "code": "INT64" + } + }, + { + "is_primary_key": false, + "name": "playername", + "ordinal_position": 2, + "type": { + "code": "STRING" + } + } + ], + "commit_timestamp": "2023-02-24T17:17:00.678847-08:00", + "is_last_record_in_transaction_in_partition": true, + "is_system_transaction": false, + "mod_type": "INSERT", + "mods": [ + { + "keys": { + "playerid": "3" + }, + "new_values": { + "playername": "b" + }, + "old_values": {} + } + ], + "number_of_partitions_in_transaction": 1, + "number_of_records_in_transaction": 1, + "record_sequence": "00000000", + "server_transaction_id": "NTQ5MTAxNjk2MzM2OTMxOTM5NQ==", + "table_name": "players", + "transaction_tag": "", + "value_capture_type": "OLD_AND_NEW_VALUES" + } +}`, + want: ChangeRecord{ + DataChangeRecords: []*DataChangeRecord{ + { + CommitTimestamp: mustParseTime("2023-02-24T17:17:00.678847-08:00"), + IsLastRecordInTransactionInPartition: true, + IsSystemTransaction: false, + ModType: "INSERT", + NumberOfRecordsInTransaction: 1, + NumberOfPartitionsInTransaction: 1, + RecordSequence: "00000000", + ServerTransactionID: "NTQ5MTAxNjk2MzM2OTMxOTM5NQ==", + TableName: "players", + TransactionTag: "", + ValueCaptureType: "OLD_AND_NEW_VALUES", + ColumnTypes: []*ColumnType{ + { + Name: "playerid", + Type: spanner.NullJSON{ + Value: map[string]interface{}{"code": "INT64"}, + Valid: true, + }, + IsPrimaryKey: true, + OrdinalPosition: 1, + }, + { + Name: "playername", + Type: spanner.NullJSON{ + Value: map[string]interface{}{"code": "STRING"}, + Valid: true, + }, + IsPrimaryKey: false, + OrdinalPosition: 2, + }, + }, + Mods: []*Mod{ + { + Keys: spanner.NullJSON{ + Value: map[string]interface{}{"playerid": "3"}, + Valid: true, + }, + NewValues: spanner.NullJSON{ + Value: map[string]interface{}{"playername": "b"}, + Valid: true, + }, + OldValues: spanner.NullJSON{ + Value: map[string]interface{}{}, + Valid: true, + }, + }, + }, + }, + }, + }, + }, + { + desc: "heartbeat record", + changeRecordJSON: ` +{ + "heartbeat_record": { + "timestamp": "2023-02-24T17:16:43.811345-08:00" + } +}`, + want: ChangeRecord{ + HeartbeatRecords: []*HeartbeatRecord{ + { + Timestamp: mustParseTime("2023-02-24T17:16:43.811345-08:00"), + }, + }, + }, + }, + } + for _, test := range tests { + t.Run(test.desc, func(t *testing.T) { + var jsonVal interface{} + require.NoError(t, json.Unmarshal([]byte(test.changeRecordJSON), &jsonVal)) + + row, err := spanner.NewRow([]string{"read_json_playersstream"}, []interface{}{spanner.NullJSON{ + Valid: true, + Value: jsonVal, + }}) + require.NoError(t, err) + + got, err := decodePostgresRow(row) + require.NoError(t, err) + assert.Equal(t, test.want, got) + }) + } +} + +func mustParseTime(value string) time.Time { + t, err := time.Parse(time.RFC3339Nano, value) + if err != nil { + panic(fmt.Sprintf("invalid time %q: %v", value, err)) + } + return t +} diff --git a/internal/impl/gcp/enterprise/changestreams/querier.go b/internal/impl/gcp/enterprise/changestreams/querier.go new file mode 100644 index 0000000000..d915c7504f --- /dev/null +++ b/internal/impl/gcp/enterprise/changestreams/querier.go @@ -0,0 +1,123 @@ +// Copyright 2025 Redpanda Data, Inc. +// +// Licensed as a Redpanda Enterprise file under the Redpanda Community +// License (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// https://github.com/redpanda-data/connect/blob/main/licenses/rcl.md + +package changestreams + +import ( + "context" + "errors" + "fmt" + + "cloud.google.com/go/spanner" + "cloud.google.com/go/spanner/apiv1/spannerpb" + "google.golang.org/api/iterator" + + "github.com/redpanda-data/benthos/v4/public/service" + "github.com/redpanda-data/connect/v4/internal/impl/gcp/enterprise/changestreams/metadata" +) + +type readResult struct { + ChangeRecords []*ChangeRecord `spanner:"ChangeRecord" json:"change_record"` +} + +type querier interface { + query(ctx context.Context, pm metadata.PartitionMetadata, cb func(ctx context.Context, cr ChangeRecord) error) error +} + +type clientQuerier struct { + client *spanner.Client + dialect dialect + streamName string + priority spannerpb.RequestOptions_Priority + log *service.Logger +} + +// query executes a change stream query for the specified stream and partition. +// It processes each record from the change stream and calls the callback function. +func (q clientQuerier) query( + ctx context.Context, + pm metadata.PartitionMetadata, + cb func(ctx context.Context, cr ChangeRecord) error, +) error { + var stmt spanner.Statement + if q.isPostgres() { + stmt = spanner.Statement{ + SQL: fmt.Sprintf(`SELECT * FROM spanner.read_json_%s($1, $2, $3, $4, null)`, q.streamName), + Params: map[string]any{ + "p1": pm.Watermark, + "p2": pm.EndTimestamp, + "p3": pm.PartitionToken, + "p4": pm.HeartbeatMillis, + }, + } + // Convert to NULL + if pm.EndTimestamp.IsZero() { + stmt.Params["p2"] = nil + } + if pm.PartitionToken == "" { + stmt.Params["p3"] = nil + } + } else { + stmt = spanner.Statement{ + SQL: fmt.Sprintf(`SELECT ChangeRecord FROM READ_%s(@start_timestamp, @end_timestamp, @partition_token, @heartbeat_millis)`, q.streamName), + Params: map[string]any{ + "start_timestamp": pm.Watermark, + "end_timestamp": pm.EndTimestamp, + "partition_token": pm.PartitionToken, + "heartbeat_millis": pm.HeartbeatMillis, + }, + } + // Convert to NULL + if pm.EndTimestamp.IsZero() { + stmt.Params["end_timestamp"] = nil + } + if pm.PartitionToken == "" { + stmt.Params["partition_token"] = nil + } + } + q.log.Tracef("Executing query: %s with params: %v", stmt.SQL, stmt.Params) + + iter := q.client.Single().QueryWithOptions(ctx, stmt, spanner.QueryOptions{Priority: q.priority}) + defer iter.Stop() + + for { + row, err := iter.Next() + if err != nil { + if errors.Is(err, iterator.Done) { + break + } + return fmt.Errorf("get change stream results: %w", err) + } + + if q.isPostgres() { + cr, err := decodePostgresRow(row) + if err != nil { + return fmt.Errorf("decode postgres row: %w", err) + } + if err := cb(ctx, cr); err != nil { + return err + } + } else { + var rr readResult + if err := row.ToStruct(&rr); err != nil { + return fmt.Errorf("row to struct: %w", err) + } + for _, cr := range rr.ChangeRecords { + if err := cb(ctx, *cr); err != nil { + return err + } + } + } + } + + return nil +} + +func (q clientQuerier) isPostgres() bool { + return q.dialect == dialectPostgreSQL +} diff --git a/internal/impl/gcp/enterprise/changestreams/querier_mock_test.go b/internal/impl/gcp/enterprise/changestreams/querier_mock_test.go new file mode 100644 index 0000000000..75d1cac210 --- /dev/null +++ b/internal/impl/gcp/enterprise/changestreams/querier_mock_test.go @@ -0,0 +1,50 @@ +// Copyright 2025 Redpanda Data, Inc. +// +// Licensed as a Redpanda Enterprise file under the Redpanda Community +// License (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// https://github.com/redpanda-data/connect/blob/main/licenses/rcl.md + +package changestreams + +import ( + "context" + "fmt" + + "github.com/stretchr/testify/mock" + + "github.com/redpanda-data/connect/v4/internal/impl/gcp/enterprise/changestreams/metadata" +) + +type mockQuerier struct { + mock.Mock + expectCallbackError bool +} + +func (m *mockQuerier) query(ctx context.Context, pm metadata.PartitionMetadata, cb func(ctx context.Context, cr ChangeRecord) error) error { + args := m.Called(ctx, pm, cb) + return args.Error(0) +} + +func (m *mockQuerier) ExpectQuery(partitionToken string) *mock.Call { + return m.On("query", mock.Anything, mock.MatchedBy(func(actual metadata.PartitionMetadata) bool { + return actual.PartitionToken == partitionToken + }), mock.Anything) +} + +func (m *mockQuerier) ExpectQueryWithRecords(partitionToken string, records ...ChangeRecord) *mock.Call { + return m.ExpectQuery(partitionToken).Return(nil).Run(func(args mock.Arguments) { + ctx := args.Get(0).(context.Context) + cb := args.Get(2).(func(ctx context.Context, cr ChangeRecord) error) + for _, record := range records { + if err := cb(ctx, record); err != nil { + // We can't return an error from a Run function. + if m.expectCallbackError { + return + } + panic(fmt.Sprintf("error in callback: %v", err)) + } + } + }) +} diff --git a/internal/impl/gcp/enterprise/changestreams/subscriber.go b/internal/impl/gcp/enterprise/changestreams/subscriber.go new file mode 100644 index 0000000000..0760edb4f2 --- /dev/null +++ b/internal/impl/gcp/enterprise/changestreams/subscriber.go @@ -0,0 +1,477 @@ +// Copyright 2025 Redpanda Data, Inc. +// +// Licensed as a Redpanda Enterprise file under the Redpanda Community +// License (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// https://github.com/redpanda-data/connect/blob/main/licenses/rcl.md + +package changestreams + +import ( + "context" + "errors" + "fmt" + "time" + + "cloud.google.com/go/spanner" + adminapi "cloud.google.com/go/spanner/admin/database/apiv1" + "cloud.google.com/go/spanner/apiv1/spannerpb" + "golang.org/x/sync/errgroup" + "google.golang.org/api/option" + "google.golang.org/grpc/codes" + + "github.com/redpanda-data/benthos/v4/public/service" + + "github.com/redpanda-data/connect/v4/internal/impl/gcp/enterprise/changestreams/metadata" +) + +// Config is the configuration for a Subscriber. +type Config struct { + ProjectID string + InstanceID string + DatabaseID string + StreamID string + StartTimestamp time.Time + EndTimestamp time.Time + HeartbeatInterval time.Duration + MetadataTable string + MinWatermarkCacheTTL time.Duration + AllowedModTypes []string + + SpannerClientConfig spanner.ClientConfig + SpannerClientOptions []option.ClientOption + ChangeStreamQueryPriority spannerpb.RequestOptions_Priority +} + +// Subscriber is a partition aware Spanner change stream consumer. It reads +// change records from the stream and passes them to the provided callback. +// It persists the state of the stream partitions to the metadata table in +// order to resume from the last record processed. +// +// The watermark is updated after each record callback. Callbacks for single +// partitions are executed sequentially. Callbacks for multiple partitions are +// executed in parallel. +// +// Subscriber supports both PostgreSQL and GoogleSQL dialects. It automatically +// detects the Spanner dialect and uses the appropriate dialect in the queries. +// It creates the metadata table if it does not exist. If MetadataTable is +// not set, it uses a random table name, this should be used in tests only. +type Subscriber struct { + conf Config + client *spanner.Client + store *metadata.Store + minWatermark timeCache + querier querier + resumed map[string]struct{} + eg *errgroup.Group + cb CallbackFunc + log *service.Logger + metrics *Metrics + + testingAdminClient *adminapi.DatabaseAdminClient + testingPostFinished func(partitionToken string, err error) +} + +// NewSubscriber creates Spanner client and initializes the Subscriber. +func NewSubscriber( + ctx context.Context, + conf Config, + cb CallbackFunc, + log *service.Logger, + metrics *Metrics, +) (*Subscriber, error) { + if cb == nil { + return nil, errors.New("no callback provided") + } + + dbName := fmt.Sprintf("projects/%s/instances/%s/databases/%s", conf.ProjectID, conf.InstanceID, conf.DatabaseID) + client, err := spanner.NewClientWithConfig(ctx, dbName, conf.SpannerClientConfig, conf.SpannerClientOptions...) + if err != nil { + return nil, err + } + + dialect, err := detectDialect(ctx, client) + if err != nil { + client.Close() + return nil, fmt.Errorf("failed to detect dialect: %w", err) + } + + var tableNames metadata.TableNames + if conf.MetadataTable != "" { + tableNames = metadata.TableNamesFromExistingTable(conf.DatabaseID, conf.MetadataTable) + } else { + log.Infof("Using random table names for metadata table, this should only be used for testing") + tableNames = metadata.RandomTableNames(conf.DatabaseID) + } + + sConf := metadata.StoreConfig{ + ProjectID: conf.ProjectID, + InstanceID: conf.InstanceID, + DatabaseID: conf.DatabaseID, + Dialect: dialect, + TableNames: tableNames, + } + + if len(conf.AllowedModTypes) != 0 { + cb = filteredCallback(cb, modTypeFilter(conf.AllowedModTypes)) + } + + store, err := metadata.NewStore(sConf, client) + if err != nil { + client.Close() + return nil, fmt.Errorf("create metadata store: %w", err) + } + + return &Subscriber{ + conf: conf, + client: client, + store: store, + minWatermark: timeCache{ + d: conf.MinWatermarkCacheTTL, + now: now, + }, + querier: clientQuerier{ + client: client, + dialect: dialect, + streamName: conf.StreamID, + priority: conf.ChangeStreamQueryPriority, + log: log, + }, + cb: cb, + log: log, + metrics: metrics, + }, nil +} + +// Setup creates the metadata table and detects the root partitions. +func (s *Subscriber) Setup(ctx context.Context) error { + if err := s.createPartitionMetadataTableIfNotExist(ctx); err != nil { + return fmt.Errorf("create partition metadata table: %w", err) + } + + if err := s.detectRootPartitions(ctx); err != nil { + return fmt.Errorf("detect root partitions: %w", err) + } + + return nil +} + +func (s *Subscriber) createPartitionMetadataTableIfNotExist(ctx context.Context) error { + s.log.Debugf("Creating partition metadata table %s if not exist", s.store.Config().TableName) + + var adm *adminapi.DatabaseAdminClient + if s.testingAdminClient != nil { + adm = s.testingAdminClient + } else { + var err error + adm, err = adminapi.NewDatabaseAdminClient(ctx, s.conf.SpannerClientOptions...) + if err != nil { + return err + } + defer func() { + if err := adm.Close(); err != nil { + s.log.Warnf("Failed to close database admin client: %v", err) + } + }() + } + return metadata.CreatePartitionMetadataTableWithDatabaseAdminClient(ctx, s.store.Config(), adm) +} + +func (s *Subscriber) detectRootPartitions(ctx context.Context) error { + pm := metadata.PartitionMetadata{ + PartitionToken: "", // Empty token to query all partitions + StartTimestamp: s.conf.StartTimestamp, + EndTimestamp: s.conf.EndTimestamp, + HeartbeatMillis: s.conf.HeartbeatInterval.Milliseconds(), + Watermark: s.conf.StartTimestamp, + } + + if err := s.querier.query(ctx, pm, s.handleRootPartitions); err != nil { + return fmt.Errorf("query for root partitions: %w", err) + } + + return nil +} + +func (s *Subscriber) handleRootPartitions(ctx context.Context, cr ChangeRecord) error { + for _, cpr := range cr.ChildPartitionsRecords { + for _, cp := range cpr.ChildPartitions { + if len(cp.ParentPartitionTokens) != 0 { + s.log.Debugf("Ignoring child partition with parent partition tokens: %+v", cp.ParentPartitionTokens) + continue + } + + rpm := cp.toPartitionMetadata( + cpr.StartTimestamp, + s.conf.EndTimestamp, + s.conf.HeartbeatInterval.Milliseconds(), + ) + if err := s.store.Create(ctx, []metadata.PartitionMetadata{rpm}); err != nil { + if spanner.ErrCode(err) != codes.AlreadyExists { + return fmt.Errorf("create root partition metadata: %w", err) + } + } else { + s.log.Infof("Detected root partition %s", rpm.PartitionToken) + s.metrics.IncPartitionRecordCreatedCount(1) + } + } + } + + return nil +} + +// Run starts reading the change stream and processing partitions. It can be +// stopped by canceling the context. If EndTimestamp is set, the subscriber will +// stop when it reaches the end timestamp. Setup can resume the subscriber +// from the last record processed. +// +// Error can be returned only if rescheduling interrupted partitions fails or +// if the context is canceled. +// +// Setup must be called before Run. +func (s *Subscriber) Run(ctx context.Context) error { + s.log.Infof("Starting subscriber stream_id=%s start_timestamp=%v end_timestamp=%v", + s.conf.StreamID, + s.conf.StartTimestamp, + s.conf.EndTimestamp) + defer func() { + s.log.Info("Subscriber stopped") + }() + + s.eg, ctx = errgroup.WithContext(ctx) + + if pms, err := s.store.GetInterruptedPartitions(ctx); err != nil { + return fmt.Errorf("get interrupted partitions: %w", err) + } else if len(pms) > 0 { + s.resumed = make(map[string]struct{}, len(pms)) + for _, pm := range pms { + s.resumed[pm.PartitionToken] = struct{}{} + } + + s.log.Debugf("Detected %d interrupted partitions", len(pms)) + if err := s.schedule(ctx, pms); err != nil { + return fmt.Errorf("schedule interrupted partitions: %w", err) + } + } + + s.eg.Go(func() error { + defer func() { + s.log.Info("Waiting for all partitions to finish") + }() + return s.detectNewPartitionsLoop(ctx) + }) + + return s.eg.Wait() +} + +func (s *Subscriber) detectNewPartitionsLoop(ctx context.Context) error { + const resumeDuration = 100 * time.Millisecond + t := time.NewTimer(0) + defer t.Stop() + + for { + select { + case <-ctx.Done(): + return ctx.Err() + case <-t.C: + if err := s.detectNewPartitions(ctx); err != nil { + if isCancelled(err) { + return ctx.Err() + } + if errors.Is(err, errEndOfStream) { + s.log.Infof("No new partitions detected, exiting") + return nil + } + return fmt.Errorf("detect new partitions: %w", err) + } + t.Reset(resumeDuration) + } + } +} + +var errEndOfStream = errors.New("no new partitions") + +func (s *Subscriber) detectNewPartitions(ctx context.Context) error { + minWatermark := s.minWatermark.get() + if minWatermark.IsZero() { + var err error + minWatermark, err = s.store.GetUnfinishedMinWatermark(ctx) + if err != nil { + return fmt.Errorf("get unfinished min watermark: %w", err) + } + s.log.Debugf("Detected unfinished min watermark: %v", minWatermark) + } + if minWatermark.IsZero() { + return nil + } + s.minWatermark.set(minWatermark) + + if !s.conf.EndTimestamp.IsZero() && minWatermark.After(s.conf.EndTimestamp) { + s.log.Debugf("Min watermark is after end timestamp: %v", s.conf.EndTimestamp) + return errEndOfStream + } + + pms, err := s.store.GetPartitionsCreatedAfter(ctx, minWatermark) + if err != nil { + return err + } + if len(pms) == 0 { + return nil + } + s.log.Debugf("Detected %d new partitions", len(pms)) + + if err := s.schedule(ctx, pms); err != nil { + return fmt.Errorf("schedule partitions: %w", err) + } + + return nil +} + +func (s *Subscriber) schedule(ctx context.Context, pms []metadata.PartitionMetadata) error { + for _, g := range groupPartitionsByCreatedAt(pms) { + if _, err := s.store.UpdateToScheduled(ctx, tokensOf(g)); err != nil { + return fmt.Errorf("update partitions to scheduled: %w", err) + } + + for _, pm := range g { + s.eg.Go(func() error { + s.waitForParentPartitionsToFinish(ctx, pm) + + err := s.queryChangeStream(ctx, pm.PartitionToken) + if s.testingPostFinished != nil { + s.testingPostFinished(pm.PartitionToken, err) + } + if err != nil { + if isCancelled(err) { + return ctx.Err() + } + return fmt.Errorf("%s: query change stream: %w", pm.PartitionToken, err) + } + + return nil + }) + } + } + + return nil +} + +func tokensOf(partitions []metadata.PartitionMetadata) []string { + s := make([]string, len(partitions)) + for i, p := range partitions { + s[i] = p.PartitionToken + } + return s +} + +// groupPartitionsByCreatedAt groups partitions by their creation time. +// Partitions with different CreatedAt times will be placed in separate groups. +// It works only on partitions already sorted by CreatedAt in ascending order. +func groupPartitionsByCreatedAt(partitions []metadata.PartitionMetadata) [][]metadata.PartitionMetadata { + if len(partitions) == 0 { + return nil + } + + groups := [][]metadata.PartitionMetadata{{partitions[0]}} + cur := partitions[0].CreatedAt + + for _, p := range partitions[1:] { + if !p.CreatedAt.Equal(cur) { + groups = append(groups, []metadata.PartitionMetadata{p}) + cur = p.CreatedAt + } else { + lastIdx := len(groups) - 1 + groups[lastIdx] = append(groups[lastIdx], p) + } + } + + return groups +} + +// waitForParentPartitionsToFinish ensures that all parent partitions have +// finished processing before processing a child partition. +// +// Due to the parent-child partition lineage, in order to process changes for a +// particular key in commit timestamp order, records returned from child +// partitions should be processed only after records from all parent partitions +// have been processed. +func (s *Subscriber) waitForParentPartitionsToFinish(ctx context.Context, pm metadata.PartitionMetadata) { + for { + ok, err := s.store.CheckPartitionsFinished(ctx, pm.ParentTokens) + if err != nil { + s.log.Errorf("%s: error while checking parent partitions: %v", + pm.PartitionToken, err) + } + if ok { + return + } + + s.log.Debugf("%s: waiting for parent partitions to finish, next check in %s", + pm.PartitionToken, s.conf.HeartbeatInterval) + select { + case <-ctx.Done(): + return + case <-time.After(s.conf.HeartbeatInterval): + } + } +} + +func (s *Subscriber) queryChangeStream(ctx context.Context, partitionToken string) error { + s.log.Debugf("%s: updating partition to running", partitionToken) + ts, err := s.store.UpdateToRunning(ctx, partitionToken) + if err != nil { + return fmt.Errorf("update partition to running: %w", err) + } + + pm, err := s.store.GetPartition(ctx, partitionToken) + if err != nil { + return err + } + if pm.State != metadata.StateRunning { + return fmt.Errorf("partition is not running: %s", pm.State) + } + s.metrics.IncPartitionRecordRunningCount() + + if _, resumed := s.resumed[partitionToken]; !resumed { + if pm.RunningAt == nil || !ts.Equal(*pm.RunningAt) { + return fmt.Errorf("partition is already running: %s", pm.RunningAt) + } + s.metrics.UpdatePartitionCreatedToScheduled(pm.ScheduledAt.Sub(pm.CreatedAt)) + s.metrics.UpdatePartitionScheduledToRunning(pm.RunningAt.Sub(*pm.ScheduledAt)) + } + + h := s.partitionMetadataHandler(pm) + + s.log.Debugf("%s: querying partition change stream", partitionToken) + s.metrics.IncQueryCount() + if err := s.querier.query(ctx, pm, h.handleChangeRecord); err != nil { + return fmt.Errorf("process partition change stream: %w", err) + } + if err := s.cb(ctx, partitionToken, nil); err != nil { + return fmt.Errorf("end of partition: %w", err) + } + s.log.Debugf("%s: done querying partition change stream", partitionToken) + + s.log.Debugf("%s: updating partition to finished", partitionToken) + if err := s.store.UpdateWatermark(ctx, partitionToken, h.watermark()); err != nil { + return fmt.Errorf("update watermark: %w", err) + } + if _, err := s.store.UpdateToFinished(ctx, partitionToken); err != nil { + return fmt.Errorf("update partition to finished: %w", err) + } + + s.metrics.IncPartitionRecordFinishedCount() + + return nil +} + +func (s *Subscriber) Close() { + s.client.Close() +} + +func isCancelled(err error) bool { + return errors.Is(err, context.Canceled) || + errors.Is(err, context.DeadlineExceeded) || + spanner.ErrCode(err) == codes.Canceled +} diff --git a/internal/impl/gcp/enterprise/changestreams/subscriber_integration_test.go b/internal/impl/gcp/enterprise/changestreams/subscriber_integration_test.go new file mode 100644 index 0000000000..402452a35d --- /dev/null +++ b/internal/impl/gcp/enterprise/changestreams/subscriber_integration_test.go @@ -0,0 +1,632 @@ +// Copyright 2025 Redpanda Data, Inc. +// +// Licensed as a Redpanda Enterprise file under the Redpanda Community +// License (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// https://github.com/redpanda-data/connect/blob/main/licenses/rcl.md + +package changestreams + +import ( + "context" + "errors" + "log/slog" + "os" + "testing" + "time" + + "github.com/stretchr/testify/mock" + "google.golang.org/api/option" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + + "github.com/redpanda-data/benthos/v4/public/service" + "github.com/redpanda-data/benthos/v4/public/service/integration" + + "github.com/redpanda-data/connect/v4/internal/impl/gcp/enterprise/changestreams/changestreamstest" + "github.com/redpanda-data/connect/v4/internal/impl/gcp/enterprise/changestreams/metadata" +) + +var ( + testStartTimestamp = time.Now().UTC().Truncate(time.Microsecond) + rootPartitionMetadata = metadata.PartitionMetadata{ + PartitionToken: "", // Empty token to query all partitions + StartTimestamp: testStartTimestamp, + EndTimestamp: time.Time{}, + HeartbeatMillis: 1000, + Watermark: testStartTimestamp, + } + testPartitionToken = "partition0" +) + +func testPartitionMetadata(token string) metadata.PartitionMetadata { + return metadata.PartitionMetadata{ + PartitionToken: token, + ParentTokens: []string{}, + StartTimestamp: testStartTimestamp, + Watermark: testStartTimestamp, + } +} + +func testSubscriber( + t *testing.T, + e changestreamstest.EmulatorHelper, + cb CallbackFunc, + opts ...func(*Config), +) (*Subscriber, *metadata.Store, *mockQuerier) { + t.Helper() + + const databaseID = "test" + e.CreateTestDatabase(databaseID) + + conf := Config{ + ProjectID: changestreamstest.EmulatorProjectID, + InstanceID: changestreamstest.EmulatorInstanceID, + DatabaseID: databaseID, + StreamID: "test-stream", + StartTimestamp: testStartTimestamp, + EndTimestamp: time.Time{}, // No end timestamp + HeartbeatInterval: time.Second, + + SpannerClientOptions: []option.ClientOption{ + option.WithGRPCConn(e.Conn()), + }, + } + for _, o := range opts { + o(&conf) + } + + if cb == nil { + cb = func(_ context.Context, _ string, _ *DataChangeRecord) error { return nil } + } + + log := service.NewLoggerFromSlog(slog.New(slog.NewTextHandler(os.Stdout, &slog.HandlerOptions{Level: slog.LevelDebug}))) + + s, err := NewSubscriber(t.Context(), conf, cb, log, NewMetrics(nil, conf.StreamID)) + require.NoError(t, err) + + mq := new(mockQuerier) + s.querier = mq + s.testingAdminClient = e.DatabaseAdminClient + + return s, s.store, mq +} + +func testSubscriberSetup( + t *testing.T, + e changestreamstest.EmulatorHelper, + cb CallbackFunc, + opts ...func(*Config), +) (*Subscriber, *metadata.Store, *mockQuerier, chan string) { + s, ms, mq := testSubscriber(t, e, cb, opts...) + + done := make(chan string) + s.testingPostFinished = func(partitionToken string, err error) { + if err == nil { + done <- partitionToken + } + } + + // Call setup to create the metadata table + mq.ExpectQueryWithRecords(rootPartitionMetadata.PartitionToken, ChangeRecord{}) + require.NoError(t, s.Setup(t.Context())) + mq.AssertExpectations(t) + + return s, ms, mq, done +} + +func TestIntegrationSubscriberSetup(t *testing.T) { + integration.CheckSkip(t) + + e := changestreamstest.MakeEmulatorHelper(t) + defer e.Close() + + s, ms, mq := testSubscriber(t, e, nil) + defer s.Close() + + const childPartitionToken = "child-partition-token" + mq.ExpectQueryWithRecords(rootPartitionMetadata.PartitionToken, ChangeRecord{ + ChildPartitionsRecords: []*ChildPartitionsRecord{ + { + StartTimestamp: testStartTimestamp, + RecordSequence: "1", + ChildPartitions: []*ChildPartition{ + { + Token: childPartitionToken, + ParentPartitionTokens: []string{}, // Empty for root partition + }, + }, + }, + }, + }).Twice() + defer mq.AssertExpectations(t) + + // When Setup is called + require.NoError(t, s.Setup(t.Context())) + + // Then the root partition is created + cpm0, err := s.store.GetPartition(t.Context(), childPartitionToken) + require.NoError(t, err) + assert.Equal(t, metadata.StateCreated, cpm0.State) + + // Given the root partition is scheduled + _, err = ms.UpdateToScheduled(t.Context(), []string{childPartitionToken}) + require.NoError(t, err) + + // When Setup is called again + require.NoError(t, s.Setup(t.Context())) + + // Then the root partition is not changed + cpm1, err := s.store.GetPartition(t.Context(), childPartitionToken) + require.NoError(t, err) + assert.Equal(t, metadata.StateScheduled, cpm1.State) +} + +func TestIntegrationSubscriberStartContextCanceled(t *testing.T) { + integration.CheckSkip(t) + + e := changestreamstest.MakeEmulatorHelper(t) + defer e.Close() + + s, ms, mq, _ := testSubscriberSetup(t, e, nil) + defer s.Close() + + // Given a single partition + require.NoError(t, ms.Create(t.Context(), []metadata.PartitionMetadata{ + testPartitionMetadata(testPartitionToken), + })) + + // When the partition waits for context cancellation + mq.ExpectQuery(testPartitionToken).Run(func(args mock.Arguments) { + ctx := args.Get(0).(context.Context) + select { + case <-ctx.Done(): + case <-time.After(time.Second): + t.Fatalf("timed out waiting for partition1 to be aborted") + } + }).Return(context.Canceled) + + // And context is cancelled + ctx, cancel := context.WithCancel(t.Context()) + time.AfterFunc(100*time.Millisecond, cancel) + + // Then Run returns context.Canceled + require.ErrorIs(t, s.Run(ctx), context.Canceled) + + mq.AssertExpectations(t) +} + +func TestIntegrationSubscriberStartReturnsErrorOnPartitionError(t *testing.T) { + integration.CheckSkip(t) + + e := changestreamstest.MakeEmulatorHelper(t) + defer e.Close() + + s, ms, mq, _ := testSubscriberSetup(t, e, nil) + defer s.Close() + + // Given two sibling partitions + require.NoError(t, ms.Create(t.Context(), []metadata.PartitionMetadata{ + testPartitionMetadata("partition1"), + testPartitionMetadata("partition2"), + })) + + // When partition2 returns an error + testErr := errors.New("test error from partition2") + mq.ExpectQuery("partition2").Return(testErr) + + // Then partition1 is aborted + mq.ExpectQuery("partition1").Run(func(args mock.Arguments) { + ctx := args.Get(0).(context.Context) + select { + case <-ctx.Done(): + case <-time.After(time.Second): + t.Fatalf("timed out waiting for partition1 to be aborted") + } + }).Return(context.Canceled) + + require.ErrorIs(t, s.Run(t.Context()), testErr) + mq.AssertExpectations(t) +} + +func TestIntegrationSubscriberStartReturnsErrorOnCallbackError(t *testing.T) { + integration.CheckSkip(t) + + e := changestreamstest.MakeEmulatorHelper(t) + defer e.Close() + + // When callback returns an error + testErr := errors.New("test error from callback") + s, ms, mq, _ := testSubscriberSetup(t, e, func(_ context.Context, _ string, _ *DataChangeRecord) error { + return testErr + }) + defer s.Close() + + // Given partition with data + require.NoError(t, ms.Create(t.Context(), []metadata.PartitionMetadata{ + testPartitionMetadata(testPartitionToken), + })) + mq.ExpectQueryWithRecords(testPartitionToken, ChangeRecord{ + DataChangeRecords: []*DataChangeRecord{ + { + RecordSequence: "1", + CommitTimestamp: testStartTimestamp, + TableName: "test-table", + ModType: "INSERT", + }, + }, + }) + mq.expectCallbackError = true + + // Then Run returns the error + require.ErrorIs(t, s.Run(t.Context()), testErr) + mq.AssertExpectations(t) +} + +func TestIntegrationSubscriberResume(t *testing.T) { + integration.CheckSkip(t) + + e := changestreamstest.MakeEmulatorHelper(t) + defer e.Close() + + dch := make(chan *DataChangeRecord) + s, ms, mq, done := testSubscriberSetup(t, e, func(_ context.Context, _ string, dcr *DataChangeRecord) error { + if dcr != nil { + dch <- dcr + } + return nil + }) + defer s.Close() + + // Create partition in SCHEDULED state + err := ms.Create(t.Context(), []metadata.PartitionMetadata{testPartitionMetadata("scheduled")}) + require.NoError(t, err) + _, err = ms.UpdateToScheduled(t.Context(), []string{"scheduled"}) + require.NoError(t, err) + + // Create partition in RUNNING state + err = ms.Create(t.Context(), []metadata.PartitionMetadata{testPartitionMetadata("running")}) + require.NoError(t, err) + _, err = ms.UpdateToScheduled(t.Context(), []string{"running"}) + require.NoError(t, err) + _, err = ms.UpdateToRunning(t.Context(), "running") + require.NoError(t, err) + + mq.ExpectQueryWithRecords("scheduled", ChangeRecord{ + DataChangeRecords: []*DataChangeRecord{ + { + RecordSequence: "1", + CommitTimestamp: testStartTimestamp, + TableName: "test-table", + ModType: "INSERT", + }, + }, + }) + mq.ExpectQueryWithRecords("running", ChangeRecord{ + DataChangeRecords: []*DataChangeRecord{ + { + RecordSequence: "2", + CommitTimestamp: testStartTimestamp, + TableName: "test-table", + ModType: "UPDATE", + }, + }, + }) + + // When Run is called + go func() { + if err := s.Run(t.Context()); err != nil { + t.Log(err) + } + }() + + // Then partitions in SCHEDULED and RUNNING states are queried + collectN(t, 2, dch) + mq.AssertExpectations(t) + + // When partitions are finished + collectN(t, 2, done) + + // Then partitions are moved to FINISHED state + pm, err := ms.GetPartition(t.Context(), "scheduled") + require.NoError(t, err) + assert.Equal(t, metadata.StateFinished, pm.State) + + pm, err = ms.GetPartition(t.Context(), "running") + require.NoError(t, err) + assert.Equal(t, metadata.StateFinished, pm.State) +} + +func TestIntegrationSubscriberCallbackUpdatePartitionWatermark(t *testing.T) { + integration.CheckSkip(t) + + e := changestreamstest.MakeEmulatorHelper(t) + defer e.Close() + + var ( + cnt = 0 + s *Subscriber + ) + s, ms, mq, done := testSubscriberSetup(t, e, func(_ context.Context, partitionToken string, dcr *DataChangeRecord) error { + cnt += 1 + switch cnt { + case 1: + // When message is added to batch + case 2: + // Then watermark is not updated + pm, err := s.store.GetPartition(t.Context(), partitionToken) + require.NoError(t, err) + assert.Equal(t, metadata.StateRunning, pm.State) + assert.Equal(t, testStartTimestamp, pm.Watermark) + + // When UpdatePartitionWatermark is called + require.NoError(t, s.UpdatePartitionWatermark(t.Context(), partitionToken, dcr.CommitTimestamp)) + case 3: + assert.Nil(t, dcr) + + // Then watermark is updated + pm, err := s.store.GetPartition(t.Context(), partitionToken) + require.NoError(t, err) + assert.Equal(t, metadata.StateRunning, pm.State) + assert.Equal(t, testStartTimestamp.Add(2*time.Second), pm.Watermark) + default: + t.Fatal("unexpected call") + } + + return nil + }) + defer s.Close() + + // Given partition with data change records + pm := metadata.PartitionMetadata{ + PartitionToken: testPartitionToken, + ParentTokens: []string{}, + StartTimestamp: testStartTimestamp, + Watermark: testStartTimestamp, + } + require.NoError(t, ms.Create(t.Context(), []metadata.PartitionMetadata{pm})) + + mq.ExpectQueryWithRecords(testPartitionToken, ChangeRecord{ + DataChangeRecords: []*DataChangeRecord{ + { + RecordSequence: "1", + CommitTimestamp: testStartTimestamp.Add(time.Second), + TableName: "test-table", + ModType: "INSERT", + }, + { + RecordSequence: "2", + CommitTimestamp: testStartTimestamp.Add(2 * time.Second), + TableName: "test-table", + ModType: "UPDATE", + }, + }, + }) + + // When Run is called + go func() { + if err := s.Run(t.Context()); err != nil { + t.Log(err) + } + }() + + // And partition is processed + collectN(t, 1, done) + + mq.AssertExpectations(t) +} + +func TestIntegrationSubscriberAllowedModTypes(t *testing.T) { + integration.CheckSkip(t) + + e := changestreamstest.MakeEmulatorHelper(t) + defer e.Close() + + // Given subscriber with allowed mod types + dch := make(chan *DataChangeRecord, 10) // Make sure we don't block + s, ms, mq, done := testSubscriberSetup(t, e, func(_ context.Context, _ string, dcr *DataChangeRecord) error { + if dcr != nil { + dch <- dcr + } + return nil + }, func(conf *Config) { + conf.AllowedModTypes = []string{"INSERT"} // Only allow INSERT operations + }) + defer s.Close() + + // Call setup to create the metadata table + mq.ExpectQueryWithRecords(rootPartitionMetadata.PartitionToken, ChangeRecord{}) + require.NoError(t, s.Setup(t.Context())) + mq.AssertExpectations(t) + + // Given partition with INSERT and UPDATE data change records + pm := metadata.PartitionMetadata{ + PartitionToken: testPartitionToken, + ParentTokens: []string{}, + StartTimestamp: testStartTimestamp, + Watermark: testStartTimestamp, + } + require.NoError(t, ms.Create(t.Context(), []metadata.PartitionMetadata{pm})) + + mq.ExpectQueryWithRecords(testPartitionToken, ChangeRecord{ + DataChangeRecords: []*DataChangeRecord{ + { + RecordSequence: "1", + CommitTimestamp: testStartTimestamp.Add(time.Second), + TableName: "test-table", + ModType: "INSERT", // This should be processed + }, + { + RecordSequence: "2", + CommitTimestamp: testStartTimestamp.Add(2 * time.Second), + TableName: "test-table", + ModType: "UPDATE", // This should be filtered out + }, + }, + }) + + // When Run is called + go func() { + if err := s.Run(t.Context()); err != nil { + t.Log(err) + } + }() + + // And partition is processed + collectN(t, 1, done) + + // Then only INSERT data change record is processed + assert.Len(t, dch, 1) + dcrs := collectN(t, 1, dch) + assert.Equal(t, "INSERT", dcrs[0].ModType) + + mq.AssertExpectations(t) +} + +func TestIntegrationSubscriberChildTokenProcessingOrder(t *testing.T) { + integration.CheckSkip(t) + + e := changestreamstest.MakeEmulatorHelper(t) + defer e.Close() + + // Given child partition tokens where 0->1,2,3 and 2,3->4 + const ( + childToken1 = "child_token_1" + childToken2 = "child_token_2" + childToken3 = "child_token_3" + childToken4 = "child_token_4" + ) + + // And child token 3 blocks + childToken3Done := make(chan struct{}) + s, ms, mq, done := testSubscriberSetup(t, e, func(_ context.Context, partitionToken string, _ *DataChangeRecord) error { + if partitionToken == childToken3 { + select { + case <-childToken3Done: + case <-time.After(time.Second): + t.Errorf("timeout waiting for child token 3 to be processed") + } + } + return nil + }) + defer s.Close() + + ts := time.Date(2022, 5, 1, 9, 0, 0, 0, time.UTC) + heartbeatMillis := int64(10000) + + require.NoError(t, ms.Create(t.Context(), []metadata.PartitionMetadata{{ + PartitionToken: testPartitionToken, + ParentTokens: []string{}, + StartTimestamp: ts, + EndTimestamp: time.Time{}, // No end timestamp + HeartbeatMillis: heartbeatMillis, + State: metadata.StateCreated, + Watermark: ts, + }})) + mq.ExpectQueryWithRecords(testPartitionToken, ChangeRecord{ + ChildPartitionsRecords: []*ChildPartitionsRecord{ + { + StartTimestamp: ts, + RecordSequence: "1000012389", + ChildPartitions: []*ChildPartition{ + { + Token: childToken1, + ParentPartitionTokens: []string{}, + }, + { + Token: childToken2, + ParentPartitionTokens: []string{}, + }, + }, + }, + { + StartTimestamp: ts, + RecordSequence: "1000012390", + ChildPartitions: []*ChildPartition{ + { + Token: childToken3, + ParentPartitionTokens: []string{}, + }, + }, + }, + }, + }) + + ts4 := time.Date(2022, 5, 1, 9, 30, 15, 0, time.UTC) + mq.ExpectQueryWithRecords(childToken1, ChangeRecord{}).Run(func(args mock.Arguments) { + // Verify query parameters + pm := args.Get(1).(metadata.PartitionMetadata) + assert.Equal(t, ts, pm.StartTimestamp) + assert.True(t, pm.EndTimestamp.IsZero()) + assert.Equal(t, heartbeatMillis, pm.HeartbeatMillis) + }) + mq.ExpectQueryWithRecords(childToken2, ChangeRecord{ + ChildPartitionsRecords: []*ChildPartitionsRecord{ + { + StartTimestamp: ts4, + RecordSequence: "1000012389", + ChildPartitions: []*ChildPartition{ + { + Token: childToken4, + ParentPartitionTokens: []string{childToken2, childToken3}, + }, + }, + }, + }, + }) + mq.ExpectQueryWithRecords(childToken3, ChangeRecord{ + ChildPartitionsRecords: []*ChildPartitionsRecord{ + { + StartTimestamp: ts4, + RecordSequence: "1000012389", + ChildPartitions: []*ChildPartition{ + { + Token: childToken4, + ParentPartitionTokens: []string{childToken2, childToken3}, + }, + }, + }, + }, + }) + + // When Run is called + go func() { + if err := s.Run(t.Context()); err != nil { + t.Log(err) + } + }() + + // Then child partitions are processed + collectN(t, 3, done) // 0, 1, 2 + + // When detect new partitions runs + time.Sleep(500 * time.Millisecond) + + // Then child token 4 is NOT processed + mq.AssertExpectations(t) + + // When child token 3 is finished + mq.ExpectQueryWithRecords(childToken4, ChangeRecord{}) + close(childToken3Done) + + // Then child token 4 is processed + collectN(t, 2, done) + mq.AssertExpectations(t) +} + +func collectN[T any](t *testing.T, n int, ch <-chan T) []T { + t.Helper() + + var items []T + for i := 0; i < n; i++ { + select { + case item := <-ch: + items = append(items, item) + case <-time.After(time.Second): + t.Fatal("timeout waiting for channel item") + } + } + return items +} diff --git a/internal/impl/gcp/enterprise/changestreams/subscriber_test.go b/internal/impl/gcp/enterprise/changestreams/subscriber_test.go new file mode 100644 index 0000000000..bfece4c28e --- /dev/null +++ b/internal/impl/gcp/enterprise/changestreams/subscriber_test.go @@ -0,0 +1,47 @@ +// Copyright 2025 Redpanda Data, Inc. +// +// Licensed as a Redpanda Enterprise file under the Redpanda Community +// License (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// https://github.com/redpanda-data/connect/blob/main/licenses/rcl.md + +package changestreams + +import ( + "testing" + "time" + + "github.com/stretchr/testify/assert" + + "github.com/redpanda-data/connect/v4/internal/impl/gcp/enterprise/changestreams/metadata" +) + +func TestGroupPartitionsByCreatedAt(t *testing.T) { + pms := []metadata.PartitionMetadata{ + {PartitionToken: "a", CreatedAt: time.Unix(0, 10_000)}, + {PartitionToken: "b", CreatedAt: time.Unix(0, 10_000)}, + {PartitionToken: "c", CreatedAt: time.Unix(0, 20_000)}, + {PartitionToken: "d", CreatedAt: time.Unix(0, 20_000)}, + } + + got := groupPartitionsByCreatedAt(pms) + want := [][]metadata.PartitionMetadata{ + {{PartitionToken: "a", CreatedAt: time.Unix(0, 10_000)}, {PartitionToken: "b", CreatedAt: time.Unix(0, 10_000)}}, + {{PartitionToken: "c", CreatedAt: time.Unix(0, 20_000)}, {PartitionToken: "d", CreatedAt: time.Unix(0, 20_000)}}, + } + assert.Equal(t, want, got) +} + +func TestTokensOf(t *testing.T) { + pms := []metadata.PartitionMetadata{ + {PartitionToken: "a"}, + {PartitionToken: "b"}, + {PartitionToken: "c"}, + {PartitionToken: "d"}, + } + + got := tokensOf(pms) + want := []string{"a", "b", "c", "d"} + assert.Equal(t, want, got) +} diff --git a/internal/impl/gcp/enterprise/changestreams/time.go b/internal/impl/gcp/enterprise/changestreams/time.go new file mode 100644 index 0000000000..b5f9bc1931 --- /dev/null +++ b/internal/impl/gcp/enterprise/changestreams/time.go @@ -0,0 +1,64 @@ +// Copyright 2025 Redpanda Data, Inc. +// +// Licensed as a Redpanda Enterprise file under the Redpanda Community +// License (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// https://github.com/redpanda-data/connect/blob/main/licenses/rcl.md + +package changestreams + +import "time" + +var now = time.Now + +// timeCache is a cache for a single time value. +type timeCache struct { + v time.Time // cached value + t time.Time // when the value was cached + d time.Duration // cache duration + + now func() time.Time +} + +func (c *timeCache) get() time.Time { + if c.v.IsZero() || c.now().Sub(c.t) > c.d { + return time.Time{} + } + return c.v +} + +func (c *timeCache) set(v time.Time) { + c.v = v + c.t = c.now() +} + +// timeRange makes sure that we process records in monotonically increasing +// time order, and do not process records over a certain time range if the end +// time is set. +type timeRange struct { + cur time.Time + end time.Time +} + +// tryClaim claims a time as part of the current time range if it is after the +// current start time and before the end time. +// +// If the time is claimed, the start time is updated to the claimed time. +// +// Returns true if the time is claimed, false otherwise. +func (r *timeRange) tryClaim(t time.Time) bool { + if t.Before(r.cur) { + return false + } + if !r.end.IsZero() && r.end.Compare(t) <= 0 { + return false + } + + r.cur = t + return true +} + +func (r *timeRange) now() time.Time { + return r.cur +} diff --git a/internal/impl/gcp/enterprise/changestreams/time_test.go b/internal/impl/gcp/enterprise/changestreams/time_test.go new file mode 100644 index 0000000000..5dbd473f8f --- /dev/null +++ b/internal/impl/gcp/enterprise/changestreams/time_test.go @@ -0,0 +1,69 @@ +// Copyright 2025 Redpanda Data, Inc. +// +// Licensed as a Redpanda Enterprise file under the Redpanda Community +// License (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// https://github.com/redpanda-data/connect/blob/main/licenses/rcl.md + +package changestreams + +import ( + "testing" + "time" + + "github.com/stretchr/testify/assert" +) + +func TestTimeCache(t *testing.T) { + t0 := time.Unix(0, 1000) + + var nowTime time.Time + c := &timeCache{ + d: 2 * time.Second, + now: func() time.Time { + nowTime = nowTime.Add(time.Second) + return nowTime + }, + } + + // Empty cache + assert.True(t, c.get().IsZero(), "expected zero time") + + // Set and get + t.Log(nowTime) + c.set(t0) + assert.Equal(t, t0, c.get(), "time mismatch after set") + + // Get cached + t.Log(nowTime) + assert.Equal(t, t0, c.get(), "time mismatch from cache") + + // Cache expired + t.Log(nowTime) + assert.True(t, c.get().IsZero(), "expected zero time after expiration") +} + +func TestTimeRange(t *testing.T) { + r := timeRange{ + cur: time.Unix(0, 10_000), + end: time.Unix(0, 20_000), + } + + tests := []struct { + time time.Time + expected bool + }{ + {time.Unix(0, 10_000), true}, + {time.Unix(0, 10_000), true}, + {time.Unix(0, 11_000), true}, + {time.Unix(0, 11_000), true}, + {time.Unix(0, 19_000), true}, + {time.Unix(0, 20_000), false}, + } + + for _, test := range tests { + assert.Equal(t, test.expected, r.tryClaim(test.time), + "tryClaim(%v) returned unexpected result", test.time) + } +} diff --git a/internal/impl/gcp/enterprise/input_spanner_cdc.go b/internal/impl/gcp/enterprise/input_spanner_cdc.go new file mode 100644 index 0000000000..a6f56a95a1 --- /dev/null +++ b/internal/impl/gcp/enterprise/input_spanner_cdc.go @@ -0,0 +1,468 @@ +// Copyright 2025 Redpanda Data, Inc. +// +// Licensed as a Redpanda Enterprise file under the Redpanda Community +// License (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// https://github.com/redpanda-data/connect/blob/main/licenses/rcl.md + +package enterprise + +import ( + "context" + "encoding/base64" + "fmt" + "sync" + "time" + + "github.com/Jeffail/shutdown" + "google.golang.org/api/option" + + "github.com/redpanda-data/benthos/v4/public/service" + "github.com/redpanda-data/connect/v4/internal/ack" + + "github.com/redpanda-data/connect/v4/internal/impl/gcp/enterprise/changestreams" + "github.com/redpanda-data/connect/v4/internal/license" +) + +// Spanner Input Fields +const ( + siFieldCredentialsJSON = "credentials_json" + siFieldProjectID = "project_id" + siFieldInstanceID = "instance_id" + siFieldDatabaseID = "database_id" + siFieldStreamID = "stream_id" + siFieldStartTimestamp = "start_timestamp" + siFieldEndTimestamp = "end_timestamp" + siFieldHeartbeatInterval = "heartbeat_interval" + siFieldMetadataTable = "metadata_table" + siFieldMinWatermarkCacheTTL = "min_watermark_cache_ttl" + siFieldAllowedModTypes = "allowed_mod_types" + siFieldBatchPolicy = "batching" +) + +// Default values +const ( + defaultMetadataTableFormat = "cdc_metadata_%s" + shutdownTimeout = 5 * time.Second +) + +type spannerCDCInputConfig struct { + changestreams.Config +} + +func parseRFC3339Nano(pConf *service.ParsedConfig, key string) (time.Time, error) { + s, err := pConf.FieldString(key) + if err != nil { + return time.Time{}, err + } + if s == "" { + return time.Time{}, nil + } + + t, err := time.Parse(time.RFC3339Nano, s) + if err != nil { + return time.Time{}, fmt.Errorf("failed to parse %v as RFC3339Nano: %w", key, err) + } + return t, nil +} + +func spannerCDCInputConfigFromParsed(pConf *service.ParsedConfig) (conf spannerCDCInputConfig, err error) { + credentialsJSON, err := pConf.FieldString(siFieldCredentialsJSON) + if err != nil { + return + } + if credentialsJSON != "" { + credBytes, err := base64.StdEncoding.DecodeString(credentialsJSON) + if err != nil { + return conf, fmt.Errorf("decode base64 credentials: %w", err) + } + conf.SpannerClientOptions = append(conf.SpannerClientOptions, + option.WithCredentialsJSON(credBytes)) + } + + if conf.ProjectID, err = pConf.FieldString(siFieldProjectID); err != nil { + return + } + if conf.InstanceID, err = pConf.FieldString(siFieldInstanceID); err != nil { + return + } + if conf.DatabaseID, err = pConf.FieldString(siFieldDatabaseID); err != nil { + return + } + if conf.StreamID, err = pConf.FieldString(siFieldStreamID); err != nil { + return + } + if conf.StartTimestamp, err = parseRFC3339Nano(pConf, siFieldStartTimestamp); err != nil { + return + } + if conf.EndTimestamp, err = parseRFC3339Nano(pConf, siFieldEndTimestamp); err != nil { + return + } + if conf.HeartbeatInterval, err = pConf.FieldDuration(siFieldHeartbeatInterval); err != nil { + return + } + if conf.MetadataTable, err = pConf.FieldString(siFieldMetadataTable); err != nil { + return + } + if conf.MetadataTable == "" { + conf.MetadataTable = fmt.Sprintf(defaultMetadataTableFormat, conf.StreamID) + } + if pConf.Contains(siFieldAllowedModTypes) { + if conf.AllowedModTypes, err = pConf.FieldStringList(siFieldAllowedModTypes); err != nil { + return + } + } + if conf.MinWatermarkCacheTTL, err = pConf.FieldDuration(siFieldMinWatermarkCacheTTL); err != nil { + return + } + + return +} + +func spannerCDCInputSpec() *service.ConfigSpec { + return service.NewConfigSpec(). + Beta(). + Version("4.56.0"). + Categories("Services", "GCP"). + Summary("Creates an input that consumes from a spanner change stream."). + Description(` +Consumes change records from a Google Cloud Spanner change stream. This input allows +you to track and process database changes in real-time, making it useful for data +replication, event-driven architectures, and maintaining derived data stores. + +The input reads from a specified change stream within a Spanner database and converts +each change record into a message. The message payload contains the change records in +JSON format, and metadata is added with details about the Spanner instance, database, +and stream. + +Change streams provide a way to track mutations to your Spanner database tables. For +more information about Spanner change streams, refer to the Google Cloud documentation: +https://cloud.google.com/spanner/docs/change-streams +`). + Field(service.NewStringField(siFieldCredentialsJSON).Optional().Description("Base64 encoded GCP service account JSON credentials file for authentication. If not provided, Application Default Credentials (ADC) will be used.").Default("")). + Field(service.NewStringField(siFieldProjectID).Description("GCP project ID containing the Spanner instance")). + Field(service.NewStringField(siFieldInstanceID).Description("Spanner instance ID")). + Field(service.NewStringField(siFieldDatabaseID).Description("Spanner database ID")). + Field(service.NewStringField(siFieldStreamID).Description("The name of the change stream to track, the stream must exist in the database. To create a change stream, see https://cloud.google.com/spanner/docs/change-streams/manage.")). + Field(service.NewStringField(siFieldStartTimestamp).Optional().Description("RFC3339 formatted inclusive timestamp to start reading from the change stream (default: current time)").Example("2022-01-01T00:00:00Z").Default("")). + Field(service.NewStringField(siFieldEndTimestamp).Optional().Description("RFC3339 formatted exclusive timestamp to stop reading at (default: no end time)").Example("2022-01-01T00:00:00Z").Default("")). + Field(service.NewStringField(siFieldHeartbeatInterval).Advanced().Description("Duration string for heartbeat interval").Default("10s")). + Field(service.NewStringField(siFieldMetadataTable).Advanced().Optional().Description("The table to store metadata in (default: cdc_metadata_)").Default("")). + Field(service.NewStringField(siFieldMinWatermarkCacheTTL).Advanced().Description("Duration string for frequency of querying Spanner for minimum watermark.").Default("5s")). + Field(service.NewStringListField(siFieldAllowedModTypes).Advanced().Optional().Description("List of modification types to process. If not specified, all modification types are processed. Allowed values: INSERT, UPDATE, DELETE").Example([]string{"INSERT", "UPDATE", "DELETE"})). + Field(service.NewBatchPolicyField(siFieldBatchPolicy)). + Field(service.NewAutoRetryNacksToggleField()) +} + +func init() { + service.MustRegisterBatchInput("gcp_spanner_cdc", spannerCDCInputSpec(), + func(conf *service.ParsedConfig, mgr *service.Resources) (service.BatchInput, error) { + r, err := newSpannerCDCReaderFromParsed(conf, mgr) + if err != nil { + return nil, err + } + return service.AutoRetryNacksBatchedToggled(conf, r) + }) +} + +//------------------------------------------------------------------------------ + +type asyncMessage struct { + msg service.MessageBatch + ackFn service.AckFunc +} + +type spannerCDCReader struct { + conf spannerCDCInputConfig + log *service.Logger + metrics *changestreams.Metrics + + batching service.BatchPolicy + batcher *spannerPartitionBatcherFactory + resCh chan asyncMessage + subscriber *changestreams.Subscriber + stopSig *shutdown.Signaller +} + +var _ service.BatchInput = (*spannerCDCReader)(nil) + +func newSpannerCDCReaderFromParsed(pConf *service.ParsedConfig, mgr *service.Resources) (*spannerCDCReader, error) { + if err := license.CheckRunningEnterprise(mgr); err != nil { + return nil, err + } + + conf, err := spannerCDCInputConfigFromParsed(pConf) + if err != nil { + return nil, err + } + + batching, err := pConf.FieldBatchPolicy("batching") + if err != nil { + return nil, err + } else if batching.IsNoop() { + batching.Count = 1 + } + + return newSpannerCDCReader(conf, batching, mgr), nil +} + +func newSpannerCDCReader(conf spannerCDCInputConfig, batching service.BatchPolicy, mgr *service.Resources) *spannerCDCReader { + return &spannerCDCReader{ + conf: conf, + log: mgr.Logger(), + metrics: changestreams.NewMetrics(mgr.Metrics(), conf.StreamID), + batching: batching, + batcher: newSpannerPartitionBatcherFactory(batching, mgr), + resCh: make(chan asyncMessage), + stopSig: shutdown.NewSignaller(), + } +} + +func (r *spannerCDCReader) emit( + ctx context.Context, + partitionToken string, + msg service.MessageBatch, + commitTimestamp time.Time, +) (*ack.Once, error) { + if len(msg) == 0 { + return nil, nil + } + ackOnce := ack.NewOnce(func(ctx context.Context) error { + // If we processed the message and failed to update the watermark, we + // would try to update it on the next message, no need to return an error here. + if err := r.subscriber.UpdatePartitionWatermark(ctx, partitionToken, commitTimestamp); err != nil { + r.log.Errorf("%s: failed to update watermark: %v", partitionToken, err) + } + return nil + }) + select { + case <-ctx.Done(): + return nil, ctx.Err() + case r.resCh <- asyncMessage{msg: msg, ackFn: ackOnce.Ack}: + return ackOnce, nil + } +} + +var forcePeriodicFlush = &changestreams.DataChangeRecord{ + ModType: "FORCE_PERIODIC_FLUSH", // This is fake mod type to indicate periodic flush +} + +func (r *spannerCDCReader) onDataChangeRecord(ctx context.Context, partitionToken string, dcr *changestreams.DataChangeRecord) error { + batcher, _, err := r.batcher.forPartition(partitionToken) + if err != nil { + return err + } + + if err := batcher.AckError(); err != nil { + return fmt.Errorf("ack error: %v", err) + } + + // On partition end, flush the remaining messages and wait for all messages + // to be acked before returning and marking the partition as finished. + if dcr == nil { + msg, ts, err := batcher.Flush(ctx) + if err != nil { + return err + } + ack, err := r.emit(ctx, partitionToken, msg, ts) + if err != nil { + return err + } + batcher.AddAck(ack) + + if err := batcher.WaitAcks(ctx); err != nil { + return fmt.Errorf("ack error: %v", err) + } + if err := batcher.Close(ctx); err != nil { + return err + } + + return nil + } + + if dcr == forcePeriodicFlush { + msg, ts, err := batcher.Flush(ctx) + if err != nil { + return err + } + ack, err := r.emit(ctx, partitionToken, msg, ts) + if err != nil { + return err + } + batcher.AddAck(ack) + + return nil + } + + iter := batcher.MaybeFlushWith(dcr) + for mb, ts := range iter.Iter(ctx) { + ack, err := r.emit(ctx, partitionToken, mb, ts) + if err != nil { + return err + } + batcher.AddAck(ack) + } + if err := iter.Err(); err != nil { + return err + } + + return nil +} + +func (r *spannerCDCReader) Connect(ctx context.Context) error { + r.log.Infof("Connecting to Spanner CDC stream: %s (project: %s, instance: %s, database: %s)", + r.conf.StreamID, r.conf.ProjectID, r.conf.InstanceID, r.conf.DatabaseID) + + var cb changestreams.CallbackFunc = r.onDataChangeRecord + if r.batching.Period != "" { + r.log.Infof("Periodic flushing enabled: %s", r.batching.Period) + p := periodicallyFlushingSpannerCDCReader{ + spannerCDCReader: r, + reqCh: make(map[string]chan callbackRequest), + } + cb = p.onDataChangeRecord + } + + var err error + r.subscriber, err = changestreams.NewSubscriber(ctx, r.conf.Config, cb, r.log, r.metrics) + if err != nil { + return fmt.Errorf("create Spanner change stream reader: %w", err) + } + + if err := r.subscriber.Setup(ctx); err != nil { + return fmt.Errorf("setup Spanner change stream reader: %w", err) + } + + // Reset our stop signal + r.stopSig = shutdown.NewSignaller() + ctx, cancel := r.stopSig.SoftStopCtx(context.Background()) + + go func() { + defer cancel() + if err := r.subscriber.Run(ctx); err != nil { + r.log.Errorf("Spanner change stream reader error: %v", err) + } + r.subscriber.Close() + r.stopSig.TriggerHasStopped() + }() + + return nil +} + +func (r *spannerCDCReader) ReadBatch(ctx context.Context) (service.MessageBatch, service.AckFunc, error) { + select { + case <-ctx.Done(): + return nil, nil, ctx.Err() + case <-r.stopSig.HasStoppedChan(): + return nil, nil, service.ErrNotConnected + case am := <-r.resCh: + return am.msg, am.ackFn, nil + } +} + +func (r *spannerCDCReader) Close(ctx context.Context) error { + r.stopSig.TriggerSoftStop() + select { + case <-ctx.Done(): + case <-time.After(shutdownTimeout): + case <-r.stopSig.HasStoppedChan(): + } + r.stopSig.TriggerHardStop() + select { + case <-ctx.Done(): + return ctx.Err() + case <-time.After(shutdownTimeout): + case <-r.stopSig.HasStoppedChan(): + } + return nil +} + +type callbackRequest struct { + partitionToken string + dcr *changestreams.DataChangeRecord + errCh chan error +} + +// periodicallyFlushingSpannerCDCReader synchronizes callback invocations with +// periodic flushes to ensure ordering of messages. The flush period is +// governed by the spannerPartitionBatcher.period timer. +// +// When spannerPartitionBatcher.Close is called the timer is stopped and the +// go routine is terminated. +// +// All calls to spannerCDCReader.onDataChangeRecord use the same context as the +// first call to periodicallyFlushingSpannerCDCReader.onDataChangeRecord for +// a given partition. +type periodicallyFlushingSpannerCDCReader struct { + *spannerCDCReader + mu sync.RWMutex + reqCh map[string]chan callbackRequest +} + +func (r *periodicallyFlushingSpannerCDCReader) onDataChangeRecord(ctx context.Context, partitionToken string, dcr *changestreams.DataChangeRecord) error { + batcher, cached, err := r.batcher.forPartition(partitionToken) + if err != nil { + return err + } + + if !cached { + ch := make(chan callbackRequest) + r.mu.Lock() + r.reqCh[partitionToken] = ch + r.mu.Unlock() + + softStopCh := r.stopSig.SoftStopChan() + go func() { + r.log.Debugf("%s: starting periodic flusher", partitionToken) + defer func() { + r.mu.Lock() + delete(r.reqCh, partitionToken) + r.mu.Unlock() + r.log.Debugf("%s: periodic flusher stopped", partitionToken) + }() + + for { + select { + case <-ctx.Done(): + return + case <-softStopCh: + return + case _, ok := <-batcher.period.C: + if !ok { + return + } + + err := r.spannerCDCReader.onDataChangeRecord(ctx, partitionToken, forcePeriodicFlush) + if err != nil { + r.log.Warnf("%s: periodic flush error: %v", partitionToken, err) + } + case cr := <-ch: + cr.errCh <- r.spannerCDCReader.onDataChangeRecord(ctx, partitionToken, cr.dcr) + } + } + }() + } + + r.mu.RLock() + ch := r.reqCh[partitionToken] + r.mu.RUnlock() + + errCh := make(chan error) + select { + case <-ctx.Done(): + return ctx.Err() + case ch <- callbackRequest{ + partitionToken: partitionToken, + dcr: dcr, + errCh: errCh, + }: + // ok + } + select { + case <-ctx.Done(): + return ctx.Err() + case err := <-errCh: + return err + } +} diff --git a/internal/impl/gcp/enterprise/input_spanner_partition_batcher.go b/internal/impl/gcp/enterprise/input_spanner_partition_batcher.go new file mode 100644 index 0000000000..2d3c20f674 --- /dev/null +++ b/internal/impl/gcp/enterprise/input_spanner_partition_batcher.go @@ -0,0 +1,220 @@ +// Copyright 2025 Redpanda Data, Inc. +// +// Licensed as a Redpanda Enterprise file under the Redpanda Community +// License (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// https://github.com/redpanda-data/connect/blob/main/licenses/rcl.md + +package enterprise + +import ( + "context" + "encoding/json" + "errors" + "iter" + "sync" + "time" + + "github.com/redpanda-data/benthos/v4/public/service" + "github.com/redpanda-data/connect/v4/internal/ack" + "github.com/redpanda-data/connect/v4/internal/impl/gcp/enterprise/changestreams" +) + +// spannerPartitionBatchIter goes over changestreams.DataChangeRecord.Mods, +// for every mod it creates a message and adds it to the batch, if the batch is +// full, it yields the batch and creates a new one. +// +// Iff batch is returned with nonzero time, when acked the partition watermark +// should be updated to this time. +type spannerPartitionBatchIter struct { + *spannerPartitionBatcher + dcr *changestreams.DataChangeRecord + err error +} + +func (s *spannerPartitionBatchIter) Iter(ctx context.Context) iter.Seq2[service.MessageBatch, time.Time] { + return func(yield func(service.MessageBatch, time.Time) bool) { + if s.err != nil { + return + } + + lastFlushed := false + defer func() { + if lastFlushed { + s.last = nil + } else { + s.last = s.dcr + } + }() + + first := true + for i, m := range s.dcr.Mods { + b, err := json.Marshal(m) + if err != nil { + s.err = err + return + } + + msg := service.NewMessage(b) + msg.MetaSet("table_name", s.dcr.TableName) + msg.MetaSet("mod_type", s.dcr.ModType) + msg.MetaSetMut("commit_timestamp", s.dcr.CommitTimestamp) + msg.MetaSet("record_sequence", s.dcr.RecordSequence) + msg.MetaSet("server_transaction_id", s.dcr.ServerTransactionID) + msg.MetaSetMut("is_last_record_in_transaction_in_partition", s.dcr.IsLastRecordInTransactionInPartition) + msg.MetaSet("value_capture_type", s.dcr.ValueCaptureType) + msg.MetaSetMut("number_of_records_in_transaction", s.dcr.NumberOfRecordsInTransaction) + msg.MetaSetMut("number_of_partitions_in_transaction", s.dcr.NumberOfPartitionsInTransaction) + msg.MetaSet("transaction_tag", s.dcr.TransactionTag) + msg.MetaSetMut("is_system_transaction", s.dcr.IsSystemTransaction) + + if !s.batcher.Add(msg) { + continue + } + + mb, err := s.flush(ctx) + if err != nil { + s.err = err + return + } + + // Return the watermark to be updated after processing the batch. + // Not every batch should update the watermark, we update watermark + // only after processing the whole DataChangeRecord. + var watermark time.Time + if first && s.last != nil { + watermark = s.last.CommitTimestamp + first = false + } + if i == len(s.dcr.Mods)-1 { + watermark = s.dcr.CommitTimestamp + lastFlushed = true + } + if !yield(mb, watermark) { + return + } + } + } +} + +// Err returns any error that occurred during iteration. +func (s *spannerPartitionBatchIter) Err() error { + return s.err +} + +type spannerPartitionBatcher struct { + batcher *service.Batcher + last *changestreams.DataChangeRecord + period *time.Timer + acks []*ack.Once + rm func() +} + +func (s *spannerPartitionBatcher) MaybeFlushWith(dcr *changestreams.DataChangeRecord) *spannerPartitionBatchIter { + return &spannerPartitionBatchIter{spannerPartitionBatcher: s, dcr: dcr} +} + +func (s *spannerPartitionBatcher) Flush(ctx context.Context) (service.MessageBatch, time.Time, error) { + if s.last == nil { + return nil, time.Time{}, nil + } + defer func() { + s.last = nil + }() + + msg, err := s.flush(ctx) + return msg, s.last.CommitTimestamp, err +} + +func (s *spannerPartitionBatcher) flush(ctx context.Context) (service.MessageBatch, error) { + msg, err := s.batcher.Flush(ctx) + if d, ok := s.batcher.UntilNext(); ok { + s.period.Reset(d) + } + return msg, err +} + +func (s *spannerPartitionBatcher) AddAck(ack *ack.Once) { + if ack == nil { + return + } + s.acks = append(s.acks, ack) +} + +func (s *spannerPartitionBatcher) WaitAcks(ctx context.Context) error { + var merr []error + for _, ack := range s.acks { + if err := ack.Wait(ctx); err != nil { + merr = append(merr, err) + } + } + return errors.Join(merr...) +} + +func (s *spannerPartitionBatcher) AckError() error { + for _, ack := range s.acks { + if _, err := ack.TryWait(); err != nil { + return err + } + } + return nil +} + +func (s *spannerPartitionBatcher) Close(ctx context.Context) error { + defer s.rm() + if s.period != nil { + s.period.Stop() + } + return s.batcher.Close(ctx) +} + +// spannerPartitionBatcherFactory caches active spannerPartitionBatcher instances. +type spannerPartitionBatcherFactory struct { + batching service.BatchPolicy + res *service.Resources + + mu sync.RWMutex + partitions map[string]*spannerPartitionBatcher +} + +func newSpannerPartitionBatcherFactory( + batching service.BatchPolicy, + res *service.Resources, +) *spannerPartitionBatcherFactory { + return &spannerPartitionBatcherFactory{ + batching: batching, + res: res, + partitions: make(map[string]*spannerPartitionBatcher), + } +} + +func (f *spannerPartitionBatcherFactory) forPartition(partitionToken string) (*spannerPartitionBatcher, bool, error) { + f.mu.RLock() + spb, ok := f.partitions[partitionToken] + f.mu.RUnlock() + + if !ok { + b, err := f.batching.NewBatcher(f.res) + if err != nil { + return nil, false, err + } + + spb = &spannerPartitionBatcher{ + batcher: b, + rm: func() { + f.mu.Lock() + delete(f.partitions, partitionToken) + f.mu.Unlock() + }, + } + if d, ok := spb.batcher.UntilNext(); ok { + spb.period = time.NewTimer(d) + } + + f.mu.Lock() + f.partitions[partitionToken] = spb + f.mu.Unlock() + } + return spb, ok, nil +} diff --git a/internal/impl/gcp/enterprise/input_spanner_partition_batcher_test.go b/internal/impl/gcp/enterprise/input_spanner_partition_batcher_test.go new file mode 100644 index 0000000000..03d6500ded --- /dev/null +++ b/internal/impl/gcp/enterprise/input_spanner_partition_batcher_test.go @@ -0,0 +1,106 @@ +// Copyright 2025 Redpanda Data, Inc. +// +// Licensed as a Redpanda Enterprise file under the Redpanda Community +// License (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// https://github.com/redpanda-data/connect/blob/main/licenses/rcl.md + +package enterprise + +import ( + "testing" + "time" + + "cloud.google.com/go/spanner" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + + "github.com/redpanda-data/benthos/v4/public/service" + "github.com/redpanda-data/connect/v4/internal/impl/gcp/enterprise/changestreams" +) + +func TestSpannerPartitionBatcherMaybeFlushWith(t *testing.T) { + s, err := service.NewStreamBuilder().Build() + require.NoError(t, err) + batcher, err := service.BatchPolicy{ + Count: 2, + }.NewBatcher(s.Resources()) + require.NoError(t, err) + + pb := &spannerPartitionBatcher{ + batcher: batcher, + } + + mod := &changestreams.Mod{ + Keys: spanner.NullJSON{ + Value: "foo", + }, + } + + tsn := func(i int) time.Time { + return time.Unix(int64(i), 0).UTC() + } + + { + // Given a DataChangeRecord with a single mod + dcr := &changestreams.DataChangeRecord{ + CommitTimestamp: tsn(1), + TableName: "test_table", + Mods: []*changestreams.Mod{ + mod, + }, + ModType: "INSERT", + } + + // When MaybeFlushWith is called + iter := pb.MaybeFlushWith(dcr) + + var count int + for range iter.Iter(t.Context()) { + count++ + } + require.NoError(t, iter.Err()) + assert.Equal(t, 0, count) + } + + { + // Given a DataChangeRecord with 5 mods + dcr := &changestreams.DataChangeRecord{ + CommitTimestamp: tsn(2), + TableName: "test_table", + Mods: []*changestreams.Mod{ + mod, + mod, + mod, + mod, + mod, + }, + } + + // When MaybeFlushWith is called + iter := pb.MaybeFlushWith(dcr) + var got []time.Time + for mb, ts := range iter.Iter(t.Context()) { + assert.Len(t, mb, 2) + got = append(got, ts) + } + require.NoError(t, iter.Err()) + + // Then 3 batches are returned, each with 2 mods + want := []time.Time{ + tsn(1), + {}, + tsn(2), + } + assert.Equal(t, want, got) + + // When Flush is called + mb, ts, err := pb.Flush(t.Context()) + require.NoError(t, err) + + // Then no batch is returned + require.Nil(t, mb) + require.Zero(t, ts) + } +} diff --git a/internal/impl/gcp/enterprise/integration_spanner_cdc_test.go b/internal/impl/gcp/enterprise/integration_spanner_cdc_test.go new file mode 100644 index 0000000000..637096e498 --- /dev/null +++ b/internal/impl/gcp/enterprise/integration_spanner_cdc_test.go @@ -0,0 +1,569 @@ +// Copyright 2025 Redpanda Data, Inc. +// +// Licensed as a Redpanda Enterprise file under the Redpanda Community +// License (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// https://github.com/redpanda-data/connect/blob/main/licenses/rcl.md + +package enterprise + +import ( + "context" + "encoding/json" + "errors" + "fmt" + "io" + "net/http" + "regexp" + "sort" + "strings" + "testing" + "time" + + "cloud.google.com/go/spanner" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + + _ "github.com/redpanda-data/benthos/v4/public/components/io" + _ "github.com/redpanda-data/benthos/v4/public/components/pure" + "github.com/redpanda-data/benthos/v4/public/service" + "github.com/redpanda-data/benthos/v4/public/service/integration" + "github.com/redpanda-data/connect/v4/internal/impl/gcp/enterprise/changestreams" + "github.com/redpanda-data/connect/v4/internal/impl/gcp/enterprise/changestreams/changestreamstest" + "github.com/redpanda-data/connect/v4/internal/license" +) + +func runSpannerCDCInputStream( + t *testing.T, + h changestreamstest.RealHelper, + startTimestamp time.Time, + endTimestamp time.Time, + msgs chan<- *service.Message, +) (addr string) { + port, err := integration.GetFreePort() + require.NoError(t, err) + httpConf := fmt.Sprintf(` +http: + enabled: true + address: localhost:%d`, port) + + inputConf := fmt.Sprintf(` +gcp_spanner_cdc: + project_id: %s + instance_id: %s + database_id: %s + stream_id: %s + start_timestamp: %s + end_timestamp: %s + heartbeat_interval: "5s" +`, + h.ProjectID(), + h.InstanceID(), + h.DatabaseID(), + h.Stream(), + startTimestamp.Format(time.RFC3339), + endTimestamp.Add(time.Second).Format(time.RFC3339), // end timestamp is exclusive + ) + + sb := service.NewStreamBuilder() + require.NoError(t, sb.SetYAML(httpConf)) + require.NoError(t, sb.AddInputYAML(inputConf)) + require.NoError(t, sb.SetLoggerYAML(`level: DEBUG`)) + require.NoError(t, sb.SetMetricsYAML(`json_api: {}`)) + + var count int + require.NoError(t, sb.AddConsumerFunc(func(_ context.Context, msg *service.Message) error { + count += 1 + t.Logf("Got message: %d", count) + + select { + case <-t.Context().Done(): + return t.Context().Err() + case msgs <- msg: + return nil + } + }, + )) + + s, err := sb.Build() + require.NoError(t, err, "failed to build stream") + license.InjectTestService(s.Resources()) + + t.Cleanup(func() { + if err := s.StopWithin(time.Second); err != nil { + t.Log(err) + } + }) + + go func() { + if err := s.Run(t.Context()); err != nil && !errors.Is(err, context.Canceled) { + t.Errorf("stream error: %v", err) + } + close(msgs) + }() + + return fmt.Sprintf("localhost:%d", port) +} + +type SingersTableHelper struct { + changestreamstest.RealHelper + t *testing.T +} + +func (h SingersTableHelper) CreateTableAndStream() { + h.RealHelper.CreateTableAndStream(`CREATE TABLE %s ( + SingerId INT64 NOT NULL, + FirstName STRING(MAX), + LastName STRING(MAX) + ) PRIMARY KEY (SingerId)`) +} + +func (h SingersTableHelper) InsertRows(n int) (time.Time, time.Time) { + firstCommitTimestamp := h.insertRow(1) + for i := 2; i < n; i++ { + h.insertRow(i) + } + lastCommitTimestamp := h.insertRow(n) + return firstCommitTimestamp, lastCommitTimestamp +} + +func (h SingersTableHelper) UpdateRows(n int) (time.Time, time.Time) { + firstCommitTimestamp := h.updateRow(1) + for i := 2; i < n; i++ { + h.updateRow(i) + } + lastCommitTimestamp := h.updateRow(n) + return firstCommitTimestamp, lastCommitTimestamp +} + +func (h SingersTableHelper) DeleteRows(n int) (time.Time, time.Time) { + firstCommitTimestamp := h.deleteRow(1) + for i := 2; i < n; i++ { + h.deleteRow(i) + } + lastCommitTimestamp := h.deleteRow(n) + return firstCommitTimestamp, lastCommitTimestamp +} + +func (h SingersTableHelper) insertRow(singerID int) time.Time { + ts, err := h.Client().Apply(h.t.Context(), + []*spanner.Mutation{h.insertMut(singerID)}, + spanner.TransactionTag("app=rpcn;action=insert")) + require.NoError(h.t, err) + + return ts +} + +func (h SingersTableHelper) insertMut(singerID int) *spanner.Mutation { + return spanner.InsertMap(h.Table(), map[string]any{ + "SingerId": singerID, + "FirstName": fmt.Sprintf("First Name %d", singerID), + "LastName": fmt.Sprintf("Last Name %d", singerID), + }) +} + +func (h SingersTableHelper) updateRow(singerID int) time.Time { + ts, err := h.Client().Apply(h.t.Context(), + []*spanner.Mutation{h.updateMut(singerID)}, + spanner.TransactionTag("app=rpcn;action=update")) + require.NoError(h.t, err) + + return ts +} + +func (h SingersTableHelper) updateMut(singerID int) *spanner.Mutation { + mut := spanner.UpdateMap(h.Table(), map[string]any{ + "SingerId": singerID, + "FirstName": fmt.Sprintf("Updated First Name %d", singerID), + "LastName": fmt.Sprintf("Updated Last Name %d", singerID), + }) + return mut +} + +func (h SingersTableHelper) deleteRow(singerID int) time.Time { + ts, err := h.Client().Apply(h.t.Context(), + []*spanner.Mutation{h.deleteMut(singerID)}, + spanner.TransactionTag("app=rpcn;action=delete")) + require.NoError(h.t, err) + + return ts +} + +func (h SingersTableHelper) deleteMut(singerID int) *spanner.Mutation { + return spanner.Delete(h.Table(), spanner.Key{singerID}) +} + +func TestIntegrationRealSpannerCDCInput(t *testing.T) { + integration.CheckSkip(t) + changestreamstest.CheckSkipReal(t) + + require.NoError(t, changestreamstest.MaybeDropOrphanedStreams(t.Context())) + + // How many rows to insert/update/delete + const numRows = 5 + + h := SingersTableHelper{changestreamstest.MakeRealHelper(t), t} + h.CreateTableAndStream() + + // When rows are inserted, updated and deleted + startTimestamp, _ := h.InsertRows(numRows) + h.UpdateRows(numRows) + _, endTimestamp := h.DeleteRows(numRows) + + // And the stream is started + ch := make(chan *service.Message, 3*numRows) + addr := runSpannerCDCInputStream(t, h.RealHelper, startTimestamp, endTimestamp, ch) + + // Then all the changes are received + var inserts, updates, deletes []changestreams.Mod + for _, msg := range collectN(t, numRows*3, ch) { + assert.Equal(t, h.Table(), msg.TableName) + switch msg.ModType { + case "INSERT": + transactionTag, _ := msg.MetaGet("transaction_tag") + require.Equal(t, "app=rpcn;action=insert", transactionTag) + inserts = append(inserts, msg.Mod) + case "UPDATE": + transactionTag, _ := msg.MetaGet("transaction_tag") + require.Equal(t, "app=rpcn;action=update", transactionTag) + updates = append(updates, msg.Mod) + case "DELETE": + transactionTag, _ := msg.MetaGet("transaction_tag") + require.Equal(t, "app=rpcn;action=delete", transactionTag) + deletes = append(deletes, msg.Mod) + } + } + + wantInserts := make([]changestreams.Mod, numRows) + for i := range wantInserts { + singerID := i + 1 + wantInserts[i] = changestreams.Mod{ + Keys: spanner.NullJSON{ + Value: map[string]any{"SingerId": fmt.Sprintf("%d", singerID)}, + Valid: true, + }, + NewValues: spanner.NullJSON{ + Value: map[string]any{ + "FirstName": fmt.Sprintf("First Name %d", singerID), + "LastName": fmt.Sprintf("Last Name %d", singerID), + }, + Valid: true, + }, + OldValues: spanner.NullJSON{ + Value: map[string]any{}, + Valid: true, + }, + } + } + assert.Equal(t, wantInserts, inserts) + + wantUpdates := make([]changestreams.Mod, numRows) + for i := range wantUpdates { + singerID := i + 1 + wantUpdates[i] = changestreams.Mod{ + Keys: spanner.NullJSON{ + Value: map[string]any{"SingerId": fmt.Sprintf("%d", singerID)}, + Valid: true, + }, + NewValues: spanner.NullJSON{ + Value: map[string]any{ + "FirstName": fmt.Sprintf("Updated First Name %d", singerID), + "LastName": fmt.Sprintf("Updated Last Name %d", singerID), + }, + Valid: true, + }, + OldValues: spanner.NullJSON{ + Value: map[string]any{ + "FirstName": fmt.Sprintf("First Name %d", singerID), + "LastName": fmt.Sprintf("Last Name %d", singerID), + }, + Valid: true, + }, + } + } + assert.Equal(t, wantUpdates, updates) + + wantDeletes := make([]changestreams.Mod, numRows) + for i := range wantDeletes { + singerID := i + 1 + wantDeletes[i] = changestreams.Mod{ + Keys: spanner.NullJSON{ + Value: map[string]any{"SingerId": fmt.Sprintf("%d", singerID)}, + Valid: true, + }, + NewValues: spanner.NullJSON{ + Value: map[string]any{}, + Valid: true, + }, + OldValues: spanner.NullJSON{ + Value: map[string]any{ + "FirstName": fmt.Sprintf("Updated First Name %d", singerID), + "LastName": fmt.Sprintf("Updated Last Name %d", singerID), + }, + Valid: true, + }, + } + } + assert.Equal(t, wantDeletes, deletes) + + // And metrics are set... + resp, err := http.Get("http://" + addr + "/metrics") + require.NoError(t, err) + b, err := io.ReadAll(resp.Body) + require.NoError(t, err) + t.Logf("Metrics:\n%s", string(b)) + + ms := parseMetricsSnapshot(t, b) + require.NotZero(t, ms.PartitionCreatedToScheduled) + require.NotZero(t, ms.PartitionScheduledToRunning) + require.NotZero(t, ms.DataChangeRecordCommittedToEmitted) + ms.PartitionCreatedToScheduled = timeDist{} + ms.PartitionScheduledToRunning = timeDist{} + ms.DataChangeRecordCommittedToEmitted = timeDist{} + + // This can be a bit flaky depending on if Spanner decides to split the + // partition. Adding PartitionRecordSplitCount covers both cases. + want := metricsSnapshot{ + PartitionRecordCreatedCount: 2 + ms.PartitionRecordSplitCount, + PartitionRecordRunningCount: 2 + ms.PartitionRecordSplitCount, + PartitionRecordFinishedCount: 1 + ms.PartitionRecordSplitCount, + PartitionRecordSplitCount: ms.PartitionRecordSplitCount, + PartitionRecordMergeCount: 0, + QueryCount: 2 + ms.PartitionRecordSplitCount, + DataChangeRecordCount: 3 * numRows, + HeartbeatRecordCount: 1 + ms.PartitionRecordSplitCount, + } + assert.Equal(t, want, ms) +} + +func TestIntegrationRealSpannerCDCInputMessagesOrderedByTimestampAndTransactionId(t *testing.T) { + integration.CheckSkip(t) + changestreamstest.CheckSkipReal(t) + + require.NoError(t, changestreamstest.MaybeDropOrphanedStreams(t.Context())) + + h := SingersTableHelper{changestreamstest.MakeRealHelper(t), t} + h.CreateTableAndStream() + + writeTransactionsToDatabase := func() time.Time { + // 1. Insert Singer 1 and Singer 2 + ts, err := h.Client().Apply(h.t.Context(), []*spanner.Mutation{ + h.insertMut(1), + h.insertMut(2), + }) + require.NoError(t, err) + t.Logf("First transaction committed with timestamp: %v", ts) + + // 2. Delete Singer 1 and Insert Singer 3 + ts, err = h.Client().Apply(h.t.Context(), []*spanner.Mutation{ + h.deleteMut(1), + h.insertMut(3), + }) + require.NoError(t, err) + t.Logf("Second transaction committed with timestamp: %v", ts) + + // 3. Delete Singer 2 and Singer 3 + ts, err = h.Client().Apply(h.t.Context(), []*spanner.Mutation{ + h.deleteMut(2), + h.deleteMut(3), + }) + require.NoError(t, err) + t.Logf("Third transaction committed with timestamp: %v", ts) + + // 4. Delete Singer 0 if it exists + ts, err = h.Client().Apply(h.t.Context(), []*spanner.Mutation{ + h.deleteMut(0), + }) + require.NoError(t, err) + t.Logf("Fourth transaction committed with timestamp: %v", ts) + + return ts + } + + // Given 3 batches of transactions with 2 second gaps + const expectedMessages = 1 + 7 + 2*6 + startTimestamp := h.insertRow(0) + writeTransactionsToDatabase() + time.Sleep(2 * time.Second) + writeTransactionsToDatabase() + time.Sleep(2 * time.Second) + endTimestamp := writeTransactionsToDatabase() + + // When we read from the stream + ch := make(chan *service.Message, expectedMessages) + runSpannerCDCInputStream(t, h.RealHelper, startTimestamp, endTimestamp, ch) + messages := collectN(t, expectedMessages, ch) + + // Then there are 3 batches... + + // Sort messages by commit timestamp and transaction ID + commitTimestampAt := func(idx int) time.Time { + s, ok := messages[idx].MetaGet("commit_timestamp") + require.True(t, ok) + v, err := time.Parse(time.RFC3339Nano, s) + require.NoError(t, err) + return v + } + transactionIdAt := func(idx int) string { + s, ok := messages[idx].MetaGet("server_transaction_id") + require.True(t, ok) + return s + } + sort.SliceStable(messages, func(i, j int) bool { // MUST be stable + if cmp := commitTimestampAt(i).Compare(commitTimestampAt(j)); cmp == 0 { + return transactionIdAt(i) < transactionIdAt(j) + } else { + return cmp < 0 + } + }) + + // Group by batches with 1.5 second gap threshold + groupMessagesByBatch := func() [][]spannerModMessage { + var ( + batches [][]spannerModMessage + cur []spannerModMessage + lastTs time.Time + ) + + for i, msg := range messages { + ts := commitTimestampAt(i) + + if len(cur) == 0 || ts.Sub(lastTs) < 1500*time.Millisecond { + cur = append(cur, msg) + } else { + batches = append(batches, cur) + cur = []spannerModMessage{msg} + } + lastTs = ts + } + if len(cur) != 0 { + batches = append(batches, cur) + } + + return batches + } + batches := groupMessagesByBatch() + require.Len(t, batches, 3) + + // And operation order is preserved... + + var sb strings.Builder + for i, batch := range batches { + sb.WriteString(fmt.Sprintf("Batch %d:\n", i)) + for _, m := range batch { + fmt.Fprintf(&sb, " %s: %s\n", m.ModType, m.Mod.Keys.Value) + } + } + want := `Batch 0: + INSERT: map[SingerId:0] + INSERT: map[SingerId:1] + INSERT: map[SingerId:2] + DELETE: map[SingerId:1] + INSERT: map[SingerId:3] + DELETE: map[SingerId:2] + DELETE: map[SingerId:3] + DELETE: map[SingerId:0] +Batch 1: + INSERT: map[SingerId:1] + INSERT: map[SingerId:2] + DELETE: map[SingerId:1] + INSERT: map[SingerId:3] + DELETE: map[SingerId:2] + DELETE: map[SingerId:3] +Batch 2: + INSERT: map[SingerId:1] + INSERT: map[SingerId:2] + DELETE: map[SingerId:1] + INSERT: map[SingerId:3] + DELETE: map[SingerId:2] + DELETE: map[SingerId:3] +` + assert.Equal(t, want, sb.String()) +} + +type spannerModMessage struct { + *service.Message + TableName string + ModType string + Mod changestreams.Mod +} + +func collectN(t *testing.T, n int, ch <-chan *service.Message) (mods []spannerModMessage) { + for range n { + select { + case msg := <-ch: + b, err := msg.AsBytes() + require.NoError(t, err) + + v := spannerModMessage{ + Message: msg, + } + + var ok bool + v.TableName, ok = msg.MetaGet("table_name") + require.True(t, ok) + v.ModType, ok = msg.MetaGet("mod_type") + require.True(t, ok) + + require.NoError(t, json.Unmarshal(b, &v.Mod)) + mods = append(mods, v) + case <-time.After(time.Minute): + t.Fatalf("timeout waiting for message, got %d messages wanted %d", len(mods), n) + } + } + return +} + +type timeDist struct { + P50 float64 `json:"p50"` + P90 float64 `json:"p90"` + P99 float64 `json:"p99"` +} + +type metricsSnapshot struct { + PartitionRecordCreatedCount int64 `json:"partition_record_created_count"` + PartitionRecordRunningCount int64 `json:"partition_record_running_count"` + PartitionRecordFinishedCount int64 `json:"partition_record_finished_count"` + PartitionRecordSplitCount int64 `json:"partition_record_split_count"` + PartitionRecordMergeCount int64 `json:"partition_record_merge_count"` + PartitionCreatedToScheduled timeDist `json:"partition_created_to_scheduled_ns"` + PartitionScheduledToRunning timeDist `json:"partition_scheduled_to_running_ns"` + QueryCount int64 `json:"query_count"` + DataChangeRecordCount int64 `json:"data_change_record_count"` + DataChangeRecordCommittedToEmitted timeDist `json:"data_change_record_committed_to_emitted_ns"` + HeartbeatRecordCount int64 `json:"heartbeat_record_count"` +} + +func parseMetricsSnapshot(t *testing.T, data []byte) metricsSnapshot { + // First preprocess the JSON to clean up the metric names + data, err := extractSpannerCDCMetricsJSON(data) + require.NoError(t, err) + + // Unmarshal the cleaned JSON into the metricsSnapshot struct + var ms metricsSnapshot + require.NoError(t, json.Unmarshal(data, &ms)) + return ms +} + +// extractSpannerCDCMetricsJSON transforms the raw metrics JSON into a format +// that can be directly unmarshaled into a metricsSnapshot struct. +func extractSpannerCDCMetricsJSON(data []byte) ([]byte, error) { + // Parse the raw JSON into a map + var rawData map[string]json.RawMessage + if err := json.Unmarshal(data, &rawData); err != nil { + return nil, err + } + + metricNameRegex := regexp.MustCompile(`spanner_cdc_([^{]+)(?:\{.*\})?`) + + res := make(map[string]json.RawMessage) + for k, v := range rawData { + m := metricNameRegex.FindStringSubmatch(k) + if len(m) < 2 { + continue + } + res[m[1]] = v + } + return json.Marshal(res) +} diff --git a/internal/plugins/info.csv b/internal/plugins/info.csv index cd1179454d..7657136ad6 100644 --- a/internal/plugins/info.csv +++ b/internal/plugins/info.csv @@ -87,6 +87,7 @@ gcp_cloud_storage ,output ,GCP Cloud Storage ,3.43.0 ,certif gcp_cloudtrace ,tracer ,GCP Cloud Trace ,4.2.0 ,certified ,n ,y ,y gcp_pubsub ,input ,GCP PubSub ,0.0.0 ,certified ,n ,y ,y gcp_pubsub ,output ,GCP PubSub ,0.0.0 ,certified ,n ,y ,y +gcp_spanner_cdc ,input ,gcp_spanner_cdc ,0.0.0 ,enterprise ,n ,y ,y gcp_vertex_ai_chat ,processor ,GCP Vertex AI ,4.34.0 ,enterprise ,n ,y ,y gcp_vertex_ai_embeddings ,processor ,gcp_vertex_ai_embeddings ,4.37.0 ,enterprise ,n ,y ,y generate ,input ,generate ,3.40.0 ,certified ,n ,y ,y