From f961ddb098aec91ffa69222cc41f7bd302c98397 Mon Sep 17 00:00:00 2001 From: Andrew Nicoll Date: Mon, 9 Dec 2024 11:26:01 +1030 Subject: [PATCH 1/7] Feat: implement change stream reader from GCP SpannerChangeStream --- go.mod | 19 +- go.sum | 26 +- internal/impl/gcp/spanner_change_stream.go | 281 ++++++++++++++++++ .../gcp/spanner_change_stream_mock_test.go | 39 +++ .../impl/gcp/spanner_change_stream_test.go | 150 ++++++++++ 5 files changed, 493 insertions(+), 22 deletions(-) create mode 100644 internal/impl/gcp/spanner_change_stream.go create mode 100644 internal/impl/gcp/spanner_change_stream_mock_test.go create mode 100644 internal/impl/gcp/spanner_change_stream_test.go diff --git a/go.mod b/go.mod index 9b6858ee41..10fbbf2880 100644 --- a/go.mod +++ b/go.mod @@ -6,6 +6,7 @@ require ( cloud.google.com/go/aiplatform v1.68.0 cloud.google.com/go/bigquery v1.64.0 cloud.google.com/go/pubsub v1.45.1 + cloud.google.com/go/spanner v1.73.0 cloud.google.com/go/storage v1.43.0 cloud.google.com/go/vertexai v0.12.0 github.com/Azure/azure-sdk-for-go/sdk/azcore v1.14.0 @@ -25,6 +26,7 @@ require ( github.com/Masterminds/squirrel v1.5.4 github.com/PaesslerAG/gval v1.2.2 github.com/PaesslerAG/jsonpath v0.1.1 + github.com/anicoll/screamer v0.0.0-20241206035431-9ba919b54dfd github.com/apache/pulsar-client-go v0.13.1 github.com/authzed/authzed-go v1.0.0 github.com/authzed/grpcutil v0.0.0-20240123194739-2ea1e3d2d98b @@ -61,7 +63,7 @@ require ( github.com/eclipse/paho.mqtt.golang v1.5.0 github.com/generikvault/gvalstrings v0.0.0-20180926130504-471f38f0112a github.com/getsentry/sentry-go v0.28.1 - github.com/go-faker/faker/v4 v4.4.2 + github.com/go-faker/faker/v4 v4.5.0 github.com/go-jose/go-jose/v3 v3.0.3 github.com/go-resty/resty/v2 v2.15.3 github.com/go-sql-driver/mysql v1.8.1 @@ -144,7 +146,7 @@ require ( golang.org/x/crypto v0.28.0 golang.org/x/exp v0.0.0-20241009180824-f66d83c29e7c golang.org/x/net v0.30.0 - golang.org/x/sync v0.8.0 + golang.org/x/sync v0.9.0 golang.org/x/text v0.19.0 google.golang.org/api v0.205.0 google.golang.org/protobuf v1.35.1 @@ -156,7 +158,6 @@ require ( cloud.google.com/go/longrunning v0.6.2 // indirect cloud.google.com/go/monitoring v1.21.2 // indirect cloud.google.com/go/secretmanager v1.14.2 // indirect - cloud.google.com/go/spanner v1.73.0 // 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/GoogleCloudPlatform/grpc-gcp-go/grpcgcp v1.5.0 // indirect @@ -252,8 +253,8 @@ require ( github.com/couchbase/gocbcoreps v0.1.3 // indirect github.com/couchbase/goprotostellar v1.0.2 // indirect github.com/couchbaselabs/gocbconnstr/v2 v2.0.0-20240607131231-fb385523de28 // indirect - github.com/cpuguy83/dockercfg v0.3.1 // indirect - github.com/cpuguy83/go-md2man/v2 v2.0.4 // indirect + github.com/cpuguy83/dockercfg v0.3.2 // indirect + github.com/cpuguy83/go-md2man/v2 v2.0.5 // indirect github.com/danieljoos/wincred v1.2.0 // indirect github.com/davecgh/go-spew v1.1.1 // indirect github.com/dgryski/go-rendezvous v0.0.0-20200823014737-9f7001d12a5f // indirect @@ -385,11 +386,11 @@ require ( github.com/sirupsen/logrus v1.9.3 // indirect github.com/spaolacci/murmur3 v1.1.0 // indirect github.com/stretchr/objx v0.5.2 // indirect - github.com/testcontainers/testcontainers-go v0.33.0 + github.com/testcontainers/testcontainers-go v0.34.0 github.com/tilinna/z85 v1.0.0 // indirect github.com/tklauser/go-sysconf v0.3.13 // indirect github.com/tklauser/numcpus v0.7.0 // indirect - github.com/urfave/cli/v2 v2.27.4 + github.com/urfave/cli/v2 v2.27.5 github.com/vmihailenco/tagparser/v2 v2.0.0 // indirect github.com/xdg-go/pbkdf2 v1.0.0 // indirect github.com/xdg-go/stringprep v1.0.4 // indirect @@ -432,6 +433,4 @@ require ( modernc.org/token v1.1.0 // indirect ) -go 1.22.7 - -toolchain go1.23.0 +go 1.23.2 diff --git a/go.sum b/go.sum index f59df136f6..0e95499902 100644 --- a/go.sum +++ b/go.sum @@ -762,6 +762,8 @@ github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4/go.mod h1:ybxpYRF github.com/andybalholm/brotli v1.0.4/go.mod h1:fO7iG3H7G2nSZ7m0zPUDn85XEX2GTukHGRSepvi9Eig= github.com/andybalholm/brotli v1.1.1 h1:PR2pgnyFznKEugtsUo0xLdDop5SKXd5Qf5ysW+7XdTA= github.com/andybalholm/brotli v1.1.1/go.mod h1:05ib4cKhjx3OQYUY22hTVd34Bc8upXjOLL2rKwwZBoA= +github.com/anicoll/screamer v0.0.0-20241206035431-9ba919b54dfd h1:Bz8Rksav0QRwRNzwdVeZAkMsm0PwNkJ7H4amsYt1xfw= +github.com/anicoll/screamer v0.0.0-20241206035431-9ba919b54dfd/go.mod h1:ZP3Cph589sR4ohmkdCBRVNcaR0YNQXw7E0d3a0YfQjc= github.com/antihax/optional v1.0.0/go.mod h1:uupD/76wgC+ih3iEmQUL+0Ugr19nfwCT1kdvxnR2qWY= github.com/apache/arrow/go/arrow v0.0.0-20200730104253-651201b0f516/go.mod h1:QNYViu/X0HXDHw7m3KXzWSVXIbfUvJqBFe6Gj8/pYA0= github.com/apache/arrow/go/arrow v0.0.0-20211112161151-bc219186db40 h1:q4dksr6ICHXqG5hm0ZW5IHyeEJXoIJSOZeBLmWPNeIQ= @@ -985,10 +987,10 @@ github.com/couchbaselabs/gocaves/client v0.0.0-20230404095311-05e3ba4f0259 h1:2T github.com/couchbaselabs/gocaves/client v0.0.0-20230404095311-05e3ba4f0259/go.mod h1:AVekAZwIY2stsJOMWLAS/0uA/+qdp7pjO8EHnl61QkY= github.com/couchbaselabs/gocbconnstr/v2 v2.0.0-20240607131231-fb385523de28 h1:lhGOw8rNG6RAadmmaJAF3PJ7MNt7rFuWG7BHCYMgnGE= github.com/couchbaselabs/gocbconnstr/v2 v2.0.0-20240607131231-fb385523de28/go.mod h1:o7T431UOfFVHDNvMBUmUxpHnhivwv7BziUao/nMl81E= -github.com/cpuguy83/dockercfg v0.3.1 h1:/FpZ+JaygUR/lZP2NlFI2DVfrOEMAIKP5wWEJdoYe9E= -github.com/cpuguy83/dockercfg v0.3.1/go.mod h1:sugsbF4//dDlL/i+S+rtpIWp+5h0BHJHfjj5/jFyUJc= -github.com/cpuguy83/go-md2man/v2 v2.0.4 h1:wfIWP927BUkWJb2NmU/kNDYIBTh/ziUX91+lVfRxZq4= -github.com/cpuguy83/go-md2man/v2 v2.0.4/go.mod h1:tgQtvFlXSQOSOSIRvRPT7W67SCa46tRHOmNcaadrF8o= +github.com/cpuguy83/dockercfg v0.3.2 h1:DlJTyZGBDlXqUZ2Dk2Q3xHs/FtnooJJVaad2S9GKorA= +github.com/cpuguy83/dockercfg v0.3.2/go.mod h1:sugsbF4//dDlL/i+S+rtpIWp+5h0BHJHfjj5/jFyUJc= +github.com/cpuguy83/go-md2man/v2 v2.0.5 h1:ZtcqGrnekaHpVLArFSe4HK5DoKx1T0rq2DwVB0alcyc= +github.com/cpuguy83/go-md2man/v2 v2.0.5/go.mod h1:tgQtvFlXSQOSOSIRvRPT7W67SCa46tRHOmNcaadrF8o= github.com/creack/pty v1.1.7/go.mod h1:lj5s0c3V2DBrqTV7llrYr5NG6My20zk30Fl46Y7DoTY= github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= github.com/creack/pty v1.1.21 h1:1/QdRyBaHHJP61QkWMXlOIBfsgdDeeKfK8SYVUWJKf0= @@ -1085,8 +1087,8 @@ github.com/getsentry/sentry-go v0.28.1/go.mod h1:1fQZ+7l7eeJ3wYi82q5Hg8GqAPgefRq github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04= github.com/go-errors/errors v1.4.2 h1:J6MZopCL4uSllY1OfXM374weqZFFItUbrImctkmUxIA= github.com/go-errors/errors v1.4.2/go.mod h1:sIVyrIiJhuEF+Pj9Ebtd6P/rEYROXFi3BopGUQ5a5Og= -github.com/go-faker/faker/v4 v4.4.2 h1:96WeU9QKEqRUVYdjHquY2/5bAqmVM0IfGKHV5mbfqmQ= -github.com/go-faker/faker/v4 v4.4.2/go.mod h1:4K3v4AbKXYNHMQNaREMc9/kRB9j5JJzpFo6KHRvrcIw= +github.com/go-faker/faker/v4 v4.5.0 h1:ARzAY2XoOL9tOUK+KSecUQzyXQsUaZHefjyF8x6YFHc= +github.com/go-faker/faker/v4 v4.5.0/go.mod h1:p3oq1GRjG2PZ7yqeFFfQI20Xm61DoBDlCA8RiSyZ48M= github.com/go-faster/city v1.0.1 h1:4WAxSZ3V2Ws4QRDrscLEDcibJY8uf41H6AhXDrNDcGw= github.com/go-faster/city v1.0.1/go.mod h1:jKcUJId49qdW3L1qKHH/3wPeUstCVpVSXTM6vO3VcTw= github.com/go-faster/errors v0.7.1 h1:MkJTnDoEdi9pDabt1dpWf7AA8/BaSYZqibYyhZ20AYg= @@ -1832,8 +1834,8 @@ github.com/stretchr/testify v1.8.2/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o github.com/stretchr/testify v1.8.3/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXlSw2iwfAo= github.com/stretchr/testify v1.9.0 h1:HtqpIVDClZ4nwg75+f6Lvsy/wHu+3BoSGCbBAcpTsTg= github.com/stretchr/testify v1.9.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY= -github.com/testcontainers/testcontainers-go v0.33.0 h1:zJS9PfXYT5O0ZFXM2xxXfk4J5UMw/kRiISng037Gxdw= -github.com/testcontainers/testcontainers-go v0.33.0/go.mod h1:W80YpTa8D5C3Yy16icheD01UTDu+LmXIA2Keo+jWtT8= +github.com/testcontainers/testcontainers-go v0.34.0 h1:5fbgF0vIN5u+nD3IWabQwRybuB4GY8G2HHgCkbMzMHo= +github.com/testcontainers/testcontainers-go v0.34.0/go.mod h1:6P/kMkQe8yqPHfPWNulFGdFHTD8HB2vLq/231xY2iPQ= github.com/testcontainers/testcontainers-go/modules/ollama v0.32.0 h1:nuYlIE4zOGd8m+TzjY0v41kyfYre3inp/iw1p4qn2eU= github.com/testcontainers/testcontainers-go/modules/ollama v0.32.0/go.mod h1:PeIvLbruDuwReyIbJT6Km+ZRxSaqLsS8VcTqeDgxs1A= github.com/testcontainers/testcontainers-go/modules/qdrant v0.32.0 h1:rsk1UKBcu3s5V/msn6Hsizx7f70SPDyLZKlgz0xDEJ8= @@ -1881,8 +1883,8 @@ github.com/uptrace/bun/dialect/pgdialect v1.1.12 h1:m/CM1UfOkoBTglGO5CUTKnIKKOAp github.com/uptrace/bun/dialect/pgdialect v1.1.12/go.mod h1:Ij6WIxQILxLlL2frUBxUBOZJtLElD2QQNDcu/PWDHTc= github.com/uptrace/bun/driver/pgdriver v1.1.12 h1:3rRWB1GK0psTJrHwxzNfEij2MLibggiLdTqjTtfHc1w= github.com/uptrace/bun/driver/pgdriver v1.1.12/go.mod h1:ssYUP+qwSEgeDDS1xm2XBip9el1y9Mi5mTAvLoiADLM= -github.com/urfave/cli/v2 v2.27.4 h1:o1owoI+02Eb+K107p27wEX9Bb8eqIoZCfLXloLUSWJ8= -github.com/urfave/cli/v2 v2.27.4/go.mod h1:m4QzxcD2qpra4z7WhzEGn74WZLViBnMpb1ToCAKdGRQ= +github.com/urfave/cli/v2 v2.27.5 h1:WoHEJLdsXr6dDWoJgMq/CboDmyY/8HMMH1fTECbih+w= +github.com/urfave/cli/v2 v2.27.5/go.mod h1:3Sevf16NykTbInEnD0yKkjDAeZDS0A6bzhBH5hrMvTQ= github.com/vmihailenco/bufpool v0.1.11 h1:gOq2WmBrq0i2yW5QJ16ykccQ4wH9UyEsgLm6czKAd94= github.com/vmihailenco/bufpool v0.1.11/go.mod h1:AFf/MOy3l2CFTKbxwt0mp2MwnqjNEs5H/UxrkA5jxTQ= github.com/vmihailenco/msgpack/v5 v5.3.5/go.mod h1:7xyJ9e+0+9SaZT0Wt1RGleJXzli6Q/V5KbhBonMG9jc= @@ -2216,8 +2218,8 @@ golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20220819030929-7fc1605a5dde/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20220929204114-8fcdb60fdcc0/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.1.0/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.8.0 h1:3NFvSEYkUoMifnESzZl15y791HH1qU2xm6eCJU5ZPXQ= -golang.org/x/sync v0.8.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= +golang.org/x/sync v0.9.0 h1:fEo0HyrW1GIgZdpbhCRO0PkJajUS5H9IFUztCgEo2jQ= +golang.org/x/sync v0.9.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20181116152217-5ac8a444bdc5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= diff --git a/internal/impl/gcp/spanner_change_stream.go b/internal/impl/gcp/spanner_change_stream.go new file mode 100644 index 0000000000..375b289030 --- /dev/null +++ b/internal/impl/gcp/spanner_change_stream.go @@ -0,0 +1,281 @@ +// Copyright 2024 Redpanda Data, Inc. +// +// 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 gcp + +import ( + "context" + "encoding/json" + "errors" + "fmt" + "slices" + "time" + + "cloud.google.com/go/spanner" + "github.com/Jeffail/shutdown" + "github.com/anicoll/screamer/pkg/model" + "github.com/anicoll/screamer/pkg/partitionstorage" + "github.com/anicoll/screamer/pkg/screamer" + "github.com/redpanda-data/benthos/v4/public/service" +) + +type streamReader interface { + Stream(ctx context.Context, channel chan<- *model.DataChangeRecord) error + Close() error +} + +func newSpannerChangeStreamInputConfig() *service.ConfigSpec { + return service.NewConfigSpec(). + Beta(). + Version("3.43.0"). + Categories("Services", "GCP"). + Summary("Creates an input that consumes from a spanner change stream."). + Field(service.NewStringField("partition_dsn")). + Field(service.NewStringField("partition_table")). + Field(service.NewBoolField("use_in_mememory_partition").Description("use an in memory partition table for tracking the partitions").Default(false)). + Field(service.NewStringField("stream_dsn").Optional().Default("")). + Field(service.NewStringField("stream_id").Description("The name of the change stream to track").Default("")). + Field(service.NewIntField("start_time_epoch").Optional().Description("Microsecond accurate epoch timestamp to start reading from").Default(0)). + Field(service.NewStringListField("allowed_mod_types").Default([]string{"INSERT", "UPDATE", "DELETE"})) +} + +func newSpannerStreamInput(conf *service.ParsedConfig, log *service.Logger) (out *spannerStreamInput, err error) { + out = &spannerStreamInput{ + // not buffered to prevent the cursor from getting too far ahead. + // there is still the chance that we could lose changes though. + changeChannel: make(chan *model.DataChangeRecord, 1), + log: log, + shutdownSig: shutdown.NewSignaller(), + } + out.partitionDSN, err = conf.FieldString("partition_dsn") + if err != nil { + return + } + + out.partitionTable, err = conf.FieldString("partition_table") + if err != nil { + return + } + + out.streamDSN, err = conf.FieldString("stream_dsn") + if err != nil { + return + } + + out.streamID, err = conf.FieldString("stream_id") + if err != nil { + return + } + + useInMemPartition, err := conf.FieldBool("use_in_mememory_partition") + if err != nil { + return + } + + startTimeEpoch, err := conf.FieldInt("start_time_epoch") + if err != nil { + return + } + + if startTimeEpoch > 0 { + out.startTime = func(seconds int) *time.Time { + t := time.UnixMicro(int64(startTimeEpoch)) + return &t + }(startTimeEpoch) + } + + out.allowedModTypes, err = conf.FieldStringList("allowed_mod_types") + if err != nil { + return + } + if !useInMemPartition && slices.Contains([]string{out.partitionDSN, out.partitionTable, out.streamDSN, out.streamID}, "") { + return nil, errors.New("partition_dsn, partition_table, stream_dsn, and stream_id must be set") + } else if slices.Contains([]string{out.streamDSN, out.streamID}, "") { + return nil, errors.New("stream_dsn, and stream_id must be set") + } + out.usePartitionTable = !useInMemPartition + return +} + +func init() { + err := service.RegisterInput( + "spanner_change_stream", newSpannerChangeStreamInputConfig(), + func(conf *service.ParsedConfig, mgr *service.Resources) (service.Input, error) { + return newSpannerStreamInput(conf, mgr.Logger()) + }) + if err != nil { + panic(err) + } +} + +//------------------------------------------------------------------------------ + +type spannerStreamInput struct { + streamDSN string + streamClient *db + streamID string + partitionDSN string + partitionTable string + usePartitionTable bool + startTime *time.Time + allowedModTypes []string + reader streamReader + // create a channel to pass from connection to read. + changeChannel chan *model.DataChangeRecord + + log *service.Logger + shutdownSig *shutdown.Signaller +} + +func (i *spannerStreamInput) Connect(ctx context.Context) (err error) { + jobctx, _ := i.shutdownSig.SoftStopCtx(context.Background()) + + if i.streamClient == nil { + i.streamClient, err = newDatabase(jobctx, i.streamDSN) + if err != nil { + return err + } + } + if i.reader == nil { + i.reader, err = newStreamer(jobctx, i.streamClient, i.streamID, i.partitionDSN, i.partitionTable, i.usePartitionTable, i.allowedModTypes, i.startTime) + if err != nil { + return err + } + } + go func() { + if rerr := i.reader.Stream(jobctx, i.changeChannel); rerr != nil { + i.log.Errorf("Subscription error: %v\n", rerr) + close(i.changeChannel) + panic(rerr) + } + }() + return nil +} + +func (i *spannerStreamInput) Read(ctx context.Context) (*service.Message, service.AckFunc, error) { + msg := <-i.changeChannel + data, err := json.Marshal(msg) + if err != nil { + return nil, nil, err + } + return service.NewMessage(data), func(ctx context.Context, err error) error { + // Nacks are retried automatically when we use service.AutoRetryNacks + return nil + }, nil +} + +func (i *spannerStreamInput) Close(_ context.Context) error { + close(i.changeChannel) + if i.reader != nil { + return i.reader.Close() + } + return nil +} + +// ------------------------- + +type streamerDB struct { + streamID, partitionTable string + changeStreamClient, partitionClient *db + subscriber *screamer.Subscriber + allowedModTypes []string +} + +func newStreamer(ctx context.Context, + changestreamClient *db, + streamID, partitionDSN, partitionTable string, + usePartitionTable bool, + modTypes []string, + startTime *time.Time, +) (streamReader, error) { + streamer := &streamerDB{ + streamID: streamID, + partitionTable: partitionTable, + allowedModTypes: modTypes, + changeStreamClient: changestreamClient, + } + + var pStorage screamer.PartitionStorage = partitionstorage.NewInmemory() + // only use DB meta partition table if explicitly enabled. + if usePartitionTable { + partitionClient, err := newDatabase(ctx, partitionDSN) + if err != nil { + return nil, err + } + streamer.partitionClient = partitionClient + + spannerPartitionStorage := partitionstorage.NewSpanner(partitionClient.client, partitionTable) + if err := spannerPartitionStorage.CreateTableIfNotExists(ctx); err != nil { + return nil, err + } + // assign here as we need to use the partition storage in the subscriber. + pStorage = spannerPartitionStorage + } + + options := []screamer.Option{} + // if provided with a specific startime. use that. + if startTime != nil { + options = append(options, screamer.WithStartTimestamp(*startTime)) + } + subscriber := screamer.NewSubscriber(streamer.changeStreamClient.client, streamID, pStorage, options...) + + streamer.subscriber = subscriber + + return streamer, nil +} + +// Stream provides a stream of change records from a Spanner database configured stream to your provided channel. +// Stream is blocking unless the provided context is cancelled or an error occurs. +func (s *streamerDB) Stream(ctx context.Context, channel chan<- *model.DataChangeRecord) error { + return s.subscriber.SubscribeFunc(ctx, func(dcr *model.DataChangeRecord) error { + if slices.Contains(s.allowedModTypes, string(dcr.ModType)) { + channel <- dcr + } + return nil + }) +} + +func (s *streamerDB) Close() error { + if s.changeStreamClient != nil { + s.changeStreamClient.Close() + } + if s.partitionClient != nil { + s.partitionClient.Close() + } + return nil +} + +// ----------------- + +type db struct { + client *spanner.Client +} + +func newDatabase(ctx context.Context, dsn string) (*db, error) { + spannerClient, err := spanner.NewClient(ctx, dsn) + if err != nil { + return nil, fmt.Errorf("spanner.NewClient: %w", err) + } + + return &db{ + client: spannerClient, + }, nil +} + +func (s *db) Close() error { + if s.client != nil { + s.client.Close() + } + return nil +} diff --git a/internal/impl/gcp/spanner_change_stream_mock_test.go b/internal/impl/gcp/spanner_change_stream_mock_test.go new file mode 100644 index 0000000000..6e7beea0a5 --- /dev/null +++ b/internal/impl/gcp/spanner_change_stream_mock_test.go @@ -0,0 +1,39 @@ +// Copyright 2024 Redpanda Data, Inc. +// +// 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 gcp + +import ( + "context" + + "github.com/anicoll/screamer/pkg/model" + "github.com/stretchr/testify/mock" +) + +type mockStreamReader struct { + mock.Mock +} + +var _ streamReader = &mockStreamReader{} + +func (mt *mockStreamReader) Stream(ctx context.Context, channel chan<- *model.DataChangeRecord) error { + args := mt.Called(ctx, channel) + + return args.Error(0) +} + +func (mt *mockStreamReader) Close() error { + args := mt.Called() + return args.Error(0) +} diff --git a/internal/impl/gcp/spanner_change_stream_test.go b/internal/impl/gcp/spanner_change_stream_test.go new file mode 100644 index 0000000000..1f3091969b --- /dev/null +++ b/internal/impl/gcp/spanner_change_stream_test.go @@ -0,0 +1,150 @@ +// Copyright 2024 Redpanda Data, Inc. +// +// 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 gcp + +import ( + "context" + "encoding/json" + "testing" + "time" + + "cloud.google.com/go/spanner/spannertest" + "github.com/anicoll/screamer/pkg/model" + "github.com/google/uuid" + "github.com/stretchr/testify/mock" + "github.com/stretchr/testify/require" +) + +var testSpannerStreamInputYAML = ` +stream_dsn: "projects/test-project/instances/test-instance/databases/test-db" +stream_id: "OutboxStream" +use_in_mememory_partition: true +partition_dsn: "projects/test/instances/test/databases/test-events-md" # optional default "" +partition_table: "meta_partitions_table" # optional default "" +allowed_mod_types: + - "INSERT" + - "UPDATE" + - "DELETE" +` + +func TestGCPSpannerChangeStreamInput_Read(t *testing.T) { + spec := newSpannerChangeStreamInputConfig() + + parsed, err := spec.ParseYAML(testSpannerStreamInputYAML, nil) + require.NoError(t, err) + + proc, err := newSpannerStreamInput(parsed, nil) + require.NoError(t, err) + + dataChangeRecord := &model.DataChangeRecord{ + CommitTimestamp: time.Now(), + RecordSequence: "0000001", + ServerTransactionID: uuid.NewString(), + IsLastRecordInTransactionInPartition: true, + TableName: "test_table", + ColumnTypes: []*model.ColumnType{ + {Name: "ID", Type: model.Type{Code: model.TypeCode_INT64}}, + {Name: "Value", Type: model.Type{Code: model.TypeCode_STRING}}, + }, + Mods: []*model.Mod{ + { + Keys: map[string]interface{}{}, + NewValues: map[string]interface{}{}, + OldValues: map[string]interface{}{}, + }, + }, + ModType: model.ModType_INSERT, + NumberOfRecordsInTransaction: 1, + NumberOfPartitionsInTransaction: 2, + } + proc.changeChannel <- dataChangeRecord + ctx := context.Background() + + msg, _, err := proc.Read(ctx) + require.NoError(t, err) + require.NotNil(t, msg) + + expectedMsg, err := json.Marshal(dataChangeRecord) + require.NoError(t, err) + + gotMsg, err := msg.AsBytes() + require.NoError(t, err) + require.Equal(t, expectedMsg, gotMsg) +} + +func TestGCPSpannerChangeStreamInput_Connect(t *testing.T) { + spec := newSpannerChangeStreamInputConfig() + ctx, cancel := context.WithCancel(context.Background()) + + parsed, err := spec.ParseYAML(testSpannerStreamInputYAML, nil) + require.NoError(t, err) + + proc, err := newSpannerStreamInput(parsed, nil) + require.NoError(t, err) + + srv, err := spannertest.NewServer("localhost:0") + require.NoError(t, err) + t.Setenv("SPANNER_EMULATOR_HOST", srv.Addr) + + t.Cleanup(func() { + srv.Close() + }) + + mockStreamReader := &mockStreamReader{} + proc.reader = mockStreamReader + + mockStreamReader.On("Stream", ctx, mock.Anything).Once().Return(nil) + defer mockStreamReader.AssertExpectations(t) + + err = proc.Connect(ctx) + require.NoError(t, err) + cancel() + time.Sleep(time.Millisecond * 100) +} + +func TestGCPSpannerChangeStreamInput_Close(t *testing.T) { + spec := newSpannerChangeStreamInputConfig() + ctx, cancel := context.WithCancel(context.Background()) + + parsed, err := spec.ParseYAML(testSpannerStreamInputYAML, nil) + require.NoError(t, err) + + proc, err := newSpannerStreamInput(parsed, nil) + require.NoError(t, err) + + srv, err := spannertest.NewServer("localhost:0") + require.NoError(t, err) + t.Setenv("SPANNER_EMULATOR_HOST", srv.Addr) + + t.Cleanup(func() { + srv.Close() + }) + + mockStreamReader := &mockStreamReader{} + defer mockStreamReader.AssertExpectations(t) + proc.reader = mockStreamReader + + mockStreamReader.On("Stream", ctx, mock.Anything).Once().Return(nil) + + mockStreamReader.On("Close", mock.Anything).Once().Return(nil) + + err = proc.Connect(ctx) + require.NoError(t, err) + cancel() + time.Sleep(time.Millisecond * 100) + + err = proc.Close(context.Background()) + require.NoError(t, err) +} From ab644a1369a3767f539dae97b54fd23769461fa2 Mon Sep 17 00:00:00 2001 From: Andrew Nicoll Date: Mon, 9 Dec 2024 13:31:20 +1030 Subject: [PATCH 2/7] rename files --- .../{spanner_change_stream.go => input_spanner_change_stream.go} | 0 ...ream_mock_test.go => input_spanner_change_stream_mock_test.go} | 0 ..._change_stream_test.go => input_spanner_change_stream_test.go} | 0 3 files changed, 0 insertions(+), 0 deletions(-) rename internal/impl/gcp/{spanner_change_stream.go => input_spanner_change_stream.go} (100%) rename internal/impl/gcp/{spanner_change_stream_mock_test.go => input_spanner_change_stream_mock_test.go} (100%) rename internal/impl/gcp/{spanner_change_stream_test.go => input_spanner_change_stream_test.go} (100%) diff --git a/internal/impl/gcp/spanner_change_stream.go b/internal/impl/gcp/input_spanner_change_stream.go similarity index 100% rename from internal/impl/gcp/spanner_change_stream.go rename to internal/impl/gcp/input_spanner_change_stream.go diff --git a/internal/impl/gcp/spanner_change_stream_mock_test.go b/internal/impl/gcp/input_spanner_change_stream_mock_test.go similarity index 100% rename from internal/impl/gcp/spanner_change_stream_mock_test.go rename to internal/impl/gcp/input_spanner_change_stream_mock_test.go diff --git a/internal/impl/gcp/spanner_change_stream_test.go b/internal/impl/gcp/input_spanner_change_stream_test.go similarity index 100% rename from internal/impl/gcp/spanner_change_stream_test.go rename to internal/impl/gcp/input_spanner_change_stream_test.go From 4b734f339da4be4a9a285045b9fa87390f0c77e2 Mon Sep 17 00:00:00 2001 From: Andrew Nicoll Date: Mon, 9 Dec 2024 15:57:47 +1030 Subject: [PATCH 3/7] update documentation --- .../inputs/gcp_spanner_change_stream.adoc | 148 ++++++++++++++++++ .../impl/gcp/input_spanner_change_stream.go | 36 ++--- .../gcp/input_spanner_change_stream_test.go | 7 +- internal/plugins/info.csv | 1 + 4 files changed, 170 insertions(+), 22 deletions(-) create mode 100644 docs/modules/components/pages/inputs/gcp_spanner_change_stream.adoc diff --git a/docs/modules/components/pages/inputs/gcp_spanner_change_stream.adoc b/docs/modules/components/pages/inputs/gcp_spanner_change_stream.adoc new file mode 100644 index 0000000000..43a69aa036 --- /dev/null +++ b/docs/modules/components/pages/inputs/gcp_spanner_change_stream.adoc @@ -0,0 +1,148 @@ += gcp_spanner_change_stream +: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 3.43.0. + + +[tabs] +====== +Common:: ++ +-- + +```yml +# Common config fields, showing default values +input: + label: "" + gcp_spanner_change_stream: + stream_dsn: projects//instances//databases/ # No default (required) + stream_id: "" + partition_dsn: projects//instances//databases/ # No default (optional) + partition_table: "" # No default (optional) + use_in_mememory_partition: false +``` + +-- +Advanced:: ++ +-- + +```yml +# All config fields, showing default values +input: + label: "" + gcp_spanner_change_stream: + stream_dsn: projects//instances//databases/ # No default (required) + stream_id: "" + start_time_epoch: 0 + partition_dsn: projects//instances//databases/ # No default (optional) + partition_table: "" # No default (optional) + use_in_mememory_partition: false + allowed_mod_types: + - INSERT + - UPDATE + - DELETE +``` + +-- +====== + +== Fields + +=== `stream_dsn` + +Required field to use to connect to spanner for the change stream. + + +*Type*: `string` + + +```yml +# Examples + +stream_dsn: projects//instances//databases/ +``` + +=== `stream_id` + +Required name of the change stream to track. + + +*Type*: `string` + +*Default*: `""` + +=== `start_time_epoch` + +Optional microsecond accurate epoch timestamp to start reading from. If empty time.Now() will be used. + + +*Type*: `int` + +*Default*: `0` + +=== `partition_dsn` + +Field used to set the DSN for the metadata partition table, can be the same as stream_dsn. + + +*Type*: `string` + + +```yml +# Examples + +partition_dsn: projects//instances//databases/ +``` + +=== `partition_table` + +Name of the table to create/use in spanner to track change stream partition metadata. + + +*Type*: `string` + + +=== `use_in_mememory_partition` + +use an in memory partition table for tracking the partitions. + + +*Type*: `bool` + +*Default*: `false` + +=== `allowed_mod_types` + +Mod types to allow through when reading the change stream, default all. + + +*Type*: `array` + +*Default*: `["INSERT","UPDATE","DELETE"]` + + diff --git a/internal/impl/gcp/input_spanner_change_stream.go b/internal/impl/gcp/input_spanner_change_stream.go index 375b289030..acaa47c617 100644 --- a/internal/impl/gcp/input_spanner_change_stream.go +++ b/internal/impl/gcp/input_spanner_change_stream.go @@ -41,13 +41,13 @@ func newSpannerChangeStreamInputConfig() *service.ConfigSpec { Version("3.43.0"). Categories("Services", "GCP"). Summary("Creates an input that consumes from a spanner change stream."). - Field(service.NewStringField("partition_dsn")). - Field(service.NewStringField("partition_table")). - Field(service.NewBoolField("use_in_mememory_partition").Description("use an in memory partition table for tracking the partitions").Default(false)). - Field(service.NewStringField("stream_dsn").Optional().Default("")). - Field(service.NewStringField("stream_id").Description("The name of the change stream to track").Default("")). - Field(service.NewIntField("start_time_epoch").Optional().Description("Microsecond accurate epoch timestamp to start reading from").Default(0)). - Field(service.NewStringListField("allowed_mod_types").Default([]string{"INSERT", "UPDATE", "DELETE"})) + Field(service.NewStringField("stream_dsn").Description("Required field to use to connect to spanner for the change stream.").Example("projects//instances//databases/")). + Field(service.NewStringField("stream_id").Description("Required name of the change stream to track.").Default("")). + Field(service.NewIntField("start_time_epoch").Advanced().Optional().Default(0).Description("Optional microsecond accurate epoch timestamp to start reading from. If empty time.Now() will be used.")). + Field(service.NewStringField("partition_dsn").Optional().Description("Field used to set the DSN for the metadata partition table, can be the same as stream_dsn.").Example("projects//instances//databases/")). + Field(service.NewStringField("partition_table").Optional().Description("Name of the table to create/use in spanner to track change stream partition metadata.")). + Field(service.NewBoolField("use_in_mememory_partition").Description("use an in memory partition table for tracking the partitions.").Default(false)). + Field(service.NewStringListField("allowed_mod_types").Advanced().Description("Mod types to allow through when reading the change stream, default all.").Default([]string{"INSERT", "UPDATE", "DELETE"})) } func newSpannerStreamInput(conf *service.ParsedConfig, log *service.Logger) (out *spannerStreamInput, err error) { @@ -58,23 +58,23 @@ func newSpannerStreamInput(conf *service.ParsedConfig, log *service.Logger) (out log: log, shutdownSig: shutdown.NewSignaller(), } - out.partitionDSN, err = conf.FieldString("partition_dsn") - if err != nil { + if out.partitionDSN, err = conf.FieldString("partition_dsn"); err != nil { return } - out.partitionTable, err = conf.FieldString("partition_table") - if err != nil { + if out.partitionTable, err = conf.FieldString("partition_table"); err != nil { return } - out.streamDSN, err = conf.FieldString("stream_dsn") - if err != nil { + if out.streamDSN, err = conf.FieldString("stream_dsn"); err != nil { return } - out.streamID, err = conf.FieldString("stream_id") - if err != nil { + if out.streamID, err = conf.FieldString("stream_id"); err != nil { + return + } + + if out.allowedModTypes, err = conf.FieldStringList("allowed_mod_types"); err != nil { return } @@ -95,10 +95,6 @@ func newSpannerStreamInput(conf *service.ParsedConfig, log *service.Logger) (out }(startTimeEpoch) } - out.allowedModTypes, err = conf.FieldStringList("allowed_mod_types") - if err != nil { - return - } if !useInMemPartition && slices.Contains([]string{out.partitionDSN, out.partitionTable, out.streamDSN, out.streamID}, "") { return nil, errors.New("partition_dsn, partition_table, stream_dsn, and stream_id must be set") } else if slices.Contains([]string{out.streamDSN, out.streamID}, "") { @@ -110,7 +106,7 @@ func newSpannerStreamInput(conf *service.ParsedConfig, log *service.Logger) (out func init() { err := service.RegisterInput( - "spanner_change_stream", newSpannerChangeStreamInputConfig(), + "gcp_spanner_change_stream", newSpannerChangeStreamInputConfig(), func(conf *service.ParsedConfig, mgr *service.Resources) (service.Input, error) { return newSpannerStreamInput(conf, mgr.Logger()) }) diff --git a/internal/impl/gcp/input_spanner_change_stream_test.go b/internal/impl/gcp/input_spanner_change_stream_test.go index 1f3091969b..bbb6d4a759 100644 --- a/internal/impl/gcp/input_spanner_change_stream_test.go +++ b/internal/impl/gcp/input_spanner_change_stream_test.go @@ -48,6 +48,9 @@ func TestGCPSpannerChangeStreamInput_Read(t *testing.T) { proc, err := newSpannerStreamInput(parsed, nil) require.NoError(t, err) + mockStreamReader := &mockStreamReader{} + proc.reader = mockStreamReader + dataChangeRecord := &model.DataChangeRecord{ CommitTimestamp: time.Now(), RecordSequence: "0000001", @@ -105,7 +108,7 @@ func TestGCPSpannerChangeStreamInput_Connect(t *testing.T) { mockStreamReader := &mockStreamReader{} proc.reader = mockStreamReader - mockStreamReader.On("Stream", ctx, mock.Anything).Once().Return(nil) + mockStreamReader.On("Stream", mock.AnythingOfType("*context.cancelCtx"), mock.Anything).Once().Return(nil) defer mockStreamReader.AssertExpectations(t) err = proc.Connect(ctx) @@ -136,7 +139,7 @@ func TestGCPSpannerChangeStreamInput_Close(t *testing.T) { defer mockStreamReader.AssertExpectations(t) proc.reader = mockStreamReader - mockStreamReader.On("Stream", ctx, mock.Anything).Once().Return(nil) + mockStreamReader.On("Stream", mock.AnythingOfType("*context.cancelCtx"), mock.Anything).Once().Return(nil) mockStreamReader.On("Close", mock.Anything).Once().Return(nil) diff --git a/internal/plugins/info.csv b/internal/plugins/info.csv index 397eb732c5..c57c787274 100644 --- a/internal/plugins/info.csv +++ b/internal/plugins/info.csv @@ -84,6 +84,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_change_stream ,input ,gcp_spanner_change_stream ,0.0.0 ,community ,n ,n ,n 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 From cb776f6ae08e5c176875b9a603c6dc8e7340ff59 Mon Sep 17 00:00:00 2001 From: Andrew Nicoll Date: Mon, 16 Dec 2024 12:59:43 +1030 Subject: [PATCH 4/7] full spanner CDC implementation --- go.mod | 39 +- go.sum | 82 ++- ..._change_stream.go => input_spanner_cdc.go} | 97 ++-- ...test.go => input_spanner_cdc_mock_test.go} | 5 +- ...ream_test.go => input_spanner_cdc_test.go} | 21 +- internal/impl/gcp/spannercdc/changerecord.go | 190 +++++++ internal/impl/gcp/spannercdc/config.go | 15 + internal/impl/gcp/spannercdc/consumer.go | 16 + internal/impl/gcp/spannercdc/helper_test.go | 103 ++++ internal/impl/gcp/spannercdc/inmemory.go | 159 ++++++ internal/impl/gcp/spannercdc/partition.go | 39 ++ internal/impl/gcp/spannercdc/reader.go | 328 +++++++++++ internal/impl/gcp/spannercdc/spanner.go | 303 +++++++++++ internal/impl/gcp/spannercdc/spanner_test.go | 515 ++++++++++++++++++ 14 files changed, 1797 insertions(+), 115 deletions(-) rename internal/impl/gcp/{input_spanner_change_stream.go => input_spanner_cdc.go} (60%) rename internal/impl/gcp/{input_spanner_change_stream_mock_test.go => input_spanner_cdc_mock_test.go} (89%) rename internal/impl/gcp/{input_spanner_change_stream_test.go => input_spanner_cdc_test.go} (88%) create mode 100644 internal/impl/gcp/spannercdc/changerecord.go create mode 100644 internal/impl/gcp/spannercdc/config.go create mode 100644 internal/impl/gcp/spannercdc/consumer.go create mode 100644 internal/impl/gcp/spannercdc/helper_test.go create mode 100644 internal/impl/gcp/spannercdc/inmemory.go create mode 100644 internal/impl/gcp/spannercdc/partition.go create mode 100644 internal/impl/gcp/spannercdc/reader.go create mode 100644 internal/impl/gcp/spannercdc/spanner.go create mode 100644 internal/impl/gcp/spannercdc/spanner_test.go diff --git a/go.mod b/go.mod index 10fbbf2880..fd6e1f36b3 100644 --- a/go.mod +++ b/go.mod @@ -3,7 +3,7 @@ module github.com/redpanda-data/connect/v4 replace github.com/99designs/keyring => github.com/Jeffail/keyring v1.2.3 require ( - cloud.google.com/go/aiplatform v1.68.0 + cloud.google.com/go/aiplatform v1.69.0 cloud.google.com/go/bigquery v1.64.0 cloud.google.com/go/pubsub v1.45.1 cloud.google.com/go/spanner v1.73.0 @@ -26,7 +26,6 @@ require ( github.com/Masterminds/squirrel v1.5.4 github.com/PaesslerAG/gval v1.2.2 github.com/PaesslerAG/jsonpath v0.1.1 - github.com/anicoll/screamer v0.0.0-20241206035431-9ba919b54dfd github.com/apache/pulsar-client-go v0.13.1 github.com/authzed/authzed-go v1.0.0 github.com/authzed/grpcutil v0.0.0-20240123194739-2ea1e3d2d98b @@ -117,7 +116,7 @@ require ( github.com/smira/go-statsd v1.3.3 github.com/snowflakedb/gosnowflake v1.11.0 github.com/sourcegraph/conc v0.3.0 - github.com/stretchr/testify v1.9.0 + github.com/stretchr/testify v1.10.0 github.com/testcontainers/testcontainers-go/modules/ollama v0.32.0 github.com/testcontainers/testcontainers-go/modules/qdrant v0.32.0 github.com/tetratelabs/wazero v1.7.3 @@ -143,13 +142,13 @@ require ( go.opentelemetry.io/otel/sdk v1.29.0 go.opentelemetry.io/otel/trace v1.29.0 go.uber.org/multierr v1.11.0 - golang.org/x/crypto v0.28.0 + golang.org/x/crypto v0.30.0 golang.org/x/exp v0.0.0-20241009180824-f66d83c29e7c - golang.org/x/net v0.30.0 - golang.org/x/sync v0.9.0 - golang.org/x/text v0.19.0 - google.golang.org/api v0.205.0 - google.golang.org/protobuf v1.35.1 + golang.org/x/net v0.32.0 + golang.org/x/sync v0.10.0 + golang.org/x/text v0.21.0 + google.golang.org/api v0.211.0 + google.golang.org/protobuf v1.35.2 modernc.org/sqlite v1.32.0 ) @@ -187,8 +186,8 @@ require ( require ( cloud.google.com/go v0.116.0 // indirect - cloud.google.com/go/auth v0.10.1 // indirect - cloud.google.com/go/auth/oauth2adapt v0.2.5 // indirect + cloud.google.com/go/auth v0.12.1 // indirect + cloud.google.com/go/auth/oauth2adapt v0.2.6 // indirect cloud.google.com/go/compute/metadata v0.5.2 // indirect cloud.google.com/go/iam v1.2.2 // indirect cloud.google.com/go/trace v1.11.2 // indirect @@ -293,7 +292,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.4 // indirect - github.com/googleapis/gax-go/v2 v2.13.0 // indirect + github.com/googleapis/gax-go/v2 v2.14.0 // indirect github.com/gorilla/css v1.0.1 // indirect github.com/gorilla/handlers v1.5.2 // indirect github.com/gorilla/mux v1.8.1 // indirect @@ -407,16 +406,16 @@ require ( go.uber.org/atomic v1.11.0 // indirect go.uber.org/zap v1.27.0 // indirect golang.org/x/mod v0.21.0 // indirect - golang.org/x/oauth2 v0.23.0 - golang.org/x/sys v0.26.0 // indirect - golang.org/x/term v0.25.0 // indirect - golang.org/x/time v0.7.0 // indirect + golang.org/x/oauth2 v0.24.0 + golang.org/x/sys v0.28.0 // indirect + golang.org/x/term v0.27.0 // indirect + golang.org/x/time v0.8.0 // indirect golang.org/x/tools v0.26.0 // indirect golang.org/x/xerrors v0.0.0-20240903120638-7835f813f4da // indirect - google.golang.org/genproto v0.0.0-20241113202542-65e8d215514f // indirect - google.golang.org/genproto/googleapis/api v0.0.0-20241104194629-dd2ea8efbc28 // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20241113202542-65e8d215514f // indirect - google.golang.org/grpc v1.68.0 + google.golang.org/genproto v0.0.0-20241118233622-e639e219e697 // indirect + google.golang.org/genproto/googleapis/api v0.0.0-20241118233622-e639e219e697 // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20241206012308-a4fef0638583 // indirect + google.golang.org/grpc v1.68.1 gopkg.in/inf.v0 v0.9.1 // indirect gopkg.in/jcmturner/aescts.v1 v1.0.1 // indirect gopkg.in/jcmturner/dnsutils.v1 v1.0.1 // indirect diff --git a/go.sum b/go.sum index 0e95499902..86c572e8e8 100644 --- a/go.sum +++ b/go.sum @@ -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.68.0 h1:EPPqgHDJpBZKRvv+OsB3cr0jYz3EL2pZ+802rBPcG8U= -cloud.google.com/go/aiplatform v1.68.0/go.mod h1:105MFA3svHjC3Oazl7yjXAmIR89LKhRAeNdnDKJczME= +cloud.google.com/go/aiplatform v1.69.0 h1:XvBzK8e6/6ufbi/i129Vmn/gVqFwbNPmRQ89K+MGlgc= +cloud.google.com/go/aiplatform v1.69.0/go.mod h1:nUsIqzS3khlnWvpjfJbP+2+h+VrFyYsTm7RNCAViiY8= 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,10 +103,10 @@ 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.10.1 h1:TnK46qldSfHWt2a0b/hciaiVJsmDXWy9FqyUan0uYiI= -cloud.google.com/go/auth v0.10.1/go.mod h1:xxA5AqpDrvS+Gkmo9RqrGGRh6WSNKKOXhY3zNOr38tI= -cloud.google.com/go/auth/oauth2adapt v0.2.5 h1:2p29+dePqsCHPP1bqDJcKj4qxRyYCcbzKpFyKGt3MTk= -cloud.google.com/go/auth/oauth2adapt v0.2.5/go.mod h1:AlmsELtlEBnaNTL7jCj8VQFLy6mbZv0s4Q7NGBeQ5E8= +cloud.google.com/go/auth v0.12.1 h1:n2Bj25BUMM0nvE9D2XLTiImanwZhO3DkfWSYS/SAJP4= +cloud.google.com/go/auth v0.12.1/go.mod h1:BFMu+TNpF3DmvfBO9ClqTR/SiqVIm7LukKF9mbendF4= +cloud.google.com/go/auth/oauth2adapt v0.2.6 h1:V6a6XDu2lTwPZWOawrAa9HUK+DB2zfJyTuciBG5hFkU= +cloud.google.com/go/auth/oauth2adapt v0.2.6/go.mod h1:AlmsELtlEBnaNTL7jCj8VQFLy6mbZv0s4Q7NGBeQ5E8= cloud.google.com/go/automl v1.5.0/go.mod h1:34EjfoFGMZ5sgJ9EoLsRtdPSNZLcfflJR39VbVNS2M0= cloud.google.com/go/automl v1.6.0/go.mod h1:ugf8a6Fx+zP0D59WLhqgTDsQI9w07o64uf/Is3Nh5p8= cloud.google.com/go/automl v1.7.0/go.mod h1:RL9MYCCsJEOmt0Wf3z9uzG0a7adTT1fe+aObgSpkCt8= @@ -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.22.2 h1:9Bi8YO+WBE0YSSQL1tX62Gy/KcdNGLufyVlEJ0eYMrc= -cloud.google.com/go/datacatalog v1.22.2/go.mod h1:9Wamq8TDfL2680Sav7q3zEhBJSPBrDxJU8WtPJ25dBM= +cloud.google.com/go/datacatalog v1.23.0 h1:9F2zIbWNNmtrSkPIyGRQNsIugG5VgVVFip6+tXSdWLg= +cloud.google.com/go/datacatalog v1.23.0/go.mod h1:9Wamq8TDfL2680Sav7q3zEhBJSPBrDxJU8WtPJ25dBM= 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= @@ -762,8 +762,6 @@ github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4/go.mod h1:ybxpYRF github.com/andybalholm/brotli v1.0.4/go.mod h1:fO7iG3H7G2nSZ7m0zPUDn85XEX2GTukHGRSepvi9Eig= github.com/andybalholm/brotli v1.1.1 h1:PR2pgnyFznKEugtsUo0xLdDop5SKXd5Qf5ysW+7XdTA= github.com/andybalholm/brotli v1.1.1/go.mod h1:05ib4cKhjx3OQYUY22hTVd34Bc8upXjOLL2rKwwZBoA= -github.com/anicoll/screamer v0.0.0-20241206035431-9ba919b54dfd h1:Bz8Rksav0QRwRNzwdVeZAkMsm0PwNkJ7H4amsYt1xfw= -github.com/anicoll/screamer v0.0.0-20241206035431-9ba919b54dfd/go.mod h1:ZP3Cph589sR4ohmkdCBRVNcaR0YNQXw7E0d3a0YfQjc= github.com/antihax/optional v1.0.0/go.mod h1:uupD/76wgC+ih3iEmQUL+0Ugr19nfwCT1kdvxnR2qWY= github.com/apache/arrow/go/arrow v0.0.0-20200730104253-651201b0f516/go.mod h1:QNYViu/X0HXDHw7m3KXzWSVXIbfUvJqBFe6Gj8/pYA0= github.com/apache/arrow/go/arrow v0.0.0-20211112161151-bc219186db40 h1:q4dksr6ICHXqG5hm0ZW5IHyeEJXoIJSOZeBLmWPNeIQ= @@ -1283,8 +1281,8 @@ 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.13.0 h1:yitjD5f7jQHhyDsnhKEBU52NdvvdSeGzlAnDPT0hH1s= -github.com/googleapis/gax-go/v2 v2.13.0/go.mod h1:Z/fvTZXF8/uw7Xu5GuslPw+bplx6SS338j1Is2S+B7A= +github.com/googleapis/gax-go/v2 v2.14.0 h1:f+jMrjBPl+DL9nI4IQzLUxMq7XrAqFYB7hBPqMNIe8o= +github.com/googleapis/gax-go/v2 v2.14.0/go.mod h1:lhBCnjdLrWRaPvLWhmc8IS24m9mr07qSYnHncrgo+zk= 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/go-type-adapters v1.0.0/go.mod h1:zHW75FOG2aur7gAO2B+MLby+cLsWGBF62rFAi7WjWO4= @@ -1832,8 +1830,8 @@ github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO github.com/stretchr/testify v1.8.1/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= github.com/stretchr/testify v1.8.2/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= github.com/stretchr/testify v1.8.3/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXlSw2iwfAo= -github.com/stretchr/testify v1.9.0 h1:HtqpIVDClZ4nwg75+f6Lvsy/wHu+3BoSGCbBAcpTsTg= -github.com/stretchr/testify v1.9.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY= +github.com/stretchr/testify v1.10.0 h1:Xv5erBjTwe/5IxqUQTdXv5kgmIvbHo3QQyRwhJsOfJA= +github.com/stretchr/testify v1.10.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY= github.com/testcontainers/testcontainers-go v0.34.0 h1:5fbgF0vIN5u+nD3IWabQwRybuB4GY8G2HHgCkbMzMHo= github.com/testcontainers/testcontainers-go v0.34.0/go.mod h1:6P/kMkQe8yqPHfPWNulFGdFHTD8HB2vLq/231xY2iPQ= github.com/testcontainers/testcontainers-go/modules/ollama v0.32.0 h1:nuYlIE4zOGd8m+TzjY0v41kyfYre3inp/iw1p4qn2eU= @@ -2037,8 +2035,8 @@ golang.org/x/crypto v0.5.0/go.mod h1:NK/OQwhpMQP3MwtdjgLlYHnH9ebylxKWv3e0fK+mkQU golang.org/x/crypto v0.6.0/go.mod h1:OFC/31mSvZgRz0V1QTNCzfAI1aIRzbiufJtkMIlEp58= golang.org/x/crypto v0.19.0/go.mod h1:Iy9bg/ha4yyC70EfRS8jz+B6ybOBKMaSxLj6P6oBDfU= golang.org/x/crypto v0.20.0/go.mod h1:Xwo95rrVNIoSMx9wa1JroENMToLWn3RNVrTBpLHgZPQ= -golang.org/x/crypto v0.28.0 h1:GBDwsMXVQi34v5CCYUm2jkJvu4cbtru2U4TN2PSyQnw= -golang.org/x/crypto v0.28.0/go.mod h1:rmgy+3RHxRZMyY0jjAJShp2zgEdOqj2AO7U0pYmeQ7U= +golang.org/x/crypto v0.30.0 h1:RwoQn3GkWiMkzlX562cLB7OxWvjH1L8xutO2WoJcRoY= +golang.org/x/crypto v0.30.0/go.mod h1:kDsLvtWBEx7MV9tJOj9bnXsPbxwJQ6csT/x4KIN4Ssk= golang.org/x/exp v0.0.0-20180321215751-8460e604b9de/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20180807140117-3d87b88a115f/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= @@ -2169,8 +2167,8 @@ golang.org/x/net v0.8.0/go.mod h1:QVkue5JL9kW//ek3r6jTKnTFis1tRmNAW2P1shuFdJc= golang.org/x/net v0.9.0/go.mod h1:d48xBJpPfHeWQsugry2m+kC02ZBRGRgulfHnEXEuWns= golang.org/x/net v0.10.0/go.mod h1:0qNGK6F8kojg2nk9dLZ2mShWaEBan6FAoqfSigmmuDg= golang.org/x/net v0.21.0/go.mod h1:bIjVDfnllIU7BJ2DNgfnXvpSvtn8VRwhlsaeUTyUS44= -golang.org/x/net v0.30.0 h1:AcW1SDZMkb8IpzCdQUaIq2sP4sZ4zw+55h6ynffypl4= -golang.org/x/net v0.30.0/go.mod h1:2wGyMJ5iFasEhkwi13ChkO/t1ECNC4X4eBKkVFyYFlU= +golang.org/x/net v0.32.0 h1:ZqPmj8Kzc+Y6e0+skZsuACbx+wzMgo5MQsJh9Qd6aYI= +golang.org/x/net v0.32.0/go.mod h1:CwU0IoeOlnQQWJ6ioyFrfRuomB8GKF6KbYXZVyeXNfs= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= @@ -2200,8 +2198,8 @@ golang.org/x/oauth2 v0.4.0/go.mod h1:RznEsdpjGAINPTOF0UH/t+xJ75L18YO3Ho6Pyn+uRec golang.org/x/oauth2 v0.5.0/go.mod h1:9/XBHVqLaWO3/BRHs5jbpYCnOZVjj5V0ndyaAM7KB4I= golang.org/x/oauth2 v0.6.0/go.mod h1:ycmewcwgD4Rpr3eZJLSB4Kyyljb3qDh40vJ8STE5HKw= golang.org/x/oauth2 v0.7.0/go.mod h1:hPLQkd9LyjfXTiRohC/41GhcFqxisoUQ99sCUOHO9x4= -golang.org/x/oauth2 v0.23.0 h1:PbgcYx2W7i4LvjJWEbf0ngHV6qJYr86PkAV3bXdLEbs= -golang.org/x/oauth2 v0.23.0/go.mod h1:XYTD2NtWslqkgxebSiOHnXEap4TF09sJSc7H1sXbhtI= +golang.org/x/oauth2 v0.24.0 h1:KTBBxWqUa0ykRPLtV69rRto9TLXcqYkeswu48x/gvNE= +golang.org/x/oauth2 v0.24.0/go.mod h1:XYTD2NtWslqkgxebSiOHnXEap4TF09sJSc7H1sXbhtI= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -2218,8 +2216,8 @@ golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20220819030929-7fc1605a5dde/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20220929204114-8fcdb60fdcc0/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.1.0/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.9.0 h1:fEo0HyrW1GIgZdpbhCRO0PkJajUS5H9IFUztCgEo2jQ= -golang.org/x/sync v0.9.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= +golang.org/x/sync v0.10.0 h1:3NQrjDixjgGwUOCaF8w2+VYHv0Ve/vGYSbdkTa98gmQ= +golang.org/x/sync v0.10.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20181116152217-5ac8a444bdc5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= @@ -2320,8 +2318,8 @@ golang.org/x/sys v0.6.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.7.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.8.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.17.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= -golang.org/x/sys v0.26.0 h1:KHjCJyddX0LoSTb3J+vWpupP9p0oznkqVk/IfjymZbo= -golang.org/x/sys v0.26.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.28.0 h1:Fksou7UEQUWlKvIdsqzJmUmCX3cZuD2+P3XyyzwMhlA= +golang.org/x/sys v0.28.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/term v0.0.0-20201117132131-f5c789dd3221/go.mod h1:Nr5EML6q2oocZ2LXRh80K7BxOlk5/8JxuGnuhpl+muw= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= @@ -2334,8 +2332,8 @@ golang.org/x/term v0.6.0/go.mod h1:m6U89DPEgQRMq3DNkDClhWw02AUbt2daBVO4cn4Hv9U= golang.org/x/term v0.7.0/go.mod h1:P32HKFT3hSsZrRxla30E9HqToFYAQPCMs/zFMBUFqPY= golang.org/x/term v0.8.0/go.mod h1:xPskH00ivmX89bAKVGSKKtLOWNx2+17Eiy94tnKShWo= golang.org/x/term v0.17.0/go.mod h1:lLRBjIVuehSbZlaOtGMbcMncT+aqLLLmKrsjNrUguwk= -golang.org/x/term v0.25.0 h1:WtHI/ltw4NvSUig5KARz9h521QvRC8RmF/cuYqifU24= -golang.org/x/term v0.25.0/go.mod h1:RPyXicDX+6vLxogjjRxjgD2TKtmAO6NZBsBRfrOLu7M= +golang.org/x/term v0.27.0 h1:WP60Sv1nlK1T6SupCHbXzSaN0b9wUmsPoRS9b61A23Q= +golang.org/x/term v0.27.0/go.mod h1:iMsnZpn0cago0GOrHO2+Y7u7JPn5AylBrcoWkElMTSM= golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= @@ -2353,8 +2351,8 @@ golang.org/x/text v0.7.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= golang.org/x/text v0.8.0/go.mod h1:e1OnstbJyHTd6l/uOt8jFFHp6TRDWZR/bV3emEE/zU8= golang.org/x/text v0.9.0/go.mod h1:e1OnstbJyHTd6l/uOt8jFFHp6TRDWZR/bV3emEE/zU8= golang.org/x/text v0.14.0/go.mod h1:18ZOQIKpY8NJVqYksKHtTdi31H5itFRjB5/qKTNYzSU= -golang.org/x/text v0.19.0 h1:kTxAhCbGbxhK0IwgSKiMO5awPoDQ0RpfiVYBfK860YM= -golang.org/x/text v0.19.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= +golang.org/x/text v0.21.0 h1:zyQAAkrwaneQ066sspRyJaG9VNi/YJ1NfzcGB3hZ/qo= +golang.org/x/text v0.21.0/go.mod h1:4IBbMaMmOPCJ8SecivzSH54+73PCFmPWxNTLm+vZkEQ= golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20191024005414-555d28b269f0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= @@ -2362,8 +2360,8 @@ golang.org/x/time v0.0.0-20211116232009-f0f3c7e86c11/go.mod h1:tRJNPiyCQ0inRvYxb golang.org/x/time v0.0.0-20220922220347-f3bd1da661af/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.1.0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.3.0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= -golang.org/x/time v0.7.0 h1:ntUhktv3OPE6TgYxXWv9vKvUSJyIFJlyohwbkEwPrKQ= -golang.org/x/time v0.7.0/go.mod h1:3BpzKBy/shNhVucY/MWOyx10tF3SFh9QdLuxbVysPQM= +golang.org/x/time v0.8.0 h1:9i3RxcPv3PZnitoVGMPDKZSq1xW1gK1Xy3ArNOGZfEg= +golang.org/x/time v0.8.0/go.mod h1:3BpzKBy/shNhVucY/MWOyx10tF3SFh9QdLuxbVysPQM= golang.org/x/tools v0.0.0-20180525024113-a5b4c53f6e8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= @@ -2515,8 +2513,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.205.0 h1:LFaxkAIpDb/GsrWV20dMMo5MR0h8UARTbn24LmD+0Pg= -google.golang.org/api v0.205.0/go.mod h1:NrK1EMqO8Xk6l6QwRAmrXXg2v6dzukhlOyvkYtnvUuc= +google.golang.org/api v0.211.0 h1:IUpLjq09jxBSV1lACO33CGY3jsRcbctfGzhj+ZSE/Bg= +google.golang.org/api v0.211.0/go.mod h1:XOloB4MXFH4UTlQSGuNUxw0UT74qdENK8d6JNsXKLi0= 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= @@ -2658,12 +2656,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-20241113202542-65e8d215514f h1:zDoHYmMzMacIdjNe+P2XiTmPsLawi/pCbSPfxt6lTfw= -google.golang.org/genproto v0.0.0-20241113202542-65e8d215514f/go.mod h1:Q5m6g8b5KaFFzsQFIGdJkSJDGeJiybVenoYFMMa3ohI= -google.golang.org/genproto/googleapis/api v0.0.0-20241104194629-dd2ea8efbc28 h1:M0KvPgPmDZHPlbRbaNU1APr28TvwvvdUPlSv7PUvy8g= -google.golang.org/genproto/googleapis/api v0.0.0-20241104194629-dd2ea8efbc28/go.mod h1:dguCy7UOdZhTvLzDyt15+rOrawrpM4q7DD9dQ1P11P4= -google.golang.org/genproto/googleapis/rpc v0.0.0-20241113202542-65e8d215514f h1:C1QccEa9kUwvMgEUORqQD9S17QesQijxjZ84sO82mfo= -google.golang.org/genproto/googleapis/rpc v0.0.0-20241113202542-65e8d215514f/go.mod h1:GX3210XPVPUjJbTUbvwI8f2IpZDMZuPJWDzDuebbviI= +google.golang.org/genproto v0.0.0-20241118233622-e639e219e697 h1:ToEetK57OidYuqD4Q5w+vfEnPvPpuTwedCNVohYJfNk= +google.golang.org/genproto v0.0.0-20241118233622-e639e219e697/go.mod h1:JJrvXBWRZaFMxBufik1a4RpFw4HhgVtBBWQeQgUj2cc= +google.golang.org/genproto/googleapis/api v0.0.0-20241118233622-e639e219e697 h1:pgr/4QbFyktUv9CtQ/Fq4gzEE6/Xs7iCXbktaGzLHbQ= +google.golang.org/genproto/googleapis/api v0.0.0-20241118233622-e639e219e697/go.mod h1:+D9ySVjN8nY8YCVjc5O7PZDIdZporIDY3KaGfJunh88= +google.golang.org/genproto/googleapis/rpc v0.0.0-20241206012308-a4fef0638583 h1:IfdSdTcLFy4lqUQrQJLkLt1PB+AsqVz6lwkWPzWEz10= +google.golang.org/genproto/googleapis/rpc v0.0.0-20241206012308-a4fef0638583/go.mod h1:5uTbfoYQed2U9p3KIj2/Zzm02PYhndfdmML0qC3q3FU= 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= @@ -2705,8 +2703,8 @@ google.golang.org/grpc v1.52.3/go.mod h1:pu6fVzoFb+NBYNAvQL08ic+lvB2IojljRYuun5v google.golang.org/grpc v1.53.0/go.mod h1:OnIrk0ipVdj4N5d9IUoFUx72/VlD7+jUsHwZgwSMQpw= google.golang.org/grpc v1.54.0/go.mod h1:PUSEXI6iWghWaB6lXM4knEgpJNu2qUcKfDtNci3EC2g= google.golang.org/grpc v1.56.3/go.mod h1:I9bI3vqKfayGqPUAwGdOSu7kt6oIJLixfffKrpXqQ9s= -google.golang.org/grpc v1.68.0 h1:aHQeeJbo8zAkAa3pRzrVjZlbz6uSfeOXlJNQM0RAbz0= -google.golang.org/grpc v1.68.0/go.mod h1:fmSPC5AsjSBCK54MyHRx48kpOti1/jRfOlwEWywNjWA= +google.golang.org/grpc v1.68.1 h1:oI5oTa11+ng8r8XMMN7jAOmWfPZWbYpCFaMUTACxkM0= +google.golang.org/grpc v1.68.1/go.mod h1:+q1XYFJjShcqn0QZHvCyeR4CXPA+llXIeUIfIe00waw= google.golang.org/grpc/cmd/protoc-gen-go-grpc v1.1.0/go.mod h1:6Kw0yEErY5E/yWrBtf03jp27GLLJujG4z/JK95pnjjw= google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= @@ -2725,8 +2723,8 @@ google.golang.org/protobuf v1.28.0/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqw google.golang.org/protobuf v1.28.1/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= google.golang.org/protobuf v1.29.1/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= google.golang.org/protobuf v1.30.0/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= -google.golang.org/protobuf v1.35.1 h1:m3LfL6/Ca+fqnjnlqQXNpFPABW1UD7mjh8KO2mKFytA= -google.golang.org/protobuf v1.35.1/go.mod h1:9fA7Ob0pmnwhb644+1+CVWFRbNajQ6iRojtC/QF5bRE= +google.golang.org/protobuf v1.35.2 h1:8Ar7bF+apOIoThw1EdZl0p1oWvMqTHmpA2fRTyZO8io= +google.golang.org/protobuf v1.35.2/go.mod h1:9fA7Ob0pmnwhb644+1+CVWFRbNajQ6iRojtC/QF5bRE= gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= diff --git a/internal/impl/gcp/input_spanner_change_stream.go b/internal/impl/gcp/input_spanner_cdc.go similarity index 60% rename from internal/impl/gcp/input_spanner_change_stream.go rename to internal/impl/gcp/input_spanner_cdc.go index acaa47c617..de04551883 100644 --- a/internal/impl/gcp/input_spanner_change_stream.go +++ b/internal/impl/gcp/input_spanner_cdc.go @@ -24,66 +24,83 @@ import ( "cloud.google.com/go/spanner" "github.com/Jeffail/shutdown" - "github.com/anicoll/screamer/pkg/model" - "github.com/anicoll/screamer/pkg/partitionstorage" - "github.com/anicoll/screamer/pkg/screamer" "github.com/redpanda-data/benthos/v4/public/service" + + "github.com/redpanda-data/connect/v4/internal/impl/gcp/spannercdc" +) + +const ( + partitionDSN string = "partition_dsn" + streamDSN string = "stream_dsn" + streamID string = "stream_id" + partitionTable string = "partition_table" + startTimeEpoch string = "start_time_epoch" + useInMemPartition string = "use_in_memory_partition" + allowedModTypes string = "allowed_mod_types" ) type streamReader interface { - Stream(ctx context.Context, channel chan<- *model.DataChangeRecord) error + Stream(ctx context.Context, channel chan<- *spannercdc.DataChangeRecord) error Close() error } -func newSpannerChangeStreamInputConfig() *service.ConfigSpec { +func newSpannerCDCInputConfig() *service.ConfigSpec { return service.NewConfigSpec(). Beta(). Version("3.43.0"). Categories("Services", "GCP"). Summary("Creates an input that consumes from a spanner change stream."). - Field(service.NewStringField("stream_dsn").Description("Required field to use to connect to spanner for the change stream.").Example("projects//instances//databases/")). - Field(service.NewStringField("stream_id").Description("Required name of the change stream to track.").Default("")). - Field(service.NewIntField("start_time_epoch").Advanced().Optional().Default(0).Description("Optional microsecond accurate epoch timestamp to start reading from. If empty time.Now() will be used.")). - Field(service.NewStringField("partition_dsn").Optional().Description("Field used to set the DSN for the metadata partition table, can be the same as stream_dsn.").Example("projects//instances//databases/")). - Field(service.NewStringField("partition_table").Optional().Description("Name of the table to create/use in spanner to track change stream partition metadata.")). - Field(service.NewBoolField("use_in_mememory_partition").Description("use an in memory partition table for tracking the partitions.").Default(false)). - Field(service.NewStringListField("allowed_mod_types").Advanced().Description("Mod types to allow through when reading the change stream, default all.").Default([]string{"INSERT", "UPDATE", "DELETE"})) + Field(service.NewStringField(streamDSN).Description("Required field to use to connect to spanner for the change stream.").Example("projects//instances//databases/")). + Field(service.NewStringField(streamID).Description("Required name of the change stream to track.").Default("")). + Field(service.NewIntField(startTimeEpoch).Advanced().Optional().Default(0).Description("Optional microsecond accurate epoch timestamp to start reading from. If empty time.Now() will be used.")). + Field(service.NewStringField(partitionDSN).Optional().Description("Field used to set the DSN for the metadata partition table, can be the same as stream_dsn.").Example("projects//instances//databases/")). + Field(service.NewStringField(partitionTable).Optional().Description("Name of the table to create/use in spanner to track change stream partition metadata.")). + Field(service.NewBoolField(useInMemPartition).Description("use an in memory partition table for tracking the partitions.").Default(false)). + Field(service.NewStringListField(allowedModTypes).Advanced().Description("Mod types to allow through when reading the change stream, default all.").Default([]string{spannercdc.ModTypeINSERT, spannercdc.ModTypeUPDATE, spannercdc.ModTypeDELETE})) } func newSpannerStreamInput(conf *service.ParsedConfig, log *service.Logger) (out *spannerStreamInput, err error) { out = &spannerStreamInput{ // not buffered to prevent the cursor from getting too far ahead. // there is still the chance that we could lose changes though. - changeChannel: make(chan *model.DataChangeRecord, 1), + changeChannel: make(chan *spannercdc.DataChangeRecord, 1), log: log, shutdownSig: shutdown.NewSignaller(), } - if out.partitionDSN, err = conf.FieldString("partition_dsn"); err != nil { + if out.partitionDSN, err = conf.FieldString(partitionDSN); err != nil { return } - if out.partitionTable, err = conf.FieldString("partition_table"); err != nil { + if out.partitionTable, err = conf.FieldString(partitionTable); err != nil { return } - if out.streamDSN, err = conf.FieldString("stream_dsn"); err != nil { + if out.streamDSN, err = conf.FieldString(streamDSN); err != nil { return } - if out.streamID, err = conf.FieldString("stream_id"); err != nil { + if out.streamID, err = conf.FieldString(streamID); err != nil { return } - if out.allowedModTypes, err = conf.FieldStringList("allowed_mod_types"); err != nil { + if out.allowedModTypes, err = conf.FieldStringList(allowedModTypes); err != nil { return } + for _, modType := range out.allowedModTypes { + if !slices.ContainsFunc(spannercdc.AllModTypes, func(s spannercdc.ModType) bool { + return modType == string(s) + }) { + err = errors.New("allowed_mod_types must be one of INSERT, UPDATE, DELETE") + return + } + } - useInMemPartition, err := conf.FieldBool("use_in_mememory_partition") + useInMemPartition, err := conf.FieldBool(useInMemPartition) if err != nil { return } - startTimeEpoch, err := conf.FieldInt("start_time_epoch") + startTimeEpoch, err := conf.FieldInt(startTimeEpoch) if err != nil { return } @@ -96,17 +113,17 @@ func newSpannerStreamInput(conf *service.ParsedConfig, log *service.Logger) (out } if !useInMemPartition && slices.Contains([]string{out.partitionDSN, out.partitionTable, out.streamDSN, out.streamID}, "") { - return nil, errors.New("partition_dsn, partition_table, stream_dsn, and stream_id must be set") + return nil, fmt.Errorf("%s, %s, %s, and %s must be set", partitionDSN, partitionTable, streamDSN, streamID) } else if slices.Contains([]string{out.streamDSN, out.streamID}, "") { - return nil, errors.New("stream_dsn, and stream_id must be set") + return nil, fmt.Errorf("%s, and %s must be set", streamDSN, streamID) } - out.usePartitionTable = !useInMemPartition + out.useInMemPartition = useInMemPartition return } func init() { err := service.RegisterInput( - "gcp_spanner_change_stream", newSpannerChangeStreamInputConfig(), + "gcp_spanner_cdc", newSpannerCDCInputConfig(), func(conf *service.ParsedConfig, mgr *service.Resources) (service.Input, error) { return newSpannerStreamInput(conf, mgr.Logger()) }) @@ -123,15 +140,14 @@ type spannerStreamInput struct { streamID string partitionDSN string partitionTable string - usePartitionTable bool + useInMemPartition bool startTime *time.Time allowedModTypes []string reader streamReader // create a channel to pass from connection to read. - changeChannel chan *model.DataChangeRecord - - log *service.Logger - shutdownSig *shutdown.Signaller + changeChannel chan *spannercdc.DataChangeRecord + log *service.Logger + shutdownSig *shutdown.Signaller } func (i *spannerStreamInput) Connect(ctx context.Context) (err error) { @@ -144,7 +160,7 @@ func (i *spannerStreamInput) Connect(ctx context.Context) (err error) { } } if i.reader == nil { - i.reader, err = newStreamer(jobctx, i.streamClient, i.streamID, i.partitionDSN, i.partitionTable, i.usePartitionTable, i.allowedModTypes, i.startTime) + i.reader, err = newStreamer(jobctx, i.streamClient, i.streamID, i.partitionDSN, i.partitionTable, i.useInMemPartition, i.allowedModTypes, i.startTime) if err != nil { return err } @@ -172,7 +188,6 @@ func (i *spannerStreamInput) Read(ctx context.Context) (*service.Message, servic } func (i *spannerStreamInput) Close(_ context.Context) error { - close(i.changeChannel) if i.reader != nil { return i.reader.Close() } @@ -184,14 +199,14 @@ func (i *spannerStreamInput) Close(_ context.Context) error { type streamerDB struct { streamID, partitionTable string changeStreamClient, partitionClient *db - subscriber *screamer.Subscriber + subscriber *spannercdc.Subscriber allowedModTypes []string } func newStreamer(ctx context.Context, changestreamClient *db, streamID, partitionDSN, partitionTable string, - usePartitionTable bool, + useInMemPartition bool, modTypes []string, startTime *time.Time, ) (streamReader, error) { @@ -202,16 +217,16 @@ func newStreamer(ctx context.Context, changeStreamClient: changestreamClient, } - var pStorage screamer.PartitionStorage = partitionstorage.NewInmemory() + var pStorage spannercdc.PartitionStorage = spannercdc.NewInmemory() // only use DB meta partition table if explicitly enabled. - if usePartitionTable { + if !useInMemPartition { partitionClient, err := newDatabase(ctx, partitionDSN) if err != nil { return nil, err } streamer.partitionClient = partitionClient - spannerPartitionStorage := partitionstorage.NewSpanner(partitionClient.client, partitionTable) + spannerPartitionStorage := spannercdc.NewSpanner(partitionClient.client, partitionTable) if err := spannerPartitionStorage.CreateTableIfNotExists(ctx); err != nil { return nil, err } @@ -219,12 +234,12 @@ func newStreamer(ctx context.Context, pStorage = spannerPartitionStorage } - options := []screamer.Option{} + options := []spannercdc.Option{} // if provided with a specific startime. use that. if startTime != nil { - options = append(options, screamer.WithStartTimestamp(*startTime)) + options = append(options, spannercdc.WithStartTimestamp(*startTime)) } - subscriber := screamer.NewSubscriber(streamer.changeStreamClient.client, streamID, pStorage, options...) + subscriber := spannercdc.NewSubscriber(streamer.changeStreamClient.client, streamID, pStorage, options...) streamer.subscriber = subscriber @@ -233,8 +248,8 @@ func newStreamer(ctx context.Context, // Stream provides a stream of change records from a Spanner database configured stream to your provided channel. // Stream is blocking unless the provided context is cancelled or an error occurs. -func (s *streamerDB) Stream(ctx context.Context, channel chan<- *model.DataChangeRecord) error { - return s.subscriber.SubscribeFunc(ctx, func(dcr *model.DataChangeRecord) error { +func (s *streamerDB) Stream(ctx context.Context, channel chan<- *spannercdc.DataChangeRecord) error { + return s.subscriber.SubscribeFunc(ctx, func(dcr *spannercdc.DataChangeRecord) error { if slices.Contains(s.allowedModTypes, string(dcr.ModType)) { channel <- dcr } diff --git a/internal/impl/gcp/input_spanner_change_stream_mock_test.go b/internal/impl/gcp/input_spanner_cdc_mock_test.go similarity index 89% rename from internal/impl/gcp/input_spanner_change_stream_mock_test.go rename to internal/impl/gcp/input_spanner_cdc_mock_test.go index 6e7beea0a5..97d5766360 100644 --- a/internal/impl/gcp/input_spanner_change_stream_mock_test.go +++ b/internal/impl/gcp/input_spanner_cdc_mock_test.go @@ -17,8 +17,9 @@ package gcp import ( "context" - "github.com/anicoll/screamer/pkg/model" "github.com/stretchr/testify/mock" + + "github.com/redpanda-data/connect/v4/internal/impl/gcp/spannercdc" ) type mockStreamReader struct { @@ -27,7 +28,7 @@ type mockStreamReader struct { var _ streamReader = &mockStreamReader{} -func (mt *mockStreamReader) Stream(ctx context.Context, channel chan<- *model.DataChangeRecord) error { +func (mt *mockStreamReader) Stream(ctx context.Context, channel chan<- *spannercdc.DataChangeRecord) error { args := mt.Called(ctx, channel) return args.Error(0) diff --git a/internal/impl/gcp/input_spanner_change_stream_test.go b/internal/impl/gcp/input_spanner_cdc_test.go similarity index 88% rename from internal/impl/gcp/input_spanner_change_stream_test.go rename to internal/impl/gcp/input_spanner_cdc_test.go index bbb6d4a759..51b6005bf2 100644 --- a/internal/impl/gcp/input_spanner_change_stream_test.go +++ b/internal/impl/gcp/input_spanner_cdc_test.go @@ -21,10 +21,11 @@ import ( "time" "cloud.google.com/go/spanner/spannertest" - "github.com/anicoll/screamer/pkg/model" "github.com/google/uuid" "github.com/stretchr/testify/mock" "github.com/stretchr/testify/require" + + "github.com/redpanda-data/connect/v4/internal/impl/gcp/spannercdc" ) var testSpannerStreamInputYAML = ` @@ -40,7 +41,7 @@ allowed_mod_types: ` func TestGCPSpannerChangeStreamInput_Read(t *testing.T) { - spec := newSpannerChangeStreamInputConfig() + spec := newSpannerCDCInputConfig() parsed, err := spec.ParseYAML(testSpannerStreamInputYAML, nil) require.NoError(t, err) @@ -51,24 +52,24 @@ func TestGCPSpannerChangeStreamInput_Read(t *testing.T) { mockStreamReader := &mockStreamReader{} proc.reader = mockStreamReader - dataChangeRecord := &model.DataChangeRecord{ + dataChangeRecord := &spannercdc.DataChangeRecord{ CommitTimestamp: time.Now(), RecordSequence: "0000001", ServerTransactionID: uuid.NewString(), IsLastRecordInTransactionInPartition: true, TableName: "test_table", - ColumnTypes: []*model.ColumnType{ - {Name: "ID", Type: model.Type{Code: model.TypeCode_INT64}}, - {Name: "Value", Type: model.Type{Code: model.TypeCode_STRING}}, + ColumnTypes: []*spannercdc.ColumnType{ + {Name: "ID", Type: spannercdc.Type{Code: spannercdc.TypeCodeINT64}}, + {Name: "Value", Type: spannercdc.Type{Code: spannercdc.TypeCodeSTRING}}, }, - Mods: []*model.Mod{ + Mods: []*spannercdc.Mod{ { Keys: map[string]interface{}{}, NewValues: map[string]interface{}{}, OldValues: map[string]interface{}{}, }, }, - ModType: model.ModType_INSERT, + ModType: spannercdc.ModTypeINSERT, NumberOfRecordsInTransaction: 1, NumberOfPartitionsInTransaction: 2, } @@ -88,7 +89,7 @@ func TestGCPSpannerChangeStreamInput_Read(t *testing.T) { } func TestGCPSpannerChangeStreamInput_Connect(t *testing.T) { - spec := newSpannerChangeStreamInputConfig() + spec := newSpannerCDCInputConfig() ctx, cancel := context.WithCancel(context.Background()) parsed, err := spec.ParseYAML(testSpannerStreamInputYAML, nil) @@ -118,7 +119,7 @@ func TestGCPSpannerChangeStreamInput_Connect(t *testing.T) { } func TestGCPSpannerChangeStreamInput_Close(t *testing.T) { - spec := newSpannerChangeStreamInputConfig() + spec := newSpannerCDCInputConfig() ctx, cancel := context.WithCancel(context.Background()) parsed, err := spec.ParseYAML(testSpannerStreamInputYAML, nil) diff --git a/internal/impl/gcp/spannercdc/changerecord.go b/internal/impl/gcp/spannercdc/changerecord.go new file mode 100644 index 0000000000..ab4413baed --- /dev/null +++ b/internal/impl/gcp/spannercdc/changerecord.go @@ -0,0 +1,190 @@ +package spannercdc + +import ( + "time" + + "cloud.google.com/go/spanner" +) + +// ChangeRecord is the change record of the table. +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"` +} + +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"` +} + +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"` +} + +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"` +} + +// DataChangeRecord is the change set of the table. +type DataChangeRecord struct { + CommitTimestamp time.Time `json:"commit_timestamp"` + RecordSequence string `json:"record_sequence"` + ServerTransactionID string `json:"server_transaction_id"` + IsLastRecordInTransactionInPartition bool `json:"is_last_record_in_transaction_in_partition"` + TableName string `json:"table_name"` + ColumnTypes []*ColumnType `json:"column_types"` + Mods []*Mod `json:"mods"` + ModType ModType `json:"mod_type"` + ValueCaptureType string `json:"value_capture_type"` + NumberOfRecordsInTransaction int64 `json:"number_of_records_in_transaction"` + NumberOfPartitionsInTransaction int64 `json:"number_of_partitions_in_transaction"` + TransactionTag string `json:"transaction_tag"` + IsSystemTransaction bool `json:"is_system_transaction"` +} + +// ColumnType is the metadata of the column. +type ColumnType struct { + Name string `json:"name"` + Type Type `json:"type"` + IsPrimaryKey bool `json:"is_primary_key,omitempty"` + OrdinalPosition int64 `json:"ordinal_position"` +} + +// Type is the type of the column. +type Type struct { + Code TypeCode `json:"code"` + ArrayElementType TypeCode `json:"array_element_type,omitempty"` +} + +// TypeCode defines the type of the column. +type TypeCode string + +// TypeCode constants. +const ( + TypeCodeNONE TypeCode = "" + TypeCodeBOOL TypeCode = "BOOL" + TypeCodeINT64 TypeCode = "INT64" + TypeCodeFLOAT64 TypeCode = "FLOAT64" + TypeCodeTIMESTAMP TypeCode = "TIMESTAMP" + TypeCodeDATE TypeCode = "DATE" + TypeCodeSTRING TypeCode = "STRING" + TypeCodeBYTES TypeCode = "BYTES" + TypeCodeNUMERIC TypeCode = "NUMERIC" + TypeCodeJSON TypeCode = "JSON" + TypeCodeARRAY TypeCode = "ARRAY" +) + +// Mod contains the keys and the values of the changed records. +type Mod struct { + Keys map[string]interface{} `json:"keys,omitempty"` + NewValues map[string]interface{} `json:"new_values,omitempty"` + OldValues map[string]interface{} `json:"old_values,omitempty"` +} + +// ModType is the type of the modification. +type ModType string + +// ModTypes available for use. +const ( + ModTypeINSERT = "INSERT" + ModTypeUPDATE = "UPDATE" + ModTypeDELETE = "DELETE" +) + +// AllModTypes contains all the available ModTypes. +var AllModTypes = []ModType{ModTypeINSERT, ModTypeUPDATE, ModTypeDELETE} + +// HeartbeatRecord stores the timestamp of the heartbeat. +type HeartbeatRecord struct { + Timestamp time.Time `spanner:"timestamp" json:"timestamp"` +} + +// ChildPartitionsRecord stores the child partitions. +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"` +} + +// ChildPartition stores the child partition. +type ChildPartition struct { + Token string `spanner:"token" json:"token"` + ParentPartitionTokens []string `spanner:"parent_partition_tokens" json:"parent_partition_tokens"` +} + +// DecodeToNonSpannerType decodes the data change record to the non-spanner type. +func (r *dataChangeRecord) DecodeToNonSpannerType() *DataChangeRecord { + columnTypes := []*ColumnType{} + for _, t := range r.ColumnTypes { + columnTypes = append(columnTypes, &ColumnType{ + Name: t.Name, + Type: decodeColumnTypeJSONToType(t.Type), + IsPrimaryKey: t.IsPrimaryKey, + OrdinalPosition: t.OrdinalPosition, + }) + } + + mods := make([]*Mod, 0, len(r.Mods)) + for _, m := range r.Mods { + mods = append(mods, &Mod{ + Keys: decodeNullJSONToMap(m.Keys), + NewValues: decodeNullJSONToMap(m.NewValues), + OldValues: decodeNullJSONToMap(m.OldValues), + }) + } + + return &DataChangeRecord{ + CommitTimestamp: r.CommitTimestamp, + RecordSequence: r.RecordSequence, + ServerTransactionID: r.ServerTransactionID, + IsLastRecordInTransactionInPartition: r.IsLastRecordInTransactionInPartition, + TableName: r.TableName, + ColumnTypes: columnTypes, + Mods: mods, + ModType: ModType(r.ModType), + ValueCaptureType: r.ValueCaptureType, + NumberOfRecordsInTransaction: r.NumberOfRecordsInTransaction, + NumberOfPartitionsInTransaction: r.NumberOfPartitionsInTransaction, + TransactionTag: r.TransactionTag, + IsSystemTransaction: r.IsSystemTransaction, + } +} + +func decodeColumnTypeJSONToType(columnType spanner.NullJSON) Type { + m := columnType.Value.(map[string]interface{}) + code := TypeCode(m["code"].(string)) + + if aet, ok := m["array_element_type"].(map[string]interface{}); ok { + arrayElementType := TypeCode(aet["code"].(string)) + return Type{ + Code: code, + ArrayElementType: arrayElementType, + } + } + + return Type{Code: code} +} + +func decodeNullJSONToMap(j spanner.NullJSON) map[string]interface{} { + if j.IsNull() { + return nil + } + return j.Value.(map[string]interface{}) +} diff --git a/internal/impl/gcp/spannercdc/config.go b/internal/impl/gcp/spannercdc/config.go new file mode 100644 index 0000000000..c9756b0c81 --- /dev/null +++ b/internal/impl/gcp/spannercdc/config.go @@ -0,0 +1,15 @@ +package spannercdc + +import "time" + +// Config spanner CDC struct. +type Config struct { + DSN string + Stream string + MetadataTable *string + Start *time.Time + End *time.Time + HeartbeatInterval *time.Duration + PartitionDSN *string + Priority int32 +} diff --git a/internal/impl/gcp/spannercdc/consumer.go b/internal/impl/gcp/spannercdc/consumer.go new file mode 100644 index 0000000000..cf64767645 --- /dev/null +++ b/internal/impl/gcp/spannercdc/consumer.go @@ -0,0 +1,16 @@ +package spannercdc + +// Consumer is the interface to consume the DataChangeRecord. +// +// Consume might be called from multiple goroutines and must be re-entrant safe. +type Consumer interface { + Consume(change *DataChangeRecord) error +} + +// ConsumerFunc type is an adapter to allow the use of ordinary functions as Consumer. +type ConsumerFunc func(*DataChangeRecord) error + +// Consume calls f(change). +func (f ConsumerFunc) Consume(change *DataChangeRecord) error { + return f(change) +} diff --git a/internal/impl/gcp/spannercdc/helper_test.go b/internal/impl/gcp/spannercdc/helper_test.go new file mode 100644 index 0000000000..5b6f445c2a --- /dev/null +++ b/internal/impl/gcp/spannercdc/helper_test.go @@ -0,0 +1,103 @@ +package spannercdc + +import ( + "context" + "fmt" + "os" + "strconv" + + database "cloud.google.com/go/spanner/admin/database/apiv1" + "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/testcontainers/testcontainers-go" + "github.com/testcontainers/testcontainers-go/wait" +) + +func NewTestContainer(ctx context.Context, image string, envVars map[string]string, ports []string, waitfor wait.Strategy, cmdArgs ...string) (testcontainers.Container, error) { + req := testcontainers.ContainerRequest{ + SkipReaper: true, + Image: image, + Env: envVars, + ExposedPorts: ports, + WaitingFor: waitfor, + Cmd: cmdArgs, + } + + // picks up local test env to clean up containers + if skipReaper := os.Getenv("SKIP_REAPER"); skipReaper != "" { + shouldSkipReaper, err := strconv.ParseBool(skipReaper) + if err != nil { + return nil, err + } + req.SkipReaper = shouldSkipReaper + } + + return testcontainers.GenericContainer(ctx, testcontainers.GenericContainerRequest{ + ContainerRequest: req, + Started: true, + }) +} + +func CreateInstance(ctx context.Context, parentProjectID, instanceID string) (string, error) { + instanceAdminClient, err := instance.NewInstanceAdminClient(ctx) + if err != nil { + return "", err + } + defer instanceAdminClient.Close() + + op, err := instanceAdminClient.CreateInstance(ctx, &instancepb.CreateInstanceRequest{ + Parent: "projects/" + parentProjectID, + InstanceId: instanceID, + Instance: &instancepb.Instance{ + Config: "projects/model/instanceConfigs/regional-us-central1", + DisplayName: instanceID, + ProcessingUnits: 100, + }, + }) + if err != nil { + return "", err + } + + resp, err := op.Wait(ctx) + if err != nil { + return "", err + } + + return resp.Name, nil +} + +func DeleteInstance(ctx context.Context, instanceName string) error { + instanceAdminClient, err := instance.NewInstanceAdminClient(ctx) + if err != nil { + return err + } + defer instanceAdminClient.Close() + + return instanceAdminClient.DeleteInstance(ctx, &instancepb.DeleteInstanceRequest{ + Name: instanceName, + }) +} + +func CreateDatabase(ctx context.Context, parentInstanceName, databaseID string) (string, error) { + databaseAdminClient, err := database.NewDatabaseAdminClient(ctx) + if err != nil { + return "", err + } + defer databaseAdminClient.Close() + + op, err := databaseAdminClient.CreateDatabase(ctx, &databasepb.CreateDatabaseRequest{ + Parent: parentInstanceName, + CreateStatement: fmt.Sprintf("CREATE DATABASE `%s`", databaseID), + }) + if err != nil { + return "", err + } + + resp, err := op.Wait(ctx) + if err != nil { + return "", err + } + + return resp.Name, nil +} diff --git a/internal/impl/gcp/spannercdc/inmemory.go b/internal/impl/gcp/spannercdc/inmemory.go new file mode 100644 index 0000000000..7b9a3fe76d --- /dev/null +++ b/internal/impl/gcp/spannercdc/inmemory.go @@ -0,0 +1,159 @@ +package spannercdc + +import ( + "context" + "sort" + "sync" + "time" +) + +// InmemoryPartitionStorage implements PartitionStorage that stores PartitionMetadata in memory. +type InmemoryPartitionStorage struct { + mu sync.Mutex + m map[string]*PartitionMetadata +} + +// NewInmemory creates new instance of InmemoryPartitionStorage +func NewInmemory() *InmemoryPartitionStorage { + return &InmemoryPartitionStorage{ + m: make(map[string]*PartitionMetadata), + } +} + +// GetUnfinishedMinWatermarkPartition returns the partition with the smallest watermark that is not finished. +func (s *InmemoryPartitionStorage) GetUnfinishedMinWatermarkPartition(ctx context.Context) (*PartitionMetadata, error) { + s.mu.Lock() + defer s.mu.Unlock() + + partitions := []*PartitionMetadata{} + for _, p := range s.m { + if p.State != StateFinished { + partitions = append(partitions, p) + } + } + + if len(partitions) == 0 { + return nil, nil + } + + sort.Slice(partitions, func(i, j int) bool { return partitions[i].Watermark.Before(partitions[j].Watermark) }) + return partitions[0], nil +} + +// GetInterruptedPartitions unimplemented for memory store. +func (s *InmemoryPartitionStorage) GetInterruptedPartitions(ctx context.Context) ([]*PartitionMetadata, error) { + // InmemoryPartitionStorage can't return any partitions + return nil, nil +} + +// InitializeRootPartition initializes the root partition. +func (s *InmemoryPartitionStorage) InitializeRootPartition(ctx context.Context, startTimestamp time.Time, endTimestamp time.Time, heartbeatInterval time.Duration) error { + s.mu.Lock() + defer s.mu.Unlock() + + p := &PartitionMetadata{ + PartitionToken: RootPartitionToken, + ParentTokens: []string{}, + StartTimestamp: startTimestamp, + EndTimestamp: endTimestamp, + HeartbeatMillis: heartbeatInterval.Milliseconds(), + State: StateCreated, + Watermark: startTimestamp, + CreatedAt: time.Now(), + } + s.m[p.PartitionToken] = p + + return nil +} + +// GetSchedulablePartitions returns all partitions that are created and have a start timestamp before minWatermark. +func (s *InmemoryPartitionStorage) GetSchedulablePartitions(ctx context.Context, minWatermark time.Time) ([]*PartitionMetadata, error) { + s.mu.Lock() + defer s.mu.Unlock() + + partitions := []*PartitionMetadata{} + for _, p := range s.m { + if p.State == StateCreated && !minWatermark.After(p.StartTimestamp) { + partitions = append(partitions, p) + } + } + + return partitions, nil +} + +// AddChildPartitions adds child partitions to the parent partition. +func (s *InmemoryPartitionStorage) AddChildPartitions(ctx context.Context, parent *PartitionMetadata, r *ChildPartitionsRecord) error { + s.mu.Lock() + defer s.mu.Unlock() + + for _, v := range r.ChildPartitions { + p := &PartitionMetadata{ + PartitionToken: v.Token, + ParentTokens: v.ParentPartitionTokens, + StartTimestamp: r.StartTimestamp, + EndTimestamp: parent.EndTimestamp, + HeartbeatMillis: parent.HeartbeatMillis, + State: StateCreated, + Watermark: r.StartTimestamp, + } + s.m[p.PartitionToken] = p + } + + return nil +} + +// UpdateToScheduled updates the partitions to scheduled state. +func (s *InmemoryPartitionStorage) UpdateToScheduled(ctx context.Context, partitions []*PartitionMetadata) error { + s.mu.Lock() + defer s.mu.Unlock() + + now := time.Now() + for _, p := range partitions { + p = s.m[p.PartitionToken] + p.ScheduledAt = &now + p.State = StateScheduled + } + + return nil +} + +// UpdateToRunning updates the partition to a running state. +func (s *InmemoryPartitionStorage) UpdateToRunning(ctx context.Context, partition *PartitionMetadata) error { + s.mu.Lock() + defer s.mu.Unlock() + + now := time.Now() + + p := s.m[partition.PartitionToken] + p.RunningAt = &now + p.State = StateRunning + + return nil +} + +// UpdateToFinished updates the partition with a finished timestamp. +func (s *InmemoryPartitionStorage) UpdateToFinished(ctx context.Context, partition *PartitionMetadata) error { + s.mu.Lock() + defer s.mu.Unlock() + + now := time.Now() + + p := s.m[partition.PartitionToken] + p.FinishedAt = &now + p.State = StateFinished + + return nil +} + +// UpdateWatermark updates the watermark of the partition. +func (s *InmemoryPartitionStorage) UpdateWatermark(ctx context.Context, partition *PartitionMetadata, watermark time.Time) error { + s.mu.Lock() + defer s.mu.Unlock() + + s.m[partition.PartitionToken].Watermark = watermark + + return nil +} + +// Assert that InmemoryPartitionStorage implements PartitionStorage. +var _ PartitionStorage = (*InmemoryPartitionStorage)(nil) diff --git a/internal/impl/gcp/spannercdc/partition.go b/internal/impl/gcp/spannercdc/partition.go new file mode 100644 index 0000000000..a3122ccfa6 --- /dev/null +++ b/internal/impl/gcp/spannercdc/partition.go @@ -0,0 +1,39 @@ +package spannercdc + +import "time" + +// PartitionMetadata is the metadata of the partition. +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"` +} + +// State defines the state of the partition. +type State string + +// States available for usage. +const ( + StateCreated State = "CREATED" + StateScheduled State = "SCHEDULED" + StateRunning State = "RUNNING" + StateFinished State = "FINISHED" +) + +// the RootPartitionToken is the token for the root partition. +const ( + RootPartitionToken = "Parent0" +) + +// IsRootPartition returns true if this is root partition. +func (p *PartitionMetadata) IsRootPartition() bool { + return p.PartitionToken == RootPartitionToken +} diff --git a/internal/impl/gcp/spannercdc/reader.go b/internal/impl/gcp/spannercdc/reader.go new file mode 100644 index 0000000000..0de5774ee9 --- /dev/null +++ b/internal/impl/gcp/spannercdc/reader.go @@ -0,0 +1,328 @@ +package spannercdc + +import ( + "context" + "errors" + "fmt" + "sync" + "time" + + "cloud.google.com/go/spanner" + "cloud.google.com/go/spanner/apiv1/spannerpb" + "golang.org/x/sync/errgroup" +) + +// PartitionStorage is a storage interface for partition metadata. +type PartitionStorage interface { + GetUnfinishedMinWatermarkPartition(ctx context.Context) (*PartitionMetadata, error) + GetInterruptedPartitions(ctx context.Context) ([]*PartitionMetadata, error) + InitializeRootPartition(ctx context.Context, startTimestamp time.Time, endTimestamp time.Time, heartbeatInterval time.Duration) error + GetSchedulablePartitions(ctx context.Context, minWatermark time.Time) ([]*PartitionMetadata, error) + AddChildPartitions(ctx context.Context, parentPartition *PartitionMetadata, childPartitionsRecord *ChildPartitionsRecord) error + UpdateToScheduled(ctx context.Context, partitions []*PartitionMetadata) error + UpdateToRunning(ctx context.Context, partition *PartitionMetadata) error + UpdateToFinished(ctx context.Context, partition *PartitionMetadata) error + UpdateWatermark(ctx context.Context, partition *PartitionMetadata, watermark time.Time) error +} + +// Subscriber subscribes change stream. +type Subscriber struct { + spannerClient *spanner.Client + streamName string + startTimestamp time.Time + endTimestamp time.Time + heartbeatInterval time.Duration + spannerRequestPriority spannerpb.RequestOptions_Priority + partitionStorage PartitionStorage + consumer Consumer + eg *errgroup.Group + mu sync.Mutex +} + +type config struct { + startTimestamp time.Time + endTimestamp time.Time + heartbeatInterval time.Duration + spannerRequestPriority spannerpb.RequestOptions_Priority +} + +// Option interface for subscriber. +type Option interface { + Apply(*config) +} + +type withStartTimestamp time.Time + +func (o withStartTimestamp) Apply(c *config) { + c.startTimestamp = time.Time(o) +} + +// WithStartTimestamp set the start timestamp option for read change streams. +// +// The value must be within the retention period of the change stream and before the current time. +// Default value is current timestamp. +func WithStartTimestamp(startTimestamp time.Time) Option { + return withStartTimestamp(startTimestamp) +} + +type withEndTimestamp time.Time + +func (o withEndTimestamp) Apply(c *config) { + c.endTimestamp = time.Time(o) +} + +// WithEndTimestamp set the end timestamp option for read change streams. +// +// The value must be within the retention period of the change stream and must be after the start timestamp. +// If not set, read latest changes until canceled. +func WithEndTimestamp(endTimestamp time.Time) Option { + return withEndTimestamp(endTimestamp) +} + +type withHeartbeatInterval time.Duration + +func (o withHeartbeatInterval) Apply(c *config) { + c.heartbeatInterval = time.Duration(o) +} + +// WithHeartbeatInterval set the heartbeat interval for read change streams. +// +// Default value is 10 seconds. +func WithHeartbeatInterval(heartbeatInterval time.Duration) Option { + return withHeartbeatInterval(heartbeatInterval) +} + +type withSpannerRequestPriotiry spannerpb.RequestOptions_Priority + +func (o withSpannerRequestPriotiry) Apply(c *config) { + c.spannerRequestPriority = spannerpb.RequestOptions_Priority(o) +} + +// WithSpannerRequestPriotiry set the request priority option for read change streams. +// +// Default value is unspecified, equivalent to high. +func WithSpannerRequestPriotiry(priority spannerpb.RequestOptions_Priority) Option { + return withSpannerRequestPriotiry(priority) +} + +var ( + defaultEndTimestamp = time.Date(9999, 12, 31, 23, 59, 59, 999999999, time.UTC) // Maximum value of Spanner TIMESTAMP type. + defaultHeartbeatInterval = 10 * time.Second + + nowFunc = time.Now +) + +// NewSubscriber creates a new subscriber of change streams. +func NewSubscriber( + client *spanner.Client, + streamName string, + partitionStorage PartitionStorage, + options ...Option, +) *Subscriber { + c := &config{ + startTimestamp: nowFunc(), + endTimestamp: defaultEndTimestamp, + heartbeatInterval: defaultHeartbeatInterval, + } + for _, o := range options { + o.Apply(c) + } + + return &Subscriber{ + spannerClient: client, + streamName: streamName, + startTimestamp: c.startTimestamp, + endTimestamp: c.endTimestamp, + heartbeatInterval: c.heartbeatInterval, + spannerRequestPriority: c.spannerRequestPriority, + partitionStorage: partitionStorage, + } +} + +// Subscribe starts subscribing to the change stream. +func (s *Subscriber) Subscribe(ctx context.Context, consumer Consumer) error { + eg, ctx := s.initErrGroup(ctx) + s.consumer = consumer + + // Initialize root partition if this is the first run or if the previous run has already been completed. + minWatermarkPartition, err := s.partitionStorage.GetUnfinishedMinWatermarkPartition(ctx) + if err != nil { + return fmt.Errorf("failed to get unfinished min watermark partition on start subscribe: %w", err) + } + if minWatermarkPartition == nil { + if err := s.partitionStorage.InitializeRootPartition(ctx, s.startTimestamp, s.endTimestamp, s.heartbeatInterval); err != nil { + return fmt.Errorf("failed to initialize root partition: %w", err) + } + } + + interruptedPartitions, err := s.partitionStorage.GetInterruptedPartitions(ctx) + if err != nil { + return fmt.Errorf("failed to get interrupted partitions: %w", err) + } + for _, p := range interruptedPartitions { + p := p + s.eg.Go(func() error { + return s.queryChangeStream(ctx, p) + }) + } + + eg.Go(func() error { + ticker := time.NewTicker(time.Second) + defer ticker.Stop() + for { + select { + case <-ticker.C: + err := s.detectNewPartitions(ctx) + switch err { + case errDone: + return nil + case nil: + // continue + default: + return err + } + case <-ctx.Done(): + return ctx.Err() + } + } + }) + + return eg.Wait() +} + +// SubscribeFunc is an adapter to allow the use of ordinary functions as Consumer. +// +// function might be called from multiple goroutines and must be re-entrant safe. +func (s *Subscriber) SubscribeFunc(ctx context.Context, f ConsumerFunc) error { + return s.Subscribe(ctx, f) +} + +func (s *Subscriber) initErrGroup(ctx context.Context) (*errgroup.Group, context.Context) { + s.mu.Lock() + defer s.mu.Unlock() + + if s.eg != nil { + panic("Subscriber has already started subscribe.") + } + + eg, ctx := errgroup.WithContext(ctx) + s.eg = eg + return eg, ctx +} + +var errDone = errors.New("all partitions have been processed") + +func (s *Subscriber) detectNewPartitions(ctx context.Context) error { + minWatermarkPartition, err := s.partitionStorage.GetUnfinishedMinWatermarkPartition(ctx) + if err != nil { + return fmt.Errorf("failed to get unfinished min watarmark partition: %w", err) + } + + if minWatermarkPartition == nil { + return errDone + } + + // To make sure changes for a key is processed in timestamp order, wait until the records returned from all parents have been processed. + partitions, err := s.partitionStorage.GetSchedulablePartitions(ctx, minWatermarkPartition.Watermark) + if err != nil { + return fmt.Errorf("failed to get schedulable partitions: %w", err) + } + if len(partitions) == 0 { + return nil + } + + if err := s.partitionStorage.UpdateToScheduled(ctx, partitions); err != nil { + return fmt.Errorf("failed to update to scheduled: %w", err) + } + + for _, p := range partitions { + p := p + s.eg.Go(func() error { + return s.queryChangeStream(ctx, p) + }) + } + + return nil +} + +func (s *Subscriber) queryChangeStream(ctx context.Context, p *PartitionMetadata) error { + if err := s.partitionStorage.UpdateToRunning(ctx, p); err != nil { + return fmt.Errorf("failed to update to running: %w", err) + } + + stmt := spanner.Statement{ + SQL: fmt.Sprintf("SELECT ChangeRecord FROM READ_%s (@startTimestamp, @endTimestamp, @partitionToken, @heartbeatMilliseconds)", s.streamName), + Params: map[string]interface{}{ + "startTimestamp": p.Watermark, + "endTimestamp": p.EndTimestamp, + "partitionToken": p.PartitionToken, + "heartbeatMilliseconds": p.HeartbeatMillis, + }, + } + + if p.IsRootPartition() { + // Must be converted to NULL (root partition). + stmt.Params["partitionToken"] = nil + } + + iter := s.spannerClient.Single().QueryWithOptions(ctx, stmt, spanner.QueryOptions{Priority: s.spannerRequestPriority}) + if err := iter.Do(func(r *spanner.Row) error { + records := []*ChangeRecord{} + if err := r.Columns(&records); err != nil { + return err + } + if err := s.handle(ctx, p, records); err != nil { + return err + } + return nil + }); err != nil { + return err + } + + if err := s.partitionStorage.UpdateToFinished(ctx, p); err != nil { + return fmt.Errorf("failed to update to finished: %w", err) + } + + return nil +} + +type watermarker struct { + watermark time.Time +} + +func (w *watermarker) set(t time.Time) { + if t.After(w.watermark) { + w.watermark = t + } +} + +func (w *watermarker) get() time.Time { + return w.watermark +} + +func (s *Subscriber) handle(ctx context.Context, p *PartitionMetadata, records []*ChangeRecord) error { + var watermarker watermarker + for _, cr := range records { + for _, record := range cr.DataChangeRecords { + if err := s.consumer.Consume(record.DecodeToNonSpannerType()); err != nil { + return err + } + watermarker.set(record.CommitTimestamp) + } + for _, record := range cr.HeartbeatRecords { + watermarker.set(record.Timestamp) + } + for _, record := range cr.ChildPartitionsRecords { + if err := s.partitionStorage.AddChildPartitions(ctx, p, record); err != nil { + return fmt.Errorf("failed to add child partitions: %w", err) + } + watermarker.set(record.StartTimestamp) + } + } + + if err := s.partitionStorage.UpdateWatermark(ctx, p, watermarker.get()); err != nil { + return fmt.Errorf("failed to update watermark: %w", err) + } + + return nil +} diff --git a/internal/impl/gcp/spannercdc/spanner.go b/internal/impl/gcp/spannercdc/spanner.go new file mode 100644 index 0000000000..a607eee6f6 --- /dev/null +++ b/internal/impl/gcp/spannercdc/spanner.go @@ -0,0 +1,303 @@ +package spannercdc + +import ( + "context" + "fmt" + "time" + + "cloud.google.com/go/spanner" + database "cloud.google.com/go/spanner/admin/database/apiv1" + "cloud.google.com/go/spanner/admin/database/apiv1/databasepb" + "cloud.google.com/go/spanner/apiv1/spannerpb" + "google.golang.org/api/iterator" + "google.golang.org/grpc/codes" +) + +// SpannerPartitionStorage implements PartitionStorage that stores PartitionMetadata in Cloud Spanner. +type SpannerPartitionStorage struct { + client *spanner.Client + tableName string + requestPriority spannerpb.RequestOptions_Priority +} + +type spannerConfig struct { + requestPriority spannerpb.RequestOptions_Priority +} + +type spannerOption interface { + Apply(*spannerConfig) +} + +type withRequestPriotiry spannerpb.RequestOptions_Priority + +func (o withRequestPriotiry) Apply(c *spannerConfig) { + c.requestPriority = spannerpb.RequestOptions_Priority(o) +} + +// WithRequestPriotiry set the priority option for spanner requests. +// Default value is unspecified, equivalent to high. +func WithRequestPriotiry(priority spannerpb.RequestOptions_Priority) spannerOption { + return withRequestPriotiry(priority) +} + +// NewSpanner creates new instance of SpannerPartitionStorage +func NewSpanner(client *spanner.Client, tableName string, options ...spannerOption) *SpannerPartitionStorage { + c := &spannerConfig{} + for _, o := range options { + o.Apply(c) + } + + return &SpannerPartitionStorage{ + client: client, + tableName: tableName, + requestPriority: c.requestPriority, + } +} + +const ( + columnPartitionToken = "PartitionToken" + columnParentTokens = "ParentTokens" + columnStartTimestamp = "StartTimestamp" + columnEndTimestamp = "EndTimestamp" + columnHeartbeatMillis = "HeartbeatMillis" + columnState = "State" + columnWatermark = "Watermark" + columnCreatedAt = "CreatedAt" + columnScheduledAt = "ScheduledAt" + columnRunningAt = "RunningAt" + columnFinishedAt = "FinishedAt" +) + +// CreateTableIfNotExists creates the metadata table if it does not already exist in the DB. +func (s *SpannerPartitionStorage) CreateTableIfNotExists(ctx context.Context) error { + databaseAdminClient, err := database.NewDatabaseAdminClient(ctx) + if err != nil { + return err + } + defer databaseAdminClient.Close() + + stmt := fmt.Sprintf(`CREATE TABLE IF NOT EXISTS %[1]s ( + %[2]s STRING(MAX) NOT NULL, + %[3]s ARRAY NOT NULL, + %[4]s TIMESTAMP NOT NULL, + %[5]s TIMESTAMP NOT NULL, + %[6]s INT64 NOT NULL, + %[7]s STRING(MAX) NOT NULL, + %[8]s TIMESTAMP NOT NULL, + %[9]s TIMESTAMP NOT NULL OPTIONS (allow_commit_timestamp=true), + %[10]s TIMESTAMP OPTIONS (allow_commit_timestamp=true), + %[11]s TIMESTAMP OPTIONS (allow_commit_timestamp=true), + %[12]s TIMESTAMP OPTIONS (allow_commit_timestamp=true), +) PRIMARY KEY (%[2]s), ROW DELETION POLICY (OLDER_THAN(%[12]s, INTERVAL 1 DAY))`, + s.tableName, + columnPartitionToken, + columnParentTokens, + columnStartTimestamp, + columnEndTimestamp, + columnHeartbeatMillis, + columnState, + columnWatermark, + columnCreatedAt, + columnScheduledAt, + columnRunningAt, + columnFinishedAt, + ) + + req := &databasepb.UpdateDatabaseDdlRequest{ + Database: s.client.DatabaseName(), + Statements: []string{stmt}, + } + op, err := databaseAdminClient.UpdateDatabaseDdl(ctx, req) + if err != nil { + return err + } + + if err := op.Wait(ctx); err != nil { + return err + } + + return nil +} + +// GetUnfinishedMinWatermarkPartition returns the partition with the smallest watermark that is not finished. +func (s *SpannerPartitionStorage) GetUnfinishedMinWatermarkPartition(ctx context.Context) (*PartitionMetadata, error) { + stmt := spanner.Statement{ + SQL: fmt.Sprintf("SELECT * FROM %s WHERE State != @state ORDER BY Watermark ASC LIMIT 1", s.tableName), + Params: map[string]interface{}{ + "state": StateFinished, + }, + } + + iter := s.client.Single().QueryWithOptions(ctx, stmt, spanner.QueryOptions{Priority: s.requestPriority}) + defer iter.Stop() + + r, err := iter.Next() + switch err { + case iterator.Done: + return nil, nil + case nil: + // break + default: + return nil, err + } + + partition := new(PartitionMetadata) + if err := r.ToStruct(partition); err != nil { + return nil, err + } + + return partition, nil +} + +// GetInterruptedPartitions returns the partitions that are scheduled or running. +func (s *SpannerPartitionStorage) GetInterruptedPartitions(ctx context.Context) ([]*PartitionMetadata, error) { + stmt := spanner.Statement{ + SQL: fmt.Sprintf("SELECT * FROM %s WHERE State IN UNNEST(@states) ORDER BY Watermark ASC", s.tableName), + Params: map[string]interface{}{ + "states": []State{StateScheduled, StateRunning}, + }, + } + + iter := s.client.Single().QueryWithOptions(ctx, stmt, spanner.QueryOptions{Priority: s.requestPriority}) + + partitions := []*PartitionMetadata{} + if err := iter.Do(func(r *spanner.Row) error { + p := new(PartitionMetadata) + if err := r.ToStruct(p); err != nil { + return err + } + partitions = append(partitions, p) + return nil + }); err != nil { + return nil, err + } + + return partitions, nil +} + +// InitializeRootPartition initializes the root partition. +func (s *SpannerPartitionStorage) InitializeRootPartition(ctx context.Context, startTimestamp time.Time, endTimestamp time.Time, heartbeatInterval time.Duration) error { + m := spanner.InsertOrUpdateMap(s.tableName, map[string]interface{}{ + columnPartitionToken: RootPartitionToken, + columnParentTokens: []string{}, + columnStartTimestamp: startTimestamp, + columnEndTimestamp: endTimestamp, + columnHeartbeatMillis: heartbeatInterval.Milliseconds(), + columnState: StateCreated, + columnWatermark: startTimestamp, + columnCreatedAt: spanner.CommitTimestamp, + columnScheduledAt: nil, + columnRunningAt: nil, + columnFinishedAt: nil, + }) + + _, err := s.client.Apply(ctx, []*spanner.Mutation{m}, spanner.Priority(s.requestPriority)) + return err +} + +// GetSchedulablePartitions returns all partitions that are created and have a start timestamp before or up to the minWatermark. +func (s *SpannerPartitionStorage) GetSchedulablePartitions(ctx context.Context, minWatermark time.Time) ([]*PartitionMetadata, error) { + stmt := spanner.Statement{ + SQL: fmt.Sprintf("SELECT * FROM %s WHERE State = @state AND StartTimestamp >= @minWatermark ORDER BY StartTimestamp ASC", s.tableName), + Params: map[string]interface{}{ + "state": StateCreated, + "minWatermark": minWatermark, + }, + } + + iter := s.client.Single().QueryWithOptions(ctx, stmt, spanner.QueryOptions{Priority: s.requestPriority}) + + partitions := []*PartitionMetadata{} + if err := iter.Do(func(r *spanner.Row) error { + p := new(PartitionMetadata) + if err := r.ToStruct(p); err != nil { + return err + } + partitions = append(partitions, p) + return nil + }); err != nil { + return nil, err + } + + return partitions, nil +} + +// AddChildPartitions adds child partitions to the parent partition. +func (s *SpannerPartitionStorage) AddChildPartitions(ctx context.Context, parent *PartitionMetadata, r *ChildPartitionsRecord) error { + for _, p := range r.ChildPartitions { + m := spanner.InsertMap(s.tableName, map[string]interface{}{ + columnPartitionToken: p.Token, + columnParentTokens: p.ParentPartitionTokens, + columnStartTimestamp: r.StartTimestamp, + columnEndTimestamp: parent.EndTimestamp, + columnHeartbeatMillis: parent.HeartbeatMillis, + columnState: StateCreated, + columnWatermark: r.StartTimestamp, + columnCreatedAt: spanner.CommitTimestamp, + }) + + if _, err := s.client.Apply(ctx, []*spanner.Mutation{m}, spanner.Priority(s.requestPriority)); err != nil { + // Ignore the AlreadyExists error because a child partition can be found multiple times if partitions are merged. + if spanner.ErrCode(err) == codes.AlreadyExists { + continue + } + return err + } + } + + return nil +} + +// UpdateToScheduled updates the partitions to be scheduled. +func (s *SpannerPartitionStorage) UpdateToScheduled(ctx context.Context, partitions []*PartitionMetadata) error { + mutations := make([]*spanner.Mutation, 0, len(partitions)) + for _, p := range partitions { + m := spanner.UpdateMap(s.tableName, map[string]interface{}{ + columnPartitionToken: p.PartitionToken, + columnState: StateScheduled, + columnScheduledAt: spanner.CommitTimestamp, + }) + mutations = append(mutations, m) + } + + _, err := s.client.Apply(ctx, mutations, spanner.Priority(s.requestPriority)) + return err +} + +// UpdateToRunning updates the partition to a running state. +func (s *SpannerPartitionStorage) UpdateToRunning(ctx context.Context, partition *PartitionMetadata) error { + m := spanner.UpdateMap(s.tableName, map[string]interface{}{ + columnPartitionToken: partition.PartitionToken, + columnState: StateRunning, + columnRunningAt: spanner.CommitTimestamp, + }) + + _, err := s.client.Apply(ctx, []*spanner.Mutation{m}, spanner.Priority(s.requestPriority)) + return err +} + +// UpdateToFinished updates the partition to a finished state. +func (s *SpannerPartitionStorage) UpdateToFinished(ctx context.Context, partition *PartitionMetadata) error { + m := spanner.UpdateMap(s.tableName, map[string]interface{}{ + columnPartitionToken: partition.PartitionToken, + columnState: StateFinished, + columnFinishedAt: spanner.CommitTimestamp, + }) + + _, err := s.client.Apply(ctx, []*spanner.Mutation{m}, spanner.Priority(s.requestPriority)) + return err +} + +// UpdateWatermark updates the watermark of the partition. +func (s *SpannerPartitionStorage) UpdateWatermark(ctx context.Context, partition *PartitionMetadata, watermark time.Time) error { + m := spanner.UpdateMap(s.tableName, map[string]interface{}{ + columnPartitionToken: partition.PartitionToken, + columnWatermark: watermark, + }) + + _, err := s.client.Apply(ctx, []*spanner.Mutation{m}, spanner.Priority(s.requestPriority)) + return err +} + +// Assert that SpannerPartitionStorage implements PartitionStorage. +var _ PartitionStorage = (*SpannerPartitionStorage)(nil) diff --git a/internal/impl/gcp/spannercdc/spanner_test.go b/internal/impl/gcp/spannercdc/spanner_test.go new file mode 100644 index 0000000000..7fd03a36fe --- /dev/null +++ b/internal/impl/gcp/spannercdc/spanner_test.go @@ -0,0 +1,515 @@ +package spannercdc + +import ( + "context" + "fmt" + "os" + "reflect" + "testing" + "time" + + "cloud.google.com/go/spanner" + database "cloud.google.com/go/spanner/admin/database/apiv1" + "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/stretchr/testify/suite" + "github.com/testcontainers/testcontainers-go" + "github.com/testcontainers/testcontainers-go/wait" + "google.golang.org/api/iterator" +) + +const ( + testTableName = "PartitionMetadata" + projectID = "local-project" + instanceID = "local-instance" + databaseID = "local-database" +) + +type SpannerTestSuite struct { + suite.Suite + ctx context.Context + container testcontainers.Container + client *spanner.Client + timeout time.Duration + dsn string +} + +func TestSpannerTestSuite(t *testing.T) { + suite.Run(t, new(SpannerTestSuite)) +} + +func (s *SpannerTestSuite) SetupSuite() { + image := "gcr.io/cloud-spanner-emulator/emulator" + ports := []string{"9010/tcp"} + s.ctx = context.Background() + s.timeout = time.Second * 1500 + s.dsn = fmt.Sprintf("projects/%s/instances/%s/databases/%s", projectID, instanceID, databaseID) + + envVars := make(map[string]string) + var err error + s.container, err = NewTestContainer(s.ctx, image, envVars, ports, wait.ForLog("gRPC server listening at")) + s.NoError(err) + + mappedPort, err := s.container.MappedPort(s.ctx, "9010") + s.NoError(err) + hostIP, err := s.container.Host(s.ctx) + s.NoError(err) + hostPort := fmt.Sprintf("%s:%s", hostIP, mappedPort.Port()) + + os.Setenv("SPANNER_EMULATOR_HOST", hostPort) + + s.createInstance() // create instance + s.createDatabase() // create database +} + +func (s *SpannerTestSuite) TearDownSuite() { + if s.container != nil { + err := s.container.Terminate(s.ctx) + s.NoError(err) + } +} + +func (s *SpannerTestSuite) AfterTest(suiteName, testName string) { + if s.client != nil { + s.client.Close() + } +} + +func (s *SpannerTestSuite) createInstance() { + instanceAdminClient, err := instance.NewInstanceAdminClient(s.ctx) + s.NoError(err) + defer instanceAdminClient.Close() + + op, err := instanceAdminClient.CreateInstance(s.ctx, &instancepb.CreateInstanceRequest{ + Parent: "projects/" + projectID, + InstanceId: instanceID, + Instance: &instancepb.Instance{ + Config: "emulator-config", + DisplayName: instanceID, + NodeCount: 1, + }, + }) + s.NoError(err) + + _, err = op.Wait(s.ctx) + s.NoError(err) +} + +func (s *SpannerTestSuite) createDatabase() { + databaseAdminClient, err := database.NewDatabaseAdminClient(s.ctx) + s.NoError(err) + defer databaseAdminClient.Close() + + op, err := databaseAdminClient.CreateDatabase(s.ctx, &databasepb.CreateDatabaseRequest{ + Parent: "projects/" + projectID + "/instances/" + instanceID, + CreateStatement: fmt.Sprintf("CREATE DATABASE `%s`", databaseID), + }) + s.NoError(err) + _, err = op.Wait(s.ctx) + s.NoError(err) +} + +func (s *SpannerTestSuite) TestSpannerPartitionStorage_CreateTableIfNotExists() { + ctx := context.Background() + var err error + s.client, err = spanner.NewClient(ctx, s.dsn) + s.NoError(err) + + storage := &SpannerPartitionStorage{ + client: s.client, + tableName: "CreateTableIfNotExists", + } + + err = storage.CreateTableIfNotExists(ctx) + s.NoError(err) + + iter := s.client.Single().Read(ctx, storage.tableName, spanner.AllKeys(), []string{columnPartitionToken}) + defer iter.Stop() + + if _, err := iter.Next(); err != iterator.Done { + s.T().Errorf("Read from %s after SpannerPartitionStorage.CreateTableIfNotExists() = %v, want %v", storage.tableName, err, iterator.Done) + } + + existsTable, err := existsTable(ctx, s.client, storage.tableName) + s.NoError(err) + if !existsTable { + s.T().Errorf("SpannerPartitionStorage.existsTable() = %v, want %v", existsTable, false) + } +} + +func existsTable(ctx context.Context, client *spanner.Client, tableName string) (bool, error) { + iter := client.Single().Query(ctx, spanner.Statement{ + SQL: "SELECT 1 FROM information_schema.tables WHERE table_catalog = '' AND table_schema = '' AND table_name = @tableName", + Params: map[string]interface{}{ + "tableName": tableName, + }, + }) + defer iter.Stop() + + if _, err := iter.Next(); err != nil { + if err == iterator.Done { + return false, nil + } + return false, err + } + + return true, nil +} + +func (s *SpannerTestSuite) setupSpannerPartitionStorage(ctx context.Context, tableName string) *SpannerPartitionStorage { + var err error + s.client, err = spanner.NewClient(ctx, s.dsn) + s.NoError(err) + + storage := &SpannerPartitionStorage{ + client: s.client, + tableName: tableName, + } + + err = storage.CreateTableIfNotExists(ctx) + s.NoError(err) + + return storage +} + +func (s *SpannerTestSuite) TestSpannerPartitionStorage_InitializeRootPartition() { + ctx := context.Background() + storage := s.setupSpannerPartitionStorage(ctx, "InitializeRootPartition") + + tests := map[string]struct { + startTimestamp time.Time + endTimestamp time.Time + heartbeatInterval time.Duration + want PartitionMetadata + }{ + "one": { + startTimestamp: time.Date(1970, 1, 1, 0, 0, 0, 0, time.UTC), + endTimestamp: time.Date(9999, 12, 31, 23, 59, 59, 999999999, time.UTC), + heartbeatInterval: 10 * time.Second, + want: PartitionMetadata{ + PartitionToken: RootPartitionToken, + ParentTokens: []string{}, + StartTimestamp: time.Date(1970, 1, 1, 0, 0, 0, 0, time.UTC), + EndTimestamp: time.Date(9999, 12, 31, 23, 59, 59, 999999999, time.UTC), + HeartbeatMillis: 10000, + State: StateCreated, + Watermark: time.Date(1970, 1, 1, 0, 0, 0, 0, time.UTC), + }, + }, + "two": { + startTimestamp: time.Date(2023, 12, 31, 23, 59, 59, 999999999, time.UTC), + endTimestamp: time.Date(2024, 1, 1, 0, 0, 0, 0, time.UTC), + heartbeatInterval: time.Hour, + want: PartitionMetadata{ + PartitionToken: RootPartitionToken, + ParentTokens: []string{}, + StartTimestamp: time.Date(2023, 12, 31, 23, 59, 59, 999999999, time.UTC), + EndTimestamp: time.Date(2024, 1, 1, 0, 0, 0, 0, time.UTC), + HeartbeatMillis: 3600000, + State: StateCreated, + Watermark: time.Date(2023, 12, 31, 23, 59, 59, 999999999, time.UTC), + }, + }, + } + for name, test := range tests { + s.Run(name, func() { + if err := storage.InitializeRootPartition(ctx, test.startTimestamp, test.endTimestamp, test.heartbeatInterval); err != nil { + s.T().Errorf("InitializeRootPartition(%q, %q, %q): %v", test.startTimestamp, test.endTimestamp, test.heartbeatInterval, err) + return + } + + columns := []string{columnPartitionToken, columnParentTokens, columnStartTimestamp, columnEndTimestamp, columnHeartbeatMillis, columnState, columnWatermark} + row, err := storage.client.Single().ReadRow(ctx, storage.tableName, spanner.Key{RootPartitionToken}, columns) + if err != nil { + s.T().Errorf("InitializeRootPartition(%q, %q, %q): %v", test.startTimestamp, test.endTimestamp, test.heartbeatInterval, err) + return + } + + got := PartitionMetadata{} + if err := row.ToStruct(&got); err != nil { + s.T().Errorf("InitializeRootPartition(%q, %q, %q): %v", test.startTimestamp, test.endTimestamp, test.heartbeatInterval, err) + return + } + if !reflect.DeepEqual(got, test.want) { + s.T().Errorf("InitializeRootPartition(%q, %q, %q): got = %+v, want %+v", test.startTimestamp, test.endTimestamp, test.heartbeatInterval, got, test.want) + } + }) + } +} + +func (s *SpannerTestSuite) TestSpannerPartitionStorage_Read() { + ctx := context.Background() + storage := s.setupSpannerPartitionStorage(ctx, "Read") + + timestamp := time.Date(2023, 1, 1, 0, 0, 0, 0, time.UTC) + + insert := func(token string, start time.Time, state State) *spanner.Mutation { + return spanner.InsertMap(storage.tableName, map[string]interface{}{ + columnPartitionToken: token, + columnParentTokens: []string{}, + columnStartTimestamp: start, + columnEndTimestamp: time.Time{}, + columnHeartbeatMillis: 0, + columnState: state, + columnWatermark: start, + columnCreatedAt: spanner.CommitTimestamp, + }) + } + + _, err := storage.client.Apply(ctx, []*spanner.Mutation{ + insert("created1", timestamp, StateCreated), + insert("created2", timestamp.Add(-2*time.Second), StateCreated), + insert("scheduled", timestamp.Add(time.Second), StateScheduled), + insert("running", timestamp.Add(2*time.Second), StateRunning), + insert("finished", timestamp.Add(-time.Second), StateFinished), + }) + s.NoError(err) + + s.Run("GetUnfinishedMinWatermarkPartition", func() { + got, err := storage.GetUnfinishedMinWatermarkPartition(ctx) + s.NoError(err) + + want := "created2" + if got.PartitionToken != want { + s.T().Errorf("GetUnfinishedMinWatermarkPartition(ctx) = %v, want = %v", got.PartitionToken, want) + } + }) + + s.Run("GetInterruptedPartitions", func() { + partitions, err := storage.GetInterruptedPartitions(ctx) + s.NoError(err) + got := []string{} + for _, p := range partitions { + got = append(got, p.PartitionToken) + } + + want := []string{"scheduled", "running"} + if !reflect.DeepEqual(got, want) { + s.T().Errorf("GetInterruptedPartitions(ctx) = %+v, want = %+v", got, want) + } + }) + + s.Run("GetSchedulablePartitions", func() { + partitions, err := storage.GetSchedulablePartitions(ctx, timestamp) + s.NoError(err) + + got := []string{} + for _, p := range partitions { + got = append(got, p.PartitionToken) + } + + want := []string{"created1"} + if !reflect.DeepEqual(got, want) { + s.T().Errorf("GetSchedulablePartitions(ctx, %q) = %+v, want = %+v", timestamp, got, want) + } + }) +} + +func (s *SpannerTestSuite) TestSpannerPartitionStorage_AddChildPartitions() { + ctx := context.Background() + storage := s.setupSpannerPartitionStorage(ctx, "AddChildPartitions") + + childStartTimestamp := time.Date(2023, 1, 1, 0, 0, 0, 0, time.UTC) + endTimestamp := time.Date(9999, 12, 31, 23, 59, 59, 999999999, time.UTC) + var heartbeatMillis int64 = 10000 + + parent := &PartitionMetadata{ + PartitionToken: "parent1", + ParentTokens: []string{}, + StartTimestamp: time.Time{}, + EndTimestamp: endTimestamp, + HeartbeatMillis: heartbeatMillis, + State: StateRunning, + Watermark: time.Time{}, + } + record := &ChildPartitionsRecord{ + StartTimestamp: childStartTimestamp, + ChildPartitions: []*ChildPartition{ + {Token: "token1", ParentPartitionTokens: []string{"parent1"}}, + {Token: "token2", ParentPartitionTokens: []string{"parent1"}}, + }, + } + err := storage.AddChildPartitions(ctx, parent, record) + s.NoError(err) + + columns := []string{columnPartitionToken, columnParentTokens, columnStartTimestamp, columnEndTimestamp, columnHeartbeatMillis, columnState, columnWatermark} + + got := []PartitionMetadata{} + err = storage.client.Single().Read(ctx, storage.tableName, spanner.AllKeys(), columns).Do(func(r *spanner.Row) error { + p := PartitionMetadata{} + if err := r.ToStruct(&p); err != nil { + return err + } + got = append(got, p) + return nil + }) + s.NoError(err) + + want := []PartitionMetadata{ + { + PartitionToken: "token1", + ParentTokens: []string{"parent1"}, + StartTimestamp: childStartTimestamp, + EndTimestamp: endTimestamp, + HeartbeatMillis: heartbeatMillis, + State: StateCreated, + Watermark: childStartTimestamp, + }, + { + PartitionToken: "token2", + ParentTokens: []string{"parent1"}, + StartTimestamp: childStartTimestamp, + EndTimestamp: endTimestamp, + HeartbeatMillis: heartbeatMillis, + State: StateCreated, + Watermark: childStartTimestamp, + }, + } + if !reflect.DeepEqual(got, want) { + s.T().Errorf("GetSchedulablePartitions(ctx, %+v, %+v): got = %+v, want %+v", parent, record, got, want) + } +} + +func (s *SpannerTestSuite) TestSpannerPartitionStorage_Update() { + ctx := context.Background() + storage := s.setupSpannerPartitionStorage(ctx, "Update") + + create := func(token string) *PartitionMetadata { + return &PartitionMetadata{ + PartitionToken: token, + ParentTokens: []string{}, + StartTimestamp: time.Time{}, + EndTimestamp: time.Time{}, + HeartbeatMillis: 0, + State: StateCreated, + Watermark: time.Time{}, + } + } + + insert := func(p *PartitionMetadata) *spanner.Mutation { + return spanner.InsertMap(storage.tableName, map[string]interface{}{ + columnPartitionToken: p.PartitionToken, + columnParentTokens: p.ParentTokens, + columnStartTimestamp: p.StartTimestamp, + columnEndTimestamp: p.EndTimestamp, + columnHeartbeatMillis: p.HeartbeatMillis, + columnState: p.State, + columnWatermark: p.Watermark, + columnCreatedAt: spanner.CommitTimestamp, + }) + } + + partitions := []*PartitionMetadata{create("token1"), create("token2")} + + _, err := storage.client.Apply(ctx, []*spanner.Mutation{ + insert(partitions[0]), + insert(partitions[1]), + }) + s.NoError(err) + + s.Run("UpdateToScheduled", func() { + err := storage.UpdateToScheduled(ctx, partitions) + s.NoError(err) + + columns := []string{columnPartitionToken, columnState} + + type partition struct { + PartitionToken string `spanner:"PartitionToken"` + State State `spanner:"State"` + } + got := []partition{} + + err = storage.client.Single().Read(ctx, storage.tableName, spanner.AllKeys(), columns).Do(func(r *spanner.Row) error { + p := partition{} + if err := r.ToStruct(&p); err != nil { + return err + } + got = append(got, p) + return nil + }) + s.NoError(err) + + want := []partition{ + {PartitionToken: "token1", State: StateScheduled}, + {PartitionToken: "token2", State: StateScheduled}, + } + if !reflect.DeepEqual(got, want) { + s.T().Errorf("UpdateToScheduled(ctx, %+v): got = %+v, want %+v", partitions, got, want) + } + }) + + s.Run("UpdateToRunning", func() { + err := storage.UpdateToRunning(ctx, partitions[0]) + s.NoError(err) + + columns := []string{columnPartitionToken, columnState} + + type partition struct { + PartitionToken string `spanner:"PartitionToken"` + State State `spanner:"State"` + } + + r, err := storage.client.Single().ReadRow(ctx, storage.tableName, spanner.Key{"token1"}, columns) + s.NoError(err) + + got := partition{} + err = r.ToStruct(&got) + s.NoError(err) + + want := partition{PartitionToken: "token1", State: StateRunning} + if !reflect.DeepEqual(got, want) { + s.T().Errorf("UpdateToRunning(ctx, %+v): got = %+v, want %+v", partitions[0], got, want) + } + }) + + s.Run("UpdateToFinished", func() { + err := storage.UpdateToFinished(ctx, partitions[0]) + s.NoError(err) + + columns := []string{columnPartitionToken, columnState} + + type partition struct { + PartitionToken string `spanner:"PartitionToken"` + State State `spanner:"State"` + } + + r, err := storage.client.Single().ReadRow(ctx, storage.tableName, spanner.Key{"token1"}, columns) + s.NoError(err) + + got := partition{} + err = r.ToStruct(&got) + s.NoError(err) + + want := partition{PartitionToken: "token1", State: StateFinished} + if !reflect.DeepEqual(got, want) { + s.T().Errorf("UpdateToFinished(ctx, %+v): got = %+v, want %+v", partitions[0], got, want) + } + }) + + s.Run("UpdateWatermark", func() { + timestamp := time.Date(2023, 1, 1, 0, 0, 0, 0, time.UTC) + + err := storage.UpdateWatermark(ctx, partitions[0], timestamp) + s.NoError(err) + + columns := []string{columnPartitionToken, columnWatermark} + + type partition struct { + PartitionToken string `spanner:"PartitionToken"` + Watermark time.Time `spanner:"Watermark"` + } + + r, err := storage.client.Single().ReadRow(ctx, storage.tableName, spanner.Key{"token1"}, columns) + s.NoError(err) + + got := partition{} + err = r.ToStruct(&got) + s.NoError(err) + + want := partition{PartitionToken: "token1", Watermark: timestamp} + if !reflect.DeepEqual(got, want) { + s.T().Errorf("UpdateWatermark(ctx, %+v, %q): got = %+v, want %+v", partitions[0], timestamp, got, want) + } + }) +} From 6ad066507e7acc90ca25f5a0d222416e3dbecd41 Mon Sep 17 00:00:00 2001 From: Andrew Nicoll Date: Mon, 16 Dec 2024 13:03:52 +1030 Subject: [PATCH 5/7] fix: test naming --- internal/impl/gcp/input_spanner_cdc_test.go | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/internal/impl/gcp/input_spanner_cdc_test.go b/internal/impl/gcp/input_spanner_cdc_test.go index 51b6005bf2..0fda1d8925 100644 --- a/internal/impl/gcp/input_spanner_cdc_test.go +++ b/internal/impl/gcp/input_spanner_cdc_test.go @@ -31,13 +31,11 @@ import ( var testSpannerStreamInputYAML = ` stream_dsn: "projects/test-project/instances/test-instance/databases/test-db" stream_id: "OutboxStream" -use_in_mememory_partition: true +use_in_memory_partition: true partition_dsn: "projects/test/instances/test/databases/test-events-md" # optional default "" partition_table: "meta_partitions_table" # optional default "" allowed_mod_types: - "INSERT" - - "UPDATE" - - "DELETE" ` func TestGCPSpannerChangeStreamInput_Read(t *testing.T) { From 7b330dcf42c81c3ad1d70097600319fe79738cb6 Mon Sep 17 00:00:00 2001 From: Andrew Nicoll Date: Mon, 16 Dec 2024 14:24:30 +1030 Subject: [PATCH 6/7] with auto retry nacks --- internal/impl/gcp/input_spanner_cdc.go | 61 ++++++++++++++++---------- 1 file changed, 38 insertions(+), 23 deletions(-) diff --git a/internal/impl/gcp/input_spanner_cdc.go b/internal/impl/gcp/input_spanner_cdc.go index de04551883..43660a7590 100644 --- a/internal/impl/gcp/input_spanner_cdc.go +++ b/internal/impl/gcp/input_spanner_cdc.go @@ -65,44 +65,44 @@ func newSpannerStreamInput(conf *service.ParsedConfig, log *service.Logger) (out // there is still the chance that we could lose changes though. changeChannel: make(chan *spannercdc.DataChangeRecord, 1), log: log, - shutdownSig: shutdown.NewSignaller(), + stopSig: shutdown.NewSignaller(), } if out.partitionDSN, err = conf.FieldString(partitionDSN); err != nil { - return + return nil, err } if out.partitionTable, err = conf.FieldString(partitionTable); err != nil { - return + return nil, err } if out.streamDSN, err = conf.FieldString(streamDSN); err != nil { - return + return nil, err } if out.streamID, err = conf.FieldString(streamID); err != nil { - return + return nil, err } if out.allowedModTypes, err = conf.FieldStringList(allowedModTypes); err != nil { - return + return nil, err } for _, modType := range out.allowedModTypes { if !slices.ContainsFunc(spannercdc.AllModTypes, func(s spannercdc.ModType) bool { return modType == string(s) }) { err = errors.New("allowed_mod_types must be one of INSERT, UPDATE, DELETE") - return + return nil, err } } useInMemPartition, err := conf.FieldBool(useInMemPartition) if err != nil { - return + return nil, err } startTimeEpoch, err := conf.FieldInt(startTimeEpoch) if err != nil { - return + return nil, err } if startTimeEpoch > 0 { @@ -118,14 +118,18 @@ func newSpannerStreamInput(conf *service.ParsedConfig, log *service.Logger) (out return nil, fmt.Errorf("%s, and %s must be set", streamDSN, streamID) } out.useInMemPartition = useInMemPartition + return } func init() { err := service.RegisterInput( - "gcp_spanner_cdc", newSpannerCDCInputConfig(), - func(conf *service.ParsedConfig, mgr *service.Resources) (service.Input, error) { - return newSpannerStreamInput(conf, mgr.Logger()) + "gcp_spanner_cdc", newSpannerCDCInputConfig(), func(conf *service.ParsedConfig, mgr *service.Resources) (service.Input, error) { + streamInput, err := newSpannerStreamInput(conf, mgr.Logger()) + if err != nil { + return nil, err + } + return conf.WrapInputExtractTracingSpanMapping("gcp_spanner_cdc", service.AutoRetryNacks(streamInput)) }) if err != nil { panic(err) @@ -147,11 +151,12 @@ type spannerStreamInput struct { // create a channel to pass from connection to read. changeChannel chan *spannercdc.DataChangeRecord log *service.Logger - shutdownSig *shutdown.Signaller + stopSig *shutdown.Signaller } func (i *spannerStreamInput) Connect(ctx context.Context) (err error) { - jobctx, _ := i.shutdownSig.SoftStopCtx(context.Background()) + i.stopSig = shutdown.NewSignaller() + jobctx, _ := i.stopSig.SoftStopCtx(context.Background()) if i.streamClient == nil { i.streamClient, err = newDatabase(jobctx, i.streamDSN) @@ -169,22 +174,32 @@ func (i *spannerStreamInput) Connect(ctx context.Context) (err error) { if rerr := i.reader.Stream(jobctx, i.changeChannel); rerr != nil { i.log.Errorf("Subscription error: %v\n", rerr) close(i.changeChannel) - panic(rerr) } }() return nil } func (i *spannerStreamInput) Read(ctx context.Context) (*service.Message, service.AckFunc, error) { - msg := <-i.changeChannel - data, err := json.Marshal(msg) - if err != nil { - return nil, nil, err + select { + case msg := <-i.changeChannel: + data, err := json.Marshal(msg) + if err != nil { + return nil, nil, err + } + messageOut := service.NewMessage(data) + messageOut.MetaSet("tabe", msg.TableName) + messageOut.MetaSet("transaction_tag", msg.TransactionTag) + messageOut.MetaSet("mod_type", string(msg.ModType)) + messageOut.MetaSet("commit_timestamp", msg.CommitTimestamp.Format(time.RFC3339Nano)) + return messageOut, func(ctx context.Context, err error) error { + // Nacks are retried automatically when we use service.AutoRetryNacks + return nil + }, nil + case <-i.stopSig.HasStoppedChan(): + return nil, nil, service.ErrNotConnected + case <-ctx.Done(): + return nil, nil, ctx.Err() } - return service.NewMessage(data), func(ctx context.Context, err error) error { - // Nacks are retried automatically when we use service.AutoRetryNacks - return nil - }, nil } func (i *spannerStreamInput) Close(_ context.Context) error { From 26e4817679e3da886d579f63234b6fefd954f0bd Mon Sep 17 00:00:00 2001 From: Andrew Nicoll Date: Mon, 16 Dec 2024 14:43:58 +1030 Subject: [PATCH 7/7] updated docs --- ...anner_change_stream.adoc => gcp_spanner_cdc.adoc} | 12 ++++++------ internal/plugins/info.csv | 2 +- 2 files changed, 7 insertions(+), 7 deletions(-) rename docs/modules/components/pages/inputs/{gcp_spanner_change_stream.adoc => gcp_spanner_cdc.adoc} (92%) diff --git a/docs/modules/components/pages/inputs/gcp_spanner_change_stream.adoc b/docs/modules/components/pages/inputs/gcp_spanner_cdc.adoc similarity index 92% rename from docs/modules/components/pages/inputs/gcp_spanner_change_stream.adoc rename to docs/modules/components/pages/inputs/gcp_spanner_cdc.adoc index 43a69aa036..1bec9451fc 100644 --- a/docs/modules/components/pages/inputs/gcp_spanner_change_stream.adoc +++ b/docs/modules/components/pages/inputs/gcp_spanner_cdc.adoc @@ -1,4 +1,4 @@ -= gcp_spanner_change_stream += gcp_spanner_cdc :type: input :status: beta :categories: ["Services","GCP"] @@ -38,12 +38,12 @@ Common:: # Common config fields, showing default values input: label: "" - gcp_spanner_change_stream: + gcp_spanner_cdc: stream_dsn: projects//instances//databases/ # No default (required) stream_id: "" partition_dsn: projects//instances//databases/ # No default (optional) partition_table: "" # No default (optional) - use_in_mememory_partition: false + use_in_memory_partition: false ``` -- @@ -55,13 +55,13 @@ Advanced:: # All config fields, showing default values input: label: "" - gcp_spanner_change_stream: + gcp_spanner_cdc: stream_dsn: projects//instances//databases/ # No default (required) stream_id: "" start_time_epoch: 0 partition_dsn: projects//instances//databases/ # No default (optional) partition_table: "" # No default (optional) - use_in_mememory_partition: false + use_in_memory_partition: false allowed_mod_types: - INSERT - UPDATE @@ -127,7 +127,7 @@ Name of the table to create/use in spanner to track change stream partition meta *Type*: `string` -=== `use_in_mememory_partition` +=== `use_in_memory_partition` use an in memory partition table for tracking the partitions. diff --git a/internal/plugins/info.csv b/internal/plugins/info.csv index c57c787274..022678b509 100644 --- a/internal/plugins/info.csv +++ b/internal/plugins/info.csv @@ -84,7 +84,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_change_stream ,input ,gcp_spanner_change_stream ,0.0.0 ,community ,n ,n ,n +gcp_spanner_cdc ,input ,gcp_spanner_cdc ,0.0.0 ,community ,n ,n ,n 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