From 76ef36600dc9aa53e1ecfb8f5f95af8f049b03f3 Mon Sep 17 00:00:00 2001 From: Vladyslav Len Date: Wed, 6 Nov 2024 23:30:35 +0100 Subject: [PATCH 01/45] feat(mysql_cdc): working on mysql_cdc plugin --- go.mod | 1 + go.sum | 3 + internal/impl/mysql/event.go | 13 + internal/impl/mysql/input_mysql_stream.go | 314 ++++++++++++++++++++++ internal/impl/mysql/integration_test.go | 18 ++ internal/impl/mysql/message.go | 16 ++ 6 files changed, 365 insertions(+) create mode 100644 internal/impl/mysql/event.go create mode 100644 internal/impl/mysql/input_mysql_stream.go create mode 100644 internal/impl/mysql/integration_test.go create mode 100644 internal/impl/mysql/message.go diff --git a/go.mod b/go.mod index 3e704d1ae1..99de6b470e 100644 --- a/go.mod +++ b/go.mod @@ -170,6 +170,7 @@ require ( github.com/containerd/platforms v0.2.1 // indirect github.com/envoyproxy/go-control-plane v0.13.0 // indirect github.com/envoyproxy/protoc-gen-validate v1.1.0 // indirect + github.com/go-mysql-org/go-mysql v1.9.1 // indirect github.com/hamba/avro/v2 v2.22.2-0.20240625062549-66aad10411d9 // indirect github.com/json-iterator/go v1.1.12 // indirect github.com/jzelinskie/stringz v0.0.3 // indirect diff --git a/go.sum b/go.sum index 68035027bc..38fd8ec560 100644 --- a/go.sum +++ b/go.sum @@ -1114,6 +1114,8 @@ github.com/go-logr/logr v1.4.2 h1:6pFjapn8bFcIbiKo3XT4j/BhANplGihG6tvd+8rYgrY= github.com/go-logr/logr v1.4.2/go.mod h1:9T104GzyrTigFIr8wt5mBrctHMim0Nb2HLGrmQ40KvY= github.com/go-logr/stdr v1.2.2 h1:hSWxHoqTgW2S2qGc0LTAI563KZ5YKYRhT3MFKZMbjag= github.com/go-logr/stdr v1.2.2/go.mod h1:mMo/vtBO5dYbehREoey6XUKy/eSumjCCveDpRre4VKE= +github.com/go-mysql-org/go-mysql v1.9.1 h1:W2ZKkHkoM4mmkasJCoSYfaE4RQNxXTb6VqiaMpKFrJc= +github.com/go-mysql-org/go-mysql v1.9.1/go.mod h1:+SgFgTlqjqOQoMc98n9oyUWEgn2KkOL1VmXDoq2ONOs= github.com/go-ole/go-ole v1.2.6/go.mod h1:pprOEPIfldk/42T2oK7lQ4v4JSDwmV0As9GaiUsvbm0= github.com/go-ole/go-ole v1.3.0 h1:Dt6ye7+vXGIKZ7Xtk4s6/xVdGDQynvom7xCFEdWr6uE= github.com/go-ole/go-ole v1.3.0/go.mod h1:5LS6F96DhAwUc7C+1HLexzMXY1xGRSryjyPPKW6zv78= @@ -1668,6 +1670,7 @@ github.com/pinecone-io/go-pinecone v1.0.0 h1:90euw+0EKSgdeE9q7iGSTVmdx9r9+x3mxWk github.com/pinecone-io/go-pinecone v1.0.0/go.mod h1:KfJhn4yThX293+fbtrZLnxe2PJYo8557Py062W4FYKk= github.com/pingcap/errors v0.11.4 h1:lFuQV/oaUMGcD2tqt+01ROSmJs75VG1ToEOkZIZ4nE4= github.com/pingcap/errors v0.11.4/go.mod h1:Oi8TUi2kEtXXLMJk9l1cGmz20kV3TaQ0usTwv5KuLY8= +github.com/pingcap/errors v0.11.5-0.20221009092201-b66cddb77c32 h1:m5ZsBa5o/0CkzZXfXLaThzKuR85SnHHetqBCpzQ30h8= github.com/pkg/browser v0.0.0-20180916011732-0a3d74bf9ce4/go.mod h1:4OwLy04Bl9Ef3GJJCoec+30X3LQs/0/m4HFRt/2LUSA= github.com/pkg/browser v0.0.0-20240102092130-5ac0b6a4141c h1:+mdjkGKdHQG3305AYmdv1U2eRNDiU2ErMBj1gwrq8eQ= github.com/pkg/browser v0.0.0-20240102092130-5ac0b6a4141c/go.mod h1:7rwL4CYBLnjLxUqIJNnCWiEdr3bn6IUYi15bNlnbCCU= diff --git a/internal/impl/mysql/event.go b/internal/impl/mysql/event.go new file mode 100644 index 0000000000..04919951ee --- /dev/null +++ b/internal/impl/mysql/event.go @@ -0,0 +1,13 @@ +// Copyright 2024 Redpanda Data, Inc. +// +// Licensed as a Redpanda Enterprise file under the Redpanda Community +// License (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// https://github.com/redpanda-data/connect/v4/blob/main/licenses/rcl.md + +package mysql + +type ProcessEventParams struct { + initValue, incrementValue int +} diff --git a/internal/impl/mysql/input_mysql_stream.go b/internal/impl/mysql/input_mysql_stream.go new file mode 100644 index 0000000000..5e92bf9360 --- /dev/null +++ b/internal/impl/mysql/input_mysql_stream.go @@ -0,0 +1,314 @@ +// Copyright 2024 Redpanda Data, Inc. +// +// Licensed as a Redpanda Enterprise file under the Redpanda Community +// License (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// https://github.com/redpanda-data/connect/v4/blob/main/licenses/rcl.md + +package mysql + +import ( + "context" + "encoding/json" + "errors" + "fmt" + "github.com/Jeffail/checkpoint" + "github.com/go-mysql-org/go-mysql/canal" + mysqlReplications "github.com/go-mysql-org/go-mysql/mysql" + "github.com/go-mysql-org/go-mysql/replication" + "github.com/go-sql-driver/mysql" + "github.com/redpanda-data/benthos/v4/public/service" + "sync" +) + +const ( + fieldMySQLDSN = "dsn" + fieldMySQLTables = "tables" + fieldStreamSnapshot = "stream_snapshot" + fieldBatching = "batching" + fieldCheckpointKey = "checkpoint_key" + fieldCheckpointLimit = "checkpoint_limit" + fieldFlavor = "flavor" +) + +var mysqlStreamConfigSpec = service.NewConfigSpec(). + Summary("Enables MySQL streaming for RedPanda Connect."). + Fields( + service.NewStringField(fieldMySQLDSN). + Description("The DSN of the MySQL database to connect to."). + Example("user:password@tcp(localhost:3306)/database"), + service.NewStringListField(fieldMySQLTables). + Description("A list of tables to stream from the database."). + Example([]string{"table1", "table2"}), + service.NewStringField(fieldCheckpointKey). + Description("The key to store the last processed binlog position."), + service.NewStringField(fieldFlavor). + Description("The flavor of MySQL to connect to."). + Example("mysql"), + service.NewBoolField(fieldStreamSnapshot). + Description("If set to true, the connector will perform a backup to get a snapshot of the database. Otherwise, it will start from the current binlog position."), + service.NewAutoRetryNacksToggleField(), + service.NewIntField(fieldCheckpointLimit). + Description("The maximum number of messages that can be processed at a given time. Increasing this limit enables parallel processing and batching at the output level. Any given LSN will not be acknowledged unless all messages under that offset are delivered in order to preserve at least once delivery guarantees."). + Default(1024), + service.NewBatchPolicyField(fieldBatching), + ) + +type asyncMessage struct { + msg service.MessageBatch + ackFn service.AckFunc +} + +type mysqlStreamInput struct { + // canal represents mysql binlog listener connection + canal *canal.Canal + mysqlConfig *mysql.Config + canal.DummyEventHandler + startBinLogPosition *mysqlReplications.Position + + dsn string + tables []string + flavor string + streamSnapshot bool + + cMut sync.Mutex + msgChan chan asyncMessage + batching service.BatchPolicy + + logger *service.Logger +} + +func newMySQLStreamInput(conf *service.ParsedConfig, mgr *service.Resources) (s service.BatchInput, err error) { + streamInput := mysqlStreamInput{ + logger: mgr.Logger(), + } + + if streamInput.dsn, err = conf.FieldString(fieldMySQLDSN); err != nil { + return nil, err + } + + streamInput.mysqlConfig, err = mysql.ParseDSN(streamInput.dsn) + if err != nil { + return nil, fmt.Errorf("error parsing mysql DSN: %v", err) + } + + if streamInput.tables, err = conf.FieldStringList(fieldMySQLTables); err != nil { + return nil, err + } + + if streamInput.flavor, err = conf.FieldString(fieldFlavor); err != nil { + return nil, err + } + + if streamInput.streamSnapshot, err = conf.FieldBool(fieldStreamSnapshot); err != nil { + return nil, err + } + + if binLogCacheKey, err := conf.FieldString(fieldCheckpointKey); err != nil { + return nil, err + } else { + if err := mgr.AccessCache(context.Background(), binLogCacheKey, func(c service.Cache) { + binLogPositionBytes, cErr := c.Get(context.Background(), binLogCacheKey) + if err != nil { + if !errors.Is(cErr, service.ErrKeyNotFound) { + mgr.Logger().With("error", cErr.Error()).Error("Failed to obtain cursor cache item.") + } + return + } + + var storedMySQLBinLogPosition mysqlReplications.Position + if err = json.Unmarshal(binLogPositionBytes, &storedMySQLBinLogPosition); err != nil { + mgr.Logger().With("error", err.Error()).Error("Failed to unmarshal stored binlog position.") + return + } + + streamInput.startBinLogPosition = &storedMySQLBinLogPosition + binLogCacheKey = string(binLogPositionBytes) + }); err != nil { + mgr.Logger().With("error", err.Error()).Error("Failed to access cursor cache.") + } + } + + i := &streamInput + + r, err := service.AutoRetryNacksBatchedToggled(conf, i) + if err != nil { + return nil, err + } + + return conf.WrapBatchInputExtractTracingSpanMapping("mysql_stream", r) +} + +func init() { + err := service.RegisterBatchInput( + "mysql_stream", mysqlStreamConfigSpec, + func(conf *service.ParsedConfig, mgr *service.Resources) (service.BatchInput, error) { + return newMySQLStreamInput(conf, mgr) + }) + if err != nil { + panic(err) + } +} + +// ---- Redpanda Connect specific methods---- + +func (i *mysqlStreamInput) Connect(ctx context.Context) error { + canalConfig := canal.NewDefaultConfig() + canalConfig.Addr = i.mysqlConfig.Addr + canalConfig.User = i.mysqlConfig.User + canalConfig.Password = i.mysqlConfig.Passwd + canalConfig.Dump.TableDB = i.mysqlConfig.DBName + + // Parse and set additional parameters + canalConfig.Charset = i.mysqlConfig.Collation + if i.mysqlConfig.TLS != nil { + canalConfig.TLSConfig = i.mysqlConfig.TLS + } + canalConfig.ParseTime = true + canalConfig.IncludeTableRegex = i.tables + + c, err := canal.NewCanal(canalConfig) + if err != nil { + return err + } + + // register event handler + c.SetEventHandler(i) + + i.canal = c + go i.startMySQLSync() + return nil +} + +func (i *mysqlStreamInput) startMySQLSync() { + // If we require snapshot streaming && we don't have a binlog position cache + // initiate default run for Canal to process snapshot and start incremental sync of binlog + if i.streamSnapshot && i.startBinLogPosition == nil { + // Doesn't work at the moment + if err := i.canal.Run(); err != nil { + panic(err) + } + } else { + coords, _ := i.canal.GetMasterPos() + // starting from the last stored binlog position + if i.startBinLogPosition != nil { + coords = *i.startBinLogPosition + } + + if err := i.canal.RunFrom(coords); err != nil { + panic(err) + } + } +} + +func (i *mysqlStreamInput) flushBatch(ctx context.Context, checkpointer *checkpoint.Capped[*int64], msg service.MessageBatch, lsn *int64) bool { + if msg == nil { + return true + } + + resolveFn, err := checkpointer.Track(ctx, lsn, int64(len(msg))) + if err != nil { + if ctx.Err() == nil { + i.logger.Errorf("Failed to checkpoint offset: %v\n", err) + } + return false + } + + select { + case i.msgChan <- asyncMessage{ + msg: msg, + ackFn: func(ctx context.Context, res error) error { + maxOffset := resolveFn() + if maxOffset == nil { + return nil + } + i.cMut.Lock() + defer i.cMut.Unlock() + + // todo;; store offset + + return nil + }, + }: + case <-ctx.Done(): + return false + } + + return true +} + +func (i *mysqlStreamInput) onMessage(e *canal.RowsEvent, params ProcessEventParams) error { + for i := params.initValue; i < len(e.Rows); i += params.incrementValue { + message := map[string]any{} + for i, v := range e.Rows[i] { + message[e.Table.Columns[i].Name] = v + } + + fmt.Println("mysql row", message) + } + return nil +} + +func (i *mysqlStreamInput) ReadBatch(ctx context.Context) (service.MessageBatch, service.AckFunc, error) { + i.cMut.Lock() + msgChan := i.msgChan + i.cMut.Unlock() + if msgChan == nil { + return nil, nil, service.ErrNotConnected + } + + select { + case m, open := <-msgChan: + if !open { + return nil, nil, service.ErrNotConnected + } + return m.msg, m.ackFn, nil + case <-ctx.Done(): + } + + return nil, nil, ctx.Err() +} + +func (i *mysqlStreamInput) Close(ctx context.Context) error { + i.canal.Close() + return nil +} + +// ---- Redpanda Connect specific methods end---- + +// --- MySQL Canal handler methods ---- + +func (i *mysqlStreamInput) OnRotate(*replication.EventHeader, *replication.RotateEvent) error { + return nil +} +func (i *mysqlStreamInput) OnTableChanged(*replication.EventHeader, string, string) error { + return nil +} +func (i *mysqlStreamInput) OnDDL(*replication.EventHeader, mysqlReplications.Position, *replication.QueryEvent) error { + return nil +} +func (i *mysqlStreamInput) OnRow(e *canal.RowsEvent) error { + switch e.Action { + case canal.InsertAction: + return i.onMessage(e, ProcessEventParams{initValue: 0, incrementValue: 1}) + case canal.DeleteAction: + return i.onMessage(e, ProcessEventParams{initValue: 0, incrementValue: 1}) + case canal.UpdateAction: + return i.onMessage(e, ProcessEventParams{initValue: 1, incrementValue: 2}) + default: + return errors.New("invalid rows action") + } +} + +func (i *mysqlStreamInput) OnXID(*replication.EventHeader, mysqlReplications.Position) error { + return nil +} +func (i *mysqlStreamInput) OnGTID(*replication.EventHeader, mysqlReplications.BinlogGTIDEvent) error { + return nil +} +func (i *mysqlStreamInput) OnPosSynced(*replication.EventHeader, mysqlReplications.Position, mysqlReplications.GTIDSet, bool) error { + return nil +} + +// --- MySQL Canal handler methods end ---- diff --git a/internal/impl/mysql/integration_test.go b/internal/impl/mysql/integration_test.go new file mode 100644 index 0000000000..7aceaad2ea --- /dev/null +++ b/internal/impl/mysql/integration_test.go @@ -0,0 +1,18 @@ +// Copyright 2024 Redpanda Data, Inc. +// +// Licensed as a Redpanda Enterprise file under the Redpanda Community +// License (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// https://github.com/redpanda-data/connect/v4/blob/main/licenses/rcl.md + +package mysql + +import ( + "github.com/redpanda-data/benthos/v4/public/service/integration" + "testing" +) + +func TestIntegrationMySQLCDC(t *testing.T) { + integration.CheckSkip(t) +} diff --git a/internal/impl/mysql/message.go b/internal/impl/mysql/message.go new file mode 100644 index 0000000000..4f3a5b8d13 --- /dev/null +++ b/internal/impl/mysql/message.go @@ -0,0 +1,16 @@ +// Copyright 2024 Redpanda Data, Inc. +// +// Licensed as a Redpanda Enterprise file under the Redpanda Community +// License (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// https://github.com/redpanda-data/connect/v4/blob/main/licenses/rcl.md + +package mysql + +type ChangeMessage struct { + IsStreaming bool `json:"is_streaming"` + Table string `json:"table"` + Event string `json:"event"` + Data map[string]any `json:"data"` +} From 19dab6ede36fd6cf01a1ba8aa270bd771e0b3a1b Mon Sep 17 00:00:00 2001 From: Vladyslav Len Date: Fri, 8 Nov 2024 10:59:03 +0100 Subject: [PATCH 02/45] chore(): continue working on mysql cdc --- go.mod | 11 +++- go.sum | 23 +++++++- internal/impl/mysql/input_mysql_stream.go | 37 ++++++++++--- internal/impl/mysql/integration_test.go | 66 ++++++++++++++++++++++- 4 files changed, 126 insertions(+), 11 deletions(-) diff --git a/go.mod b/go.mod index 99de6b470e..f6abf2bb1d 100644 --- a/go.mod +++ b/go.mod @@ -65,6 +65,7 @@ require ( github.com/getsentry/sentry-go v0.28.1 github.com/go-faker/faker/v4 v4.4.2 github.com/go-jose/go-jose/v3 v3.0.3 + github.com/go-mysql-org/go-mysql v1.9.1 github.com/go-resty/resty/v2 v2.15.3 github.com/go-sql-driver/mysql v1.8.1 github.com/gocql/gocql v1.6.0 @@ -161,16 +162,18 @@ require ( 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/BurntSushi/toml v1.4.1-0.20240526193622-a339e1f7089c // indirect github.com/GoogleCloudPlatform/grpc-gcp-go/grpcgcp v1.5.0 // indirect github.com/GoogleCloudPlatform/opentelemetry-operations-go/detectors/gcp v1.24.1 // indirect + github.com/Masterminds/semver v1.5.0 // indirect github.com/aws/aws-sdk-go-v2/service/secretsmanager v1.34.3 // indirect github.com/census-instrumentation/opencensus-proto v0.4.1 // indirect github.com/certifi/gocertifi v0.0.0-20210507211836-431795d63e8d // indirect github.com/cncf/xds/go v0.0.0-20240905190251-b4127c9b8d78 // indirect github.com/containerd/platforms v0.2.1 // indirect + github.com/cznic/mathutil v0.0.0-20181122101859-297441e03548 // indirect github.com/envoyproxy/go-control-plane v0.13.0 // indirect github.com/envoyproxy/protoc-gen-validate v1.1.0 // indirect - github.com/go-mysql-org/go-mysql v1.9.1 // indirect github.com/hamba/avro/v2 v2.22.2-0.20240625062549-66aad10411d9 // indirect github.com/json-iterator/go v1.1.12 // indirect github.com/jzelinskie/stringz v0.0.3 // indirect @@ -178,8 +181,14 @@ require ( github.com/modern-go/reflect2 v1.0.2 // indirect github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822 // indirect github.com/onsi/gomega v1.34.2 // indirect + github.com/pingcap/errors v0.11.5-0.20221009092201-b66cddb77c32 // indirect + github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c // indirect + github.com/pingcap/log v1.1.1-0.20230317032135-a0d097d16e22 // indirect + github.com/pingcap/tidb/pkg/parser v0.0.0-20231103042308-035ad5ccbe67 // indirect github.com/planetscale/vtprotobuf v0.6.1-0.20240319094008-0393e58bdf10 // indirect github.com/samber/lo v1.47.0 // indirect + github.com/siddontang/go v0.0.0-20180604090527-bdc77568d726 // indirect + github.com/siddontang/go-log v0.0.0-20180807004314-8d05993dda07 // indirect github.com/tidwall/gjson v1.18.0 // indirect github.com/tidwall/match v1.1.1 // indirect github.com/tidwall/pretty v1.2.1 // indirect diff --git a/go.sum b/go.sum index 38fd8ec560..0863e07f45 100644 --- a/go.sum +++ b/go.sum @@ -695,6 +695,8 @@ github.com/AzureAD/microsoft-authentication-extensions-for-go/cache v0.1.1/go.mo github.com/AzureAD/microsoft-authentication-library-for-go v1.2.2 h1:XHOnouVk1mxXfQidrMEnLlPk9UMeRtyBTnEFtxkV0kU= github.com/AzureAD/microsoft-authentication-library-for-go v1.2.2/go.mod h1:wP83P5OoQ5p6ip3ScPr0BAq0BvuPAvacpEuSzyouqAI= github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= +github.com/BurntSushi/toml v1.4.1-0.20240526193622-a339e1f7089c h1:pxW6RcqyfI9/kWtOwnv/G+AzdKuy2ZrqINhenH4HyNs= +github.com/BurntSushi/toml v1.4.1-0.20240526193622-a339e1f7089c/go.mod h1:ukJfTF/6rtPPRCnwkur4qwRxa8vTRFBF0uk2lLoLwho= github.com/BurntSushi/xgb v0.0.0-20160522181843-27f122750802/go.mod h1:IVnqGOEym/WlBOVXweHU+Q+/VP0lqqI8lqeDx9IjBqo= github.com/ClickHouse/ch-go v0.61.5 h1:zwR8QbYI0tsMiEcze/uIMK+Tz1D3XZXLdNrlaOpeEI4= github.com/ClickHouse/ch-go v0.61.5/go.mod h1:s1LJW/F/LcFs5HJnuogFMta50kKDO0lf9zzfrbl0RQg= @@ -728,6 +730,8 @@ github.com/Jeffail/shutdown v1.0.0 h1:afYjnY4pksqP/012m3NGJVccDI+WATdSzIMVHZKU8/ github.com/Jeffail/shutdown v1.0.0/go.mod h1:5dT4Y1oe60SJELCkmAB1pr9uQyHBhh6cwDLQTfmuO5U= github.com/JohnCGriffin/overflow v0.0.0-20211019200055-46fa312c352c h1:RGWPOewvKIROun94nF7v2cua9qP+thov/7M50KEoeSU= github.com/JohnCGriffin/overflow v0.0.0-20211019200055-46fa312c352c/go.mod h1:X0CRv0ky0k6m906ixxpzmDRLvX58TFUKS2eePweuyxk= +github.com/Masterminds/semver v1.5.0 h1:H65muMkzWKEuNDnfl9d70GUjFniHKHRbFPGBuZ3QEww= +github.com/Masterminds/semver v1.5.0/go.mod h1:MB6lktGJrhw8PrUyiEoblNEGEQ+RzHPF078ddwwvV3Y= github.com/Masterminds/semver/v3 v3.1.1/go.mod h1:VPu/7SZ7ePZ3QOrcuXROw5FAcLl4a0cBrbBpGY/8hQs= github.com/Masterminds/semver/v3 v3.3.0 h1:B8LGeaivUe71a5qox1ICM/JLl0NqZSW5CHyL+hmvYS0= github.com/Masterminds/semver/v3 v3.3.0/go.mod h1:4V+yj/TJE1HU9XfppCwVMZq3I84lprf4nC11bSS5beM= @@ -993,6 +997,8 @@ github.com/creack/pty v1.1.7/go.mod h1:lj5s0c3V2DBrqTV7llrYr5NG6My20zk30Fl46Y7Do github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= github.com/creack/pty v1.1.21 h1:1/QdRyBaHHJP61QkWMXlOIBfsgdDeeKfK8SYVUWJKf0= github.com/creack/pty v1.1.21/go.mod h1:MOBLtS5ELjhRRrroQr9kyvTxUAFNvYEK993ew/Vr4O4= +github.com/cznic/mathutil v0.0.0-20181122101859-297441e03548 h1:iwZdTE0PVqJCos1vaoKsclOGD3ADKpshg3SRtYBbwso= +github.com/cznic/mathutil v0.0.0-20181122101859-297441e03548/go.mod h1:e6NPNENfs9mPDVNRekM7lKScauxd5kXTr1Mfyig6TDM= github.com/danieljoos/wincred v1.2.0 h1:ozqKHaLK0W/ii4KVbbvluM91W2H3Sh0BncbUNPS7jLE= github.com/danieljoos/wincred v1.2.0/go.mod h1:FzQLLMKBFdvu+osBrnFODiv32YGwCfx0SkRa/eYHgec= github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= @@ -1668,9 +1674,16 @@ github.com/pierrec/lz4/v4 v4.1.21 h1:yOVMLb6qSIDP67pl/5F7RepeKYu/VmTyEXvuMI5d9mQ github.com/pierrec/lz4/v4 v4.1.21/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= github.com/pinecone-io/go-pinecone v1.0.0 h1:90euw+0EKSgdeE9q7iGSTVmdx9r9+x3mxWkrCCLab+o= github.com/pinecone-io/go-pinecone v1.0.0/go.mod h1:KfJhn4yThX293+fbtrZLnxe2PJYo8557Py062W4FYKk= -github.com/pingcap/errors v0.11.4 h1:lFuQV/oaUMGcD2tqt+01ROSmJs75VG1ToEOkZIZ4nE4= +github.com/pingcap/errors v0.11.0/go.mod h1:Oi8TUi2kEtXXLMJk9l1cGmz20kV3TaQ0usTwv5KuLY8= github.com/pingcap/errors v0.11.4/go.mod h1:Oi8TUi2kEtXXLMJk9l1cGmz20kV3TaQ0usTwv5KuLY8= github.com/pingcap/errors v0.11.5-0.20221009092201-b66cddb77c32 h1:m5ZsBa5o/0CkzZXfXLaThzKuR85SnHHetqBCpzQ30h8= +github.com/pingcap/errors v0.11.5-0.20221009092201-b66cddb77c32/go.mod h1:X2r9ueLEUZgtx2cIogM0v4Zj5uvvzhuuiu7Pn8HzMPg= +github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c h1:CgbKAHto5CQgWM9fSBIvaxsJHuGP0uM74HXtv3MyyGQ= +github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c/go.mod h1:4qGtCB0QK0wBzKtFEGDhxXnSnbQApw1gc9siScUl8ew= +github.com/pingcap/log v1.1.1-0.20230317032135-a0d097d16e22 h1:2SOzvGvE8beiC1Y4g9Onkvu6UmuBBOeWRGQEjJaT/JY= +github.com/pingcap/log v1.1.1-0.20230317032135-a0d097d16e22/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= +github.com/pingcap/tidb/pkg/parser v0.0.0-20231103042308-035ad5ccbe67 h1:m0RZ583HjzG3NweDi4xAcK54NBBPJh+zXp5Fp60dHtw= +github.com/pingcap/tidb/pkg/parser v0.0.0-20231103042308-035ad5ccbe67/go.mod h1:yRkiqLFwIqibYg2P7h4bclHjHcJiIFRLKhGRyBcKYus= github.com/pkg/browser v0.0.0-20180916011732-0a3d74bf9ce4/go.mod h1:4OwLy04Bl9Ef3GJJCoec+30X3LQs/0/m4HFRt/2LUSA= github.com/pkg/browser v0.0.0-20240102092130-5ac0b6a4141c h1:+mdjkGKdHQG3305AYmdv1U2eRNDiU2ErMBj1gwrq8eQ= github.com/pkg/browser v0.0.0-20240102092130-5ac0b6a4141c/go.mod h1:7rwL4CYBLnjLxUqIJNnCWiEdr3bn6IUYi15bNlnbCCU= @@ -1778,6 +1791,7 @@ github.com/segmentio/encoding v0.4.0 h1:MEBYvRqiUB2nfR2criEXWqwdY6HJOUrCn5hboVOV github.com/segmentio/encoding v0.4.0/go.mod h1:/d03Cd8PoaDeceuhUUUQWjU0KhWjrmYrWPgtJHYZSnI= github.com/segmentio/ksuid v1.0.4 h1:sBo2BdShXjmcugAMwjugoGUdUV0pcxY5mW4xKRn3v4c= github.com/segmentio/ksuid v1.0.4/go.mod h1:/XUiZBD3kVx5SmUOl55voK5yeAbBNNIed+2O73XgrPE= +github.com/sergi/go-diff v1.1.0/go.mod h1:STckp+ISIX8hZLjrqAeVduY0gWCT9IjLuqbuNXdaHfM= github.com/shirou/gopsutil/v3 v3.24.5 h1:i0t8kL+kQTvpAYToeuiVk3TgDeKOFioZO3Ztz/iZ9pI= github.com/shirou/gopsutil/v3 v3.24.5/go.mod h1:bsoOS1aStSs9ErQ1WWfxllSeS1K5D+U30r2NfcubMVk= github.com/shoenig/go-m1cpu v0.1.6 h1:nxdKQNcEB6vzgA2E2bvzKIYRuNj7XNJ4S/aRSwKzFtM= @@ -1789,6 +1803,10 @@ github.com/shopspring/decimal v1.2.0/go.mod h1:DKyhrW/HYNuLGql+MJL6WCR6knT2jwCFR github.com/shopspring/decimal v1.3.1/go.mod h1:DKyhrW/HYNuLGql+MJL6WCR6knT2jwCFRcu2hWCYk4o= github.com/shopspring/decimal v1.4.0 h1:bxl37RwXBklmTi0C79JfXCEBD1cqqHt0bbgBAGFp81k= github.com/shopspring/decimal v1.4.0/go.mod h1:gawqmDU56v4yIKSwfBSFip1HdCCXN8/+DMd9qYNcwME= +github.com/siddontang/go v0.0.0-20180604090527-bdc77568d726 h1:xT+JlYxNGqyT+XcU8iUrN18JYed2TvG9yN5ULG2jATM= +github.com/siddontang/go v0.0.0-20180604090527-bdc77568d726/go.mod h1:3yhqj7WBBfRhbBlzyOC3gUxftwsU0u8gqevxwIHQpMw= +github.com/siddontang/go-log v0.0.0-20180807004314-8d05993dda07 h1:oI+RNwuC9jF2g2lP0u0cVEEZrc/AYBCuFdvwrLWM/6Q= +github.com/siddontang/go-log v0.0.0-20180807004314-8d05993dda07/go.mod h1:yFdBgwXP24JziuRl2NMUahT7nGLNOKi1SIiFxMttVD4= github.com/sijms/go-ora/v2 v2.8.19 h1:7LoKZatDYGi18mkpQTR/gQvG9yOdtc7hPAex96Bqisc= github.com/sijms/go-ora/v2 v2.8.19/go.mod h1:EHxlY6x7y9HAsdfumurRfTd+v8NrEOTR3Xl4FWlH6xk= github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= @@ -1991,6 +2009,7 @@ go.uber.org/atomic v1.4.0/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE= go.uber.org/atomic v1.5.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= go.uber.org/atomic v1.6.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= +go.uber.org/atomic v1.9.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.11.0 h1:ZvwS0R+56ePWxUNi+Atn9dWONBPp/AUETXlHW0DxSjE= go.uber.org/atomic v1.11.0/go.mod h1:LUxbIzbOniOlMKjJjyPfpl4v+PKK2cNJn91OQbhoJI0= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= @@ -2000,6 +2019,7 @@ go.uber.org/multierr v1.1.0/go.mod h1:wR5kodmAFQ0UK8QlbwjlSNy0Z68gJhDJUG5sjR94q/ go.uber.org/multierr v1.3.0/go.mod h1:VgVr7evmIr6uPjLBxg28wmKNXyqE9akIJ5XnfpiKl+4= go.uber.org/multierr v1.5.0/go.mod h1:FeouvMocqHpRaaGuG9EjoKcStLC43Zu/fmqdUMPcKYU= go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= +go.uber.org/multierr v1.7.0/go.mod h1:7EAYxJLBy9rStEaz58O2t4Uvip6FSURkq8/ppBp95ak= go.uber.org/multierr v1.11.0 h1:blXXJkSxSSfBVBlC76pxqeO+LN3aDfLQo+309xJstO0= go.uber.org/multierr v1.11.0/go.mod h1:20+QtiLqy0Nd6FdQB9TLXag12DsQkrbs3htMFfDN80Y= go.uber.org/tools v0.0.0-20190618225709-2cfd321de3ee/go.mod h1:vJERXedbb3MVM5f9Ejo0C68/HhF8uaILCdgjnY+goOA= @@ -2007,6 +2027,7 @@ go.uber.org/zap v1.9.1/go.mod h1:vwi/ZaCAaUcBkycHslxD9B2zi4UTXhF60s6SWpuDF0Q= go.uber.org/zap v1.10.0/go.mod h1:vwi/ZaCAaUcBkycHslxD9B2zi4UTXhF60s6SWpuDF0Q= go.uber.org/zap v1.13.0/go.mod h1:zwrFLgMcdUuIBviXEYEH1YKNaOBnKXsx2IPda5bBwHM= go.uber.org/zap v1.18.1/go.mod h1:xg/QME4nWcxGxrpdeYfq7UvYrLh66cuVKdrbD1XF/NI= +go.uber.org/zap v1.19.0/go.mod h1:xg/QME4nWcxGxrpdeYfq7UvYrLh66cuVKdrbD1XF/NI= go.uber.org/zap v1.27.0 h1:aJMhYGrd5QSmlpLMr2MftRKl7t8J8PTZPA732ud/XR8= go.uber.org/zap v1.27.0/go.mod h1:GB2qFLM7cTU87MWRP2mPIjqfIDnGu+VIO4V/SdhGo2E= golang.org/x/crypto v0.0.0-20180723164146-c126467f60eb/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= diff --git a/internal/impl/mysql/input_mysql_stream.go b/internal/impl/mysql/input_mysql_stream.go index 5e92bf9360..0a2468ec3b 100644 --- a/internal/impl/mysql/input_mysql_stream.go +++ b/internal/impl/mysql/input_mysql_stream.go @@ -13,13 +13,15 @@ import ( "encoding/json" "errors" "fmt" + "sync" + "time" + "github.com/Jeffail/checkpoint" "github.com/go-mysql-org/go-mysql/canal" mysqlReplications "github.com/go-mysql-org/go-mysql/mysql" "github.com/go-mysql-org/go-mysql/replication" "github.com/go-sql-driver/mysql" "github.com/redpanda-data/benthos/v4/public/service" - "sync" ) const ( @@ -66,6 +68,7 @@ type mysqlStreamInput struct { mysqlConfig *mysql.Config canal.DummyEventHandler startBinLogPosition *mysqlReplications.Position + currentLogPosition *mysqlReplications.Position dsn string tables []string @@ -81,7 +84,8 @@ type mysqlStreamInput struct { func newMySQLStreamInput(conf *service.ParsedConfig, mgr *service.Resources) (s service.BatchInput, err error) { streamInput := mysqlStreamInput{ - logger: mgr.Logger(), + logger: mgr.Logger(), + msgChan: make(chan asyncMessage), } if streamInput.dsn, err = conf.FieldString(fieldMySQLDSN); err != nil { @@ -112,7 +116,7 @@ func newMySQLStreamInput(conf *service.ParsedConfig, mgr *service.Resources) (s binLogPositionBytes, cErr := c.Get(context.Background(), binLogCacheKey) if err != nil { if !errors.Is(cErr, service.ErrKeyNotFound) { - mgr.Logger().With("error", cErr.Error()).Error("Failed to obtain cursor cache item.") + mgr.Logger().Errorf("failed to obtain cursor cache item. %v", cErr) } return } @@ -132,6 +136,8 @@ func newMySQLStreamInput(conf *service.ParsedConfig, mgr *service.Resources) (s i := &streamInput + mgr.Logger().Info("Starting MySQL stream input") + r, err := service.AutoRetryNacksBatchedToggled(conf, i) if err != nil { return nil, err @@ -144,7 +150,9 @@ func init() { err := service.RegisterBatchInput( "mysql_stream", mysqlStreamConfigSpec, func(conf *service.ParsedConfig, mgr *service.Resources) (service.BatchInput, error) { - return newMySQLStreamInput(conf, mgr) + s, err := newMySQLStreamInput(conf, mgr) + fmt.Println("New service", err) + return s, err }) if err != nil { panic(err) @@ -155,10 +163,12 @@ func init() { func (i *mysqlStreamInput) Connect(ctx context.Context) error { canalConfig := canal.NewDefaultConfig() + canalConfig.Flavor = i.flavor canalConfig.Addr = i.mysqlConfig.Addr canalConfig.User = i.mysqlConfig.User canalConfig.Password = i.mysqlConfig.Passwd canalConfig.Dump.TableDB = i.mysqlConfig.DBName + fmt.Println(i.mysqlConfig.Passwd, i.mysqlConfig.User, i.mysqlConfig.Addr, i.mysqlConfig.DBName) // Parse and set additional parameters canalConfig.Charset = i.mysqlConfig.Collation @@ -172,9 +182,7 @@ func (i *mysqlStreamInput) Connect(ctx context.Context) error { if err != nil { return err } - - // register event handler - c.SetEventHandler(i) + c.AddDumpTables(i.mysqlConfig.DBName, i.tables...) i.canal = c go i.startMySQLSync() @@ -182,11 +190,15 @@ func (i *mysqlStreamInput) Connect(ctx context.Context) error { } func (i *mysqlStreamInput) startMySQLSync() { + i.canal.SetEventHandler(i) + fmt.Println("Starting MySQL sync") // If we require snapshot streaming && we don't have a binlog position cache // initiate default run for Canal to process snapshot and start incremental sync of binlog if i.streamSnapshot && i.startBinLogPosition == nil { // Doesn't work at the moment + fmt.Println("Run binglo sync....") if err := i.canal.Run(); err != nil { + fmt.Println("Mysql stream error: ", err) panic(err) } } else { @@ -196,6 +208,7 @@ func (i *mysqlStreamInput) startMySQLSync() { coords = *i.startBinLogPosition } + i.currentLogPosition = &coords if err := i.canal.RunFrom(coords); err != nil { panic(err) } @@ -239,6 +252,10 @@ func (i *mysqlStreamInput) flushBatch(ctx context.Context, checkpointer *checkpo } func (i *mysqlStreamInput) onMessage(e *canal.RowsEvent, params ProcessEventParams) error { + i.cMut.Lock() + i.currentLogPosition.Pos = e.Header.LogPos + i.cMut.Unlock() + for i := params.initValue; i < len(e.Rows); i += params.incrementValue { message := map[string]any{} for i, v := range e.Rows[i] { @@ -247,6 +264,10 @@ func (i *mysqlStreamInput) onMessage(e *canal.RowsEvent, params ProcessEventPara fmt.Println("mysql row", message) } + + // update cache checkpoint + fmt.Println("Log position.....", i.currentLogPosition.Pos, i.currentLogPosition.Name) + return nil } @@ -258,6 +279,8 @@ func (i *mysqlStreamInput) ReadBatch(ctx context.Context) (service.MessageBatch, return nil, nil, service.ErrNotConnected } + time.Sleep(time.Second * 10) + select { case m, open := <-msgChan: if !open { diff --git a/internal/impl/mysql/integration_test.go b/internal/impl/mysql/integration_test.go index 7aceaad2ea..6f419d688e 100644 --- a/internal/impl/mysql/integration_test.go +++ b/internal/impl/mysql/integration_test.go @@ -9,10 +9,72 @@ package mysql import ( - "github.com/redpanda-data/benthos/v4/public/service/integration" + "context" + "fmt" + "os" + "sync" "testing" + "time" + + _ "github.com/redpanda-data/benthos/v4/public/components/io" + _ "github.com/redpanda-data/benthos/v4/public/components/pure" + "github.com/redpanda-data/benthos/v4/public/service" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" ) func TestIntegrationMySQLCDC(t *testing.T) { - integration.CheckSkip(t) + mysqlDsn := os.Getenv("LOCAL_MYSQL_DSN") + tmpDir := t.TempDir() + + fmt.Println("TMP Ditr: ", tmpDir) + template := fmt.Sprintf(` +mysql_stream: + dsn: %s + stream_snapshot: false + checkpoint_key: binlogpos + tables: + - users + checkpoint_key: foocache + flavor: mysql +`, mysqlDsn) + + cacheConf := fmt.Sprintf(` +label: mysql_stream_cache +file: + directory: %v`, tmpDir) + + streamOutBuilder := service.NewStreamBuilder() + require.NoError(t, streamOutBuilder.SetLoggerYAML(`level: INFO`)) + require.NoError(t, streamOutBuilder.AddCacheYAML(cacheConf)) + require.NoError(t, streamOutBuilder.AddInputYAML(template)) + + var outBatches []string + var outBatchMut sync.Mutex + require.NoError(t, streamOutBuilder.AddBatchConsumerFunc(func(c context.Context, mb service.MessageBatch) error { + msgBytes, err := mb[0].AsBytes() + require.NoError(t, err) + outBatchMut.Lock() + outBatches = append(outBatches, string(msgBytes)) + outBatchMut.Unlock() + return nil + })) + + streamOut, err := streamOutBuilder.Build() + require.NoError(t, err) + + go func() { + err = streamOut.Run(context.Background()) + require.NoError(t, err) + }() + + assert.Eventually(t, func() bool { + outBatchMut.Lock() + defer outBatchMut.Unlock() + return len(outBatches) == 10000 + }, time.Minute*5, time.Millisecond*100) + + require.NoError(t, streamOut.StopWithin(time.Second*10)) } + +// mysqldump --host=public-vultr-prod-a70dc516-1330-488a-bf57-712a3d91be58-vultr-pr.vultrdb.com --port=16751 --user=vultradmin --password=AVNS_fgGkgy43bJw3NzNwDCV --master-data --single-transaction --skip-lock-tables --compact --skip-opt --quick --no-create-info --skip-extended-insert --skip-tz-utc --hex-blob --column-statistics=0 defaultdb users From 29c950fbd4be2f9e71fca2efa144a11cab7bd6c2 Mon Sep 17 00:00:00 2001 From: Vladyslav Len Date: Fri, 8 Nov 2024 11:00:14 +0100 Subject: [PATCH 03/45] chore(): removed in test secret --- internal/impl/mysql/integration_test.go | 2 -- 1 file changed, 2 deletions(-) diff --git a/internal/impl/mysql/integration_test.go b/internal/impl/mysql/integration_test.go index 6f419d688e..22041aa7f3 100644 --- a/internal/impl/mysql/integration_test.go +++ b/internal/impl/mysql/integration_test.go @@ -76,5 +76,3 @@ file: require.NoError(t, streamOut.StopWithin(time.Second*10)) } - -// mysqldump --host=public-vultr-prod-a70dc516-1330-488a-bf57-712a3d91be58-vultr-pr.vultrdb.com --port=16751 --user=vultradmin --password=AVNS_fgGkgy43bJw3NzNwDCV --master-data --single-transaction --skip-lock-tables --compact --skip-opt --quick --no-create-info --skip-extended-insert --skip-tz-utc --hex-blob --column-statistics=0 defaultdb users From 3391a645e80ad67adb70075593b51ac6bb8719a5 Mon Sep 17 00:00:00 2001 From: Vladyslav Len Date: Wed, 13 Nov 2024 13:45:56 +0100 Subject: [PATCH 04/45] chore(): working on mysql cdc --- internal/impl/mysql/event.go | 25 +++ internal/impl/mysql/input_mysql_stream.go | 209 +++++++++++++++++----- internal/impl/mysql/integration_test.go | 14 +- internal/impl/mysql/snapshot.go | 3 + 4 files changed, 202 insertions(+), 49 deletions(-) create mode 100644 internal/impl/mysql/snapshot.go diff --git a/internal/impl/mysql/event.go b/internal/impl/mysql/event.go index 04919951ee..2ad5dba70e 100644 --- a/internal/impl/mysql/event.go +++ b/internal/impl/mysql/event.go @@ -8,6 +8,31 @@ package mysql +import "github.com/go-mysql-org/go-mysql/mysql" + type ProcessEventParams struct { initValue, incrementValue int } + +type MessageOperation string + +const ( + MessageOperationInsert MessageOperation = "insert" + MessageOperationUpdate MessageOperation = "update" + MessageOperationDelete MessageOperation = "delete" +) + +type MessageType string + +const ( + MessageTypeEvent MessageType = "snapshot" + MessageTypeStreaming MessageType = "streaming" +) + +type MessageEvent struct { + Row map[string]any `json:"row"` + Table string `json:"table"` + Operation MessageOperation `json:"operation"` + Type MessageType `json:"type"` + Position *mysql.Position `json:"position"` +} diff --git a/internal/impl/mysql/input_mysql_stream.go b/internal/impl/mysql/input_mysql_stream.go index 0a2468ec3b..7f60af2545 100644 --- a/internal/impl/mysql/input_mysql_stream.go +++ b/internal/impl/mysql/input_mysql_stream.go @@ -63,31 +63,42 @@ type asyncMessage struct { } type mysqlStreamInput struct { - // canal represents mysql binlog listener connection + mutex sync.Mutex + // canal stands for mysql binlog listener connection canal *canal.Canal mysqlConfig *mysql.Config canal.DummyEventHandler startBinLogPosition *mysqlReplications.Position currentLogPosition *mysqlReplications.Position + binLogCache string dsn string tables []string flavor string streamSnapshot bool - cMut sync.Mutex - msgChan chan asyncMessage - batching service.BatchPolicy + rawMessageEvents chan MessageEvent + msgChan chan asyncMessage + batching service.BatchPolicy + batchPolicy *service.Batcher + checkPointLimit int logger *service.Logger + res *service.Resources } -func newMySQLStreamInput(conf *service.ParsedConfig, mgr *service.Resources) (s service.BatchInput, err error) { +const binLogCacheKey = "mysql_binlog_position" + +func newMySQLStreamInput(conf *service.ParsedConfig, res *service.Resources) (s service.BatchInput, err error) { streamInput := mysqlStreamInput{ - logger: mgr.Logger(), - msgChan: make(chan asyncMessage), + logger: res.Logger(), + rawMessageEvents: make(chan MessageEvent), + msgChan: make(chan asyncMessage), + res: res, } + var batching service.BatchPolicy + if streamInput.dsn, err = conf.FieldString(fieldMySQLDSN); err != nil { return nil, err } @@ -109,34 +120,51 @@ func newMySQLStreamInput(conf *service.ParsedConfig, mgr *service.Resources) (s return nil, err } - if binLogCacheKey, err := conf.FieldString(fieldCheckpointKey); err != nil { + if streamInput.checkPointLimit, err = conf.FieldInt(fieldCheckpointLimit); err != nil { + return nil, err + } + + if streamInput.binLogCache, err = conf.FieldString(fieldCheckpointKey); err != nil { return nil, err } else { - if err := mgr.AccessCache(context.Background(), binLogCacheKey, func(c service.Cache) { + if err := res.AccessCache(context.Background(), streamInput.binLogCache, func(c service.Cache) { binLogPositionBytes, cErr := c.Get(context.Background(), binLogCacheKey) if err != nil { if !errors.Is(cErr, service.ErrKeyNotFound) { - mgr.Logger().Errorf("failed to obtain cursor cache item. %v", cErr) + res.Logger().Errorf("failed to obtain cursor cache item. %v", cErr) } return } var storedMySQLBinLogPosition mysqlReplications.Position if err = json.Unmarshal(binLogPositionBytes, &storedMySQLBinLogPosition); err != nil { - mgr.Logger().With("error", err.Error()).Error("Failed to unmarshal stored binlog position.") + res.Logger().With("error", err.Error()).Error("Failed to unmarshal stored binlog position.") return } streamInput.startBinLogPosition = &storedMySQLBinLogPosition - binLogCacheKey = string(binLogPositionBytes) }); err != nil { - mgr.Logger().With("error", err.Error()).Error("Failed to access cursor cache.") + + res.Logger().With("error", err.Error()).Error("Failed to access cursor cache.") } } i := &streamInput - mgr.Logger().Info("Starting MySQL stream input") + res.Logger().Info("Starting MySQL stream input") + + if batching, err = conf.FieldBatchPolicy(fieldBatching); err != nil { + return nil, err + } else if batching.IsNoop() { + batching.Count = 1 + } + + i.batching = batching + if i.batchPolicy, err = i.batching.NewBatcher(res); err != nil { + return nil, err + } else if batching.IsNoop() { + batching.Count = 1 + } r, err := service.AutoRetryNacksBatchedToggled(conf, i) if err != nil { @@ -150,9 +178,7 @@ func init() { err := service.RegisterBatchInput( "mysql_stream", mysqlStreamConfigSpec, func(conf *service.ParsedConfig, mgr *service.Resources) (service.BatchInput, error) { - s, err := newMySQLStreamInput(conf, mgr) - fmt.Println("New service", err) - return s, err + return newMySQLStreamInput(conf, mgr) }) if err != nil { panic(err) @@ -168,7 +194,6 @@ func (i *mysqlStreamInput) Connect(ctx context.Context) error { canalConfig.User = i.mysqlConfig.User canalConfig.Password = i.mysqlConfig.Passwd canalConfig.Dump.TableDB = i.mysqlConfig.DBName - fmt.Println(i.mysqlConfig.Passwd, i.mysqlConfig.User, i.mysqlConfig.Addr, i.mysqlConfig.DBName) // Parse and set additional parameters canalConfig.Charset = i.mysqlConfig.Collation @@ -185,20 +210,74 @@ func (i *mysqlStreamInput) Connect(ctx context.Context) error { c.AddDumpTables(i.mysqlConfig.DBName, i.tables...) i.canal = c - go i.startMySQLSync() + go i.startMySQLSync(ctx) return nil } -func (i *mysqlStreamInput) startMySQLSync() { +func (i *mysqlStreamInput) readMessages(ctx context.Context) error { + var nextTimedBatchChan <-chan time.Time + var latestPos *mysqlReplications.Position + cp := checkpoint.NewCapped[*int64](int64(i.checkPointLimit)) + + for { + select { + case <-ctx.Done(): + fmt.Println("Stop reading....") + return nil + case <-nextTimedBatchChan: + nextTimedBatchChan = nil + flushedBatch, err := i.batchPolicy.Flush(ctx) + if err != nil { + i.logger.Debugf("Timed flush batch error: %w", err) + break + } + + if ok := i.flushBatch(ctx, cp, flushedBatch, latestPos); !ok { + break + } + case me := <-i.rawMessageEvents: + row, err := json.Marshal(me.Row) + if err != nil { + fmt.Println("Error marshalling row: ", err) + return err + } + + mb := service.NewMessage(row) + mb.MetaSet("operation", string(me.Operation)) + mb.MetaSet("table", me.Table) + mb.MetaSet("type", string(me.Type)) + if me.Position != nil { + latestPos = me.Position + } + + if i.batchPolicy.Add(mb) { + nextTimedBatchChan = nil + flushedBatch, err := i.batchPolicy.Flush(ctx) + if err != nil { + i.logger.Debugf("Flush batch error: %w", err) + break + } + if ok := i.flushBatch(ctx, cp, flushedBatch, latestPos); !ok { + break + } + } else { + d, ok := i.batchPolicy.UntilNext() + if ok { + nextTimedBatchChan = time.After(d) + } + } + } + } +} + +func (i *mysqlStreamInput) startMySQLSync(ctx context.Context) { i.canal.SetEventHandler(i) - fmt.Println("Starting MySQL sync") + go i.readMessages(ctx) // If we require snapshot streaming && we don't have a binlog position cache // initiate default run for Canal to process snapshot and start incremental sync of binlog if i.streamSnapshot && i.startBinLogPosition == nil { // Doesn't work at the moment - fmt.Println("Run binglo sync....") if err := i.canal.Run(); err != nil { - fmt.Println("Mysql stream error: ", err) panic(err) } } else { @@ -215,12 +294,13 @@ func (i *mysqlStreamInput) startMySQLSync() { } } -func (i *mysqlStreamInput) flushBatch(ctx context.Context, checkpointer *checkpoint.Capped[*int64], msg service.MessageBatch, lsn *int64) bool { +func (i *mysqlStreamInput) flushBatch(ctx context.Context, checkpointer *checkpoint.Capped[*int64], msg service.MessageBatch, binLogPos *mysqlReplications.Position) bool { if msg == nil { return true } - resolveFn, err := checkpointer.Track(ctx, lsn, int64(len(msg))) + posInInt := int64(binLogPos.Pos) + resolveFn, err := checkpointer.Track(ctx, &posInInt, int64(len(msg))) if err != nil { if ctx.Err() == nil { i.logger.Errorf("Failed to checkpoint offset: %v\n", err) @@ -236,10 +316,10 @@ func (i *mysqlStreamInput) flushBatch(ctx context.Context, checkpointer *checkpo if maxOffset == nil { return nil } - i.cMut.Lock() - defer i.cMut.Unlock() - // todo;; store offset + if err := i.syncBinlogPosition(context.Background()); err != nil { + return err + } return nil }, @@ -252,35 +332,68 @@ func (i *mysqlStreamInput) flushBatch(ctx context.Context, checkpointer *checkpo } func (i *mysqlStreamInput) onMessage(e *canal.RowsEvent, params ProcessEventParams) error { - i.cMut.Lock() + i.mutex.Lock() i.currentLogPosition.Pos = e.Header.LogPos - i.cMut.Unlock() + i.mutex.Unlock() - for i := params.initValue; i < len(e.Rows); i += params.incrementValue { + for pi := params.initValue; pi < len(e.Rows); pi += params.incrementValue { message := map[string]any{} - for i, v := range e.Rows[i] { + for i, v := range e.Rows[pi] { message[e.Table.Columns[i].Name] = v } - fmt.Println("mysql row", message) + i.rawMessageEvents <- MessageEvent{ + Row: message, + Operation: MessageOperation(e.Action), + Type: MessageTypeStreaming, + Table: e.Table.Name, + Position: i.currentLogPosition, + } } - // update cache checkpoint - fmt.Println("Log position.....", i.currentLogPosition.Pos, i.currentLogPosition.Name) - return nil } +func (i *mysqlStreamInput) syncBinlogPosition(ctx context.Context) error { + i.mutex.Lock() + defer i.mutex.Unlock() + + if i.currentLogPosition == nil { + i.logger.Warn("No current bingLog position") + return errors.New("no current binlog position") + } + + var ( + positionInByte []byte + err error + ) + if positionInByte, err = json.Marshal(*i.currentLogPosition); err != nil { + i.logger.Errorf("Failed to marshal binlog position: %v", err) + return err + } + + var cErr error + if err := i.res.AccessCache(ctx, i.binLogCache, func(c service.Cache) { + cErr = c.Set(ctx, binLogCacheKey, positionInByte, nil) + if cErr != nil { + i.logger.Errorf("Failed to store binlog position: %v", cErr) + } + }); err != nil { + fmt.Println("Access cache error", err) + return err + } + + return cErr +} + func (i *mysqlStreamInput) ReadBatch(ctx context.Context) (service.MessageBatch, service.AckFunc, error) { - i.cMut.Lock() + i.mutex.Lock() msgChan := i.msgChan - i.cMut.Unlock() + i.mutex.Unlock() if msgChan == nil { return nil, nil, service.ErrNotConnected } - time.Sleep(time.Second * 10) - select { case m, open := <-msgChan: if !open { @@ -294,6 +407,7 @@ func (i *mysqlStreamInput) ReadBatch(ctx context.Context) (service.MessageBatch, } func (i *mysqlStreamInput) Close(ctx context.Context) error { + i.canal.SyncedPosition() i.canal.Close() return nil } @@ -302,7 +416,12 @@ func (i *mysqlStreamInput) Close(ctx context.Context) error { // --- MySQL Canal handler methods ---- -func (i *mysqlStreamInput) OnRotate(*replication.EventHeader, *replication.RotateEvent) error { +func (i *mysqlStreamInput) OnRotate(eh *replication.EventHeader, re *replication.RotateEvent) error { + i.mutex.Lock() + i.currentLogPosition.Pos = uint32(re.Position) + i.currentLogPosition.Name = string(re.NextLogName) + i.mutex.Unlock() + return nil } func (i *mysqlStreamInput) OnTableChanged(*replication.EventHeader, string, string) error { @@ -330,7 +449,13 @@ func (i *mysqlStreamInput) OnXID(*replication.EventHeader, mysqlReplications.Pos func (i *mysqlStreamInput) OnGTID(*replication.EventHeader, mysqlReplications.BinlogGTIDEvent) error { return nil } -func (i *mysqlStreamInput) OnPosSynced(*replication.EventHeader, mysqlReplications.Position, mysqlReplications.GTIDSet, bool) error { +func (i *mysqlStreamInput) OnPosSynced(eh *replication.EventHeader, pos mysqlReplications.Position, gtid mysqlReplications.GTIDSet, synced bool) error { + i.mutex.Lock() + i.currentLogPosition = &pos + i.mutex.Unlock() + + i.syncBinlogPosition(context.Background()) + return nil } diff --git a/internal/impl/mysql/integration_test.go b/internal/impl/mysql/integration_test.go index 22041aa7f3..cd5c21fcba 100644 --- a/internal/impl/mysql/integration_test.go +++ b/internal/impl/mysql/integration_test.go @@ -11,7 +11,6 @@ package mysql import ( "context" "fmt" - "os" "sync" "testing" "time" @@ -24,25 +23,23 @@ import ( ) func TestIntegrationMySQLCDC(t *testing.T) { - mysqlDsn := os.Getenv("LOCAL_MYSQL_DSN") + mysqlDsn := "vultradmin:AVNS_fgGkgy43bJw3NzNwDCV@tcp(public-vultr-prod-a70dc516-1330-488a-bf57-712a3d91be58-vultr-pr.vultrdb.com:16751)/defaultdb" tmpDir := t.TempDir() - fmt.Println("TMP Ditr: ", tmpDir) template := fmt.Sprintf(` mysql_stream: dsn: %s stream_snapshot: false - checkpoint_key: binlogpos + checkpoint_key: foocache tables: - users - checkpoint_key: foocache flavor: mysql `, mysqlDsn) cacheConf := fmt.Sprintf(` -label: mysql_stream_cache +label: foocache file: - directory: %v`, tmpDir) + directory: %s`, tmpDir) streamOutBuilder := service.NewStreamBuilder() require.NoError(t, streamOutBuilder.SetLoggerYAML(`level: INFO`)) @@ -55,6 +52,7 @@ file: msgBytes, err := mb[0].AsBytes() require.NoError(t, err) outBatchMut.Lock() + fmt.Println(string(msgBytes)) outBatches = append(outBatches, string(msgBytes)) outBatchMut.Unlock() return nil @@ -76,3 +74,5 @@ file: require.NoError(t, streamOut.StopWithin(time.Second*10)) } + +// mysql://vultradmin:AVNS_fgGkgy43bJw3NzNwDCV@tcp(vultr-prod-a70dc516-1330-488a-bf57-712a3d91be58-vultr-prod-6745.vultrdb.com:16751)/defaultdb diff --git a/internal/impl/mysql/snapshot.go b/internal/impl/mysql/snapshot.go new file mode 100644 index 0000000000..4da929064f --- /dev/null +++ b/internal/impl/mysql/snapshot.go @@ -0,0 +1,3 @@ +package mysql + +type Snapshot struct{} From b0747e27f79e4bb0e5094a08533ddb4ff48bbe10 Mon Sep 17 00:00:00 2001 From: Vladyslav Len Date: Wed, 13 Nov 2024 21:50:16 +0100 Subject: [PATCH 05/45] chore(): added mysql tests --- internal/impl/mysql/input_mysql_stream.go | 40 +++---- internal/impl/mysql/integration_test.go | 125 ++++++++++++++++++++-- internal/impl/mysql/snapshot.go | 8 ++ 3 files changed, 146 insertions(+), 27 deletions(-) diff --git a/internal/impl/mysql/input_mysql_stream.go b/internal/impl/mysql/input_mysql_stream.go index 7f60af2545..b88dee1707 100644 --- a/internal/impl/mysql/input_mysql_stream.go +++ b/internal/impl/mysql/input_mysql_stream.go @@ -82,9 +82,13 @@ type mysqlStreamInput struct { batching service.BatchPolicy batchPolicy *service.Batcher checkPointLimit int + lastGtid *string logger *service.Logger res *service.Resources + + streaCtx context.Context + cp *checkpoint.Capped[*int64] } const binLogCacheKey = "mysql_binlog_position" @@ -95,6 +99,7 @@ func newMySQLStreamInput(conf *service.ParsedConfig, res *service.Resources) (s rawMessageEvents: make(chan MessageEvent), msgChan: make(chan asyncMessage), res: res, + streaCtx: context.Background(), } var batching service.BatchPolicy @@ -150,6 +155,7 @@ func newMySQLStreamInput(conf *service.ParsedConfig, res *service.Resources) (s } i := &streamInput + i.cp = checkpoint.NewCapped[*int64](int64(i.checkPointLimit)) res.Logger().Info("Starting MySQL stream input") @@ -217,12 +223,10 @@ func (i *mysqlStreamInput) Connect(ctx context.Context) error { func (i *mysqlStreamInput) readMessages(ctx context.Context) error { var nextTimedBatchChan <-chan time.Time var latestPos *mysqlReplications.Position - cp := checkpoint.NewCapped[*int64](int64(i.checkPointLimit)) for { select { case <-ctx.Done(): - fmt.Println("Stop reading....") return nil case <-nextTimedBatchChan: nextTimedBatchChan = nil @@ -232,13 +236,12 @@ func (i *mysqlStreamInput) readMessages(ctx context.Context) error { break } - if ok := i.flushBatch(ctx, cp, flushedBatch, latestPos); !ok { + if ok := i.flushBatch(ctx, i.cp, flushedBatch, latestPos); !ok { break } case me := <-i.rawMessageEvents: row, err := json.Marshal(me.Row) if err != nil { - fmt.Println("Error marshalling row: ", err) return err } @@ -257,7 +260,7 @@ func (i *mysqlStreamInput) readMessages(ctx context.Context) error { i.logger.Debugf("Flush batch error: %w", err) break } - if ok := i.flushBatch(ctx, cp, flushedBatch, latestPos); !ok { + if ok := i.flushBatch(ctx, i.cp, flushedBatch, latestPos); !ok { break } } else { @@ -418,18 +421,23 @@ func (i *mysqlStreamInput) Close(ctx context.Context) error { func (i *mysqlStreamInput) OnRotate(eh *replication.EventHeader, re *replication.RotateEvent) error { i.mutex.Lock() + flushedBatch, err := i.batchPolicy.Flush(i.streaCtx) + if err != nil { + i.logger.Debugf("Flush batch error: %w", err) + return err + } + + if ok := i.flushBatch(i.streaCtx, i.cp, flushedBatch, i.currentLogPosition); !ok { + return errors.New("failed to flush batch") + } + i.currentLogPosition.Pos = uint32(re.Position) i.currentLogPosition.Name = string(re.NextLogName) i.mutex.Unlock() return nil } -func (i *mysqlStreamInput) OnTableChanged(*replication.EventHeader, string, string) error { - return nil -} -func (i *mysqlStreamInput) OnDDL(*replication.EventHeader, mysqlReplications.Position, *replication.QueryEvent) error { - return nil -} + func (i *mysqlStreamInput) OnRow(e *canal.RowsEvent) error { switch e.Action { case canal.InsertAction: @@ -443,20 +451,12 @@ func (i *mysqlStreamInput) OnRow(e *canal.RowsEvent) error { } } -func (i *mysqlStreamInput) OnXID(*replication.EventHeader, mysqlReplications.Position) error { - return nil -} -func (i *mysqlStreamInput) OnGTID(*replication.EventHeader, mysqlReplications.BinlogGTIDEvent) error { - return nil -} func (i *mysqlStreamInput) OnPosSynced(eh *replication.EventHeader, pos mysqlReplications.Position, gtid mysqlReplications.GTIDSet, synced bool) error { i.mutex.Lock() i.currentLogPosition = &pos i.mutex.Unlock() - i.syncBinlogPosition(context.Background()) - - return nil + return i.syncBinlogPosition(context.Background()) } // --- MySQL Canal handler methods end ---- diff --git a/internal/impl/mysql/integration_test.go b/internal/impl/mysql/integration_test.go index cd5c21fcba..6828b8abd8 100644 --- a/internal/impl/mysql/integration_test.go +++ b/internal/impl/mysql/integration_test.go @@ -10,11 +10,15 @@ package mysql import ( "context" + "database/sql" "fmt" + "github.com/ory/dockertest/v3" + "github.com/ory/dockertest/v3/docker" "sync" "testing" "time" + _ "github.com/go-sql-driver/mysql" _ "github.com/redpanda-data/benthos/v4/public/components/io" _ "github.com/redpanda-data/benthos/v4/public/components/pure" "github.com/redpanda-data/benthos/v4/public/service" @@ -23,7 +27,72 @@ import ( ) func TestIntegrationMySQLCDC(t *testing.T) { - mysqlDsn := "vultradmin:AVNS_fgGkgy43bJw3NzNwDCV@tcp(public-vultr-prod-a70dc516-1330-488a-bf57-712a3d91be58-vultr-pr.vultrdb.com:16751)/defaultdb" + pool, err := dockertest.NewPool("") + require.NoError(t, err) + + pool.MaxWait = time.Minute + + // MySQL specific environment variables + resource, err := pool.RunWithOptions(&dockertest.RunOptions{ + Repository: "mysql", + Tag: "9.1", + Env: []string{ + "MYSQL_ROOT_PASSWORD=password", + "MYSQL_DATABASE=testdb", + }, + Cmd: []string{ + "--server-id=1", + "--log-bin=mysql-bin", + "--binlog-format=ROW", + "--binlog-row-image=FULL", + "--log-slave-updates=ON", + }, + ExposedPorts: []string{"3306/tcp"}, + }, func(config *docker.HostConfig) { + // set AutoRemove to true so that stopped container goes away by itself + config.AutoRemove = true + config.RestartPolicy = docker.RestartPolicy{ + Name: "no", + } + }) + require.NoError(t, err) + t.Cleanup(func() { + assert.NoError(t, pool.Purge(resource)) + }) + + port := resource.GetPort("3306/tcp") + dsn := fmt.Sprintf( + "root:password@tcp(localhost:%s)/testdb?parseTime=true&timeout=30s&readTimeout=30s&writeTimeout=30s&multiStatements=true", + port, + ) + + var db *sql.DB + err = pool.Retry(func() error { + var err error + db, err = sql.Open("mysql", dsn) + if err != nil { + return err + } + + db.SetMaxOpenConns(10) + db.SetMaxIdleConns(5) + db.SetConnMaxLifetime(time.Minute * 5) + + if err != nil { + return err + } + + return db.Ping() + }) + require.NoError(t, err) + + // Create table + _, err = db.Exec(` + CREATE TABLE IF NOT EXISTS foo ( + a INT PRIMARY KEY + ) +`) + require.NoError(t, err) tmpDir := t.TempDir() template := fmt.Sprintf(` @@ -32,9 +101,9 @@ mysql_stream: stream_snapshot: false checkpoint_key: foocache tables: - - users + - foo flavor: mysql -`, mysqlDsn) +`, dsn) cacheConf := fmt.Sprintf(` label: foocache @@ -52,7 +121,6 @@ file: msgBytes, err := mb[0].AsBytes() require.NoError(t, err) outBatchMut.Lock() - fmt.Println(string(msgBytes)) outBatches = append(outBatches, string(msgBytes)) outBatchMut.Unlock() return nil @@ -66,13 +134,56 @@ file: require.NoError(t, err) }() + time.Sleep(time.Second * 5) + for i := 0; i < 1000; i++ { + // Insert 10000 rows + if _, err = db.Exec("INSERT INTO foo VALUES (?)", i); err != nil { + require.NoError(t, err) + } + } + assert.Eventually(t, func() bool { outBatchMut.Lock() defer outBatchMut.Unlock() - return len(outBatches) == 10000 + return len(outBatches) == 1000 }, time.Minute*5, time.Millisecond*100) require.NoError(t, streamOut.StopWithin(time.Second*10)) -} -// mysql://vultradmin:AVNS_fgGkgy43bJw3NzNwDCV@tcp(vultr-prod-a70dc516-1330-488a-bf57-712a3d91be58-vultr-prod-6745.vultrdb.com:16751)/defaultdb + streamOutBuilder = service.NewStreamBuilder() + require.NoError(t, streamOutBuilder.SetLoggerYAML(`level: INFO`)) + require.NoError(t, streamOutBuilder.AddCacheYAML(cacheConf)) + require.NoError(t, streamOutBuilder.AddInputYAML(template)) + + outBatches = nil + require.NoError(t, streamOutBuilder.AddBatchConsumerFunc(func(c context.Context, mb service.MessageBatch) error { + msgBytes, err := mb[0].AsBytes() + require.NoError(t, err) + outBatchMut.Lock() + outBatches = append(outBatches, string(msgBytes)) + outBatchMut.Unlock() + return nil + })) + + streamOut, err = streamOutBuilder.Build() + require.NoError(t, err) + + time.Sleep(time.Second) + for i := 1001; i < 2001; i++ { + _, err = db.Exec("INSERT INTO foo VALUES (?)", i) + require.NoError(t, err) + } + + go func() { + err = streamOut.Run(context.Background()) + require.NoError(t, err) + }() + + assert.Eventually(t, func() bool { + outBatchMut.Lock() + defer outBatchMut.Unlock() + return len(outBatches) == 1000 + }, time.Minute*5, time.Millisecond*100) + + require.NoError(t, streamOut.StopWithin(time.Second*10)) +} diff --git a/internal/impl/mysql/snapshot.go b/internal/impl/mysql/snapshot.go index 4da929064f..51266ad773 100644 --- a/internal/impl/mysql/snapshot.go +++ b/internal/impl/mysql/snapshot.go @@ -1,3 +1,11 @@ +// Copyright 2024 Redpanda Data, Inc. +// +// Licensed as a Redpanda Enterprise file under the Redpanda Community +// License (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// https://github.com/redpanda-data/connect/v4/blob/main/licenses/rcl.md + package mysql type Snapshot struct{} From 77fbcc2950a030e608ea5fd910829cf77637ff76 Mon Sep 17 00:00:00 2001 From: Vladyslav Len Date: Thu, 14 Nov 2024 10:59:47 +0100 Subject: [PATCH 06/45] chore(): multi-version testing for mysql cdc --- internal/impl/mysql/integration_test.go | 253 ++++++++++++------------ 1 file changed, 128 insertions(+), 125 deletions(-) diff --git a/internal/impl/mysql/integration_test.go b/internal/impl/mysql/integration_test.go index 6828b8abd8..1350a28118 100644 --- a/internal/impl/mysql/integration_test.go +++ b/internal/impl/mysql/integration_test.go @@ -27,75 +27,77 @@ import ( ) func TestIntegrationMySQLCDC(t *testing.T) { - pool, err := dockertest.NewPool("") - require.NoError(t, err) - - pool.MaxWait = time.Minute - - // MySQL specific environment variables - resource, err := pool.RunWithOptions(&dockertest.RunOptions{ - Repository: "mysql", - Tag: "9.1", - Env: []string{ - "MYSQL_ROOT_PASSWORD=password", - "MYSQL_DATABASE=testdb", - }, - Cmd: []string{ - "--server-id=1", - "--log-bin=mysql-bin", - "--binlog-format=ROW", - "--binlog-row-image=FULL", - "--log-slave-updates=ON", - }, - ExposedPorts: []string{"3306/tcp"}, - }, func(config *docker.HostConfig) { - // set AutoRemove to true so that stopped container goes away by itself - config.AutoRemove = true - config.RestartPolicy = docker.RestartPolicy{ - Name: "no", - } - }) - require.NoError(t, err) - t.Cleanup(func() { - assert.NoError(t, pool.Purge(resource)) - }) - - port := resource.GetPort("3306/tcp") - dsn := fmt.Sprintf( - "root:password@tcp(localhost:%s)/testdb?parseTime=true&timeout=30s&readTimeout=30s&writeTimeout=30s&multiStatements=true", - port, - ) - - var db *sql.DB - err = pool.Retry(func() error { - var err error - db, err = sql.Open("mysql", dsn) - if err != nil { - return err - } - - db.SetMaxOpenConns(10) - db.SetMaxIdleConns(5) - db.SetConnMaxLifetime(time.Minute * 5) - - if err != nil { - return err - } + var mysqlTestVersions = []string{"8.0", "9.0", "9.1"} + for _, version := range mysqlTestVersions { + pool, err := dockertest.NewPool("") + require.NoError(t, err) - return db.Ping() - }) - require.NoError(t, err) + pool.MaxWait = time.Minute + + // MySQL specific environment variables + resource, err := pool.RunWithOptions(&dockertest.RunOptions{ + Repository: "mysql", + Tag: version, + Env: []string{ + "MYSQL_ROOT_PASSWORD=password", + "MYSQL_DATABASE=testdb", + }, + Cmd: []string{ + "--server-id=1", + "--log-bin=mysql-bin", + "--binlog-format=ROW", + "--binlog-row-image=FULL", + "--log-slave-updates=ON", + }, + ExposedPorts: []string{"3306/tcp"}, + }, func(config *docker.HostConfig) { + // set AutoRemove to true so that stopped container goes away by itself + config.AutoRemove = true + config.RestartPolicy = docker.RestartPolicy{ + Name: "no", + } + }) + require.NoError(t, err) + t.Cleanup(func() { + assert.NoError(t, pool.Purge(resource)) + }) + + port := resource.GetPort("3306/tcp") + dsn := fmt.Sprintf( + "root:password@tcp(localhost:%s)/testdb?parseTime=true&timeout=30s&readTimeout=30s&writeTimeout=30s&multiStatements=true", + port, + ) + + var db *sql.DB + err = pool.Retry(func() error { + var err error + db, err = sql.Open("mysql", dsn) + if err != nil { + return err + } + + db.SetMaxOpenConns(10) + db.SetMaxIdleConns(5) + db.SetConnMaxLifetime(time.Minute * 5) + + if err != nil { + return err + } + + return db.Ping() + }) + require.NoError(t, err) - // Create table - _, err = db.Exec(` + // Create table + _, err = db.Exec(` CREATE TABLE IF NOT EXISTS foo ( a INT PRIMARY KEY ) `) - require.NoError(t, err) - tmpDir := t.TempDir() + require.NoError(t, err) + tmpDir := t.TempDir() - template := fmt.Sprintf(` + template := fmt.Sprintf(` mysql_stream: dsn: %s stream_snapshot: false @@ -105,85 +107,86 @@ mysql_stream: flavor: mysql `, dsn) - cacheConf := fmt.Sprintf(` + cacheConf := fmt.Sprintf(` label: foocache file: directory: %s`, tmpDir) - streamOutBuilder := service.NewStreamBuilder() - require.NoError(t, streamOutBuilder.SetLoggerYAML(`level: INFO`)) - require.NoError(t, streamOutBuilder.AddCacheYAML(cacheConf)) - require.NoError(t, streamOutBuilder.AddInputYAML(template)) + streamOutBuilder := service.NewStreamBuilder() + require.NoError(t, streamOutBuilder.SetLoggerYAML(`level: INFO`)) + require.NoError(t, streamOutBuilder.AddCacheYAML(cacheConf)) + require.NoError(t, streamOutBuilder.AddInputYAML(template)) - var outBatches []string - var outBatchMut sync.Mutex - require.NoError(t, streamOutBuilder.AddBatchConsumerFunc(func(c context.Context, mb service.MessageBatch) error { - msgBytes, err := mb[0].AsBytes() - require.NoError(t, err) - outBatchMut.Lock() - outBatches = append(outBatches, string(msgBytes)) - outBatchMut.Unlock() - return nil - })) - - streamOut, err := streamOutBuilder.Build() - require.NoError(t, err) + var outBatches []string + var outBatchMut sync.Mutex + require.NoError(t, streamOutBuilder.AddBatchConsumerFunc(func(c context.Context, mb service.MessageBatch) error { + msgBytes, err := mb[0].AsBytes() + require.NoError(t, err) + outBatchMut.Lock() + outBatches = append(outBatches, string(msgBytes)) + outBatchMut.Unlock() + return nil + })) - go func() { - err = streamOut.Run(context.Background()) + streamOut, err := streamOutBuilder.Build() require.NoError(t, err) - }() - time.Sleep(time.Second * 5) - for i := 0; i < 1000; i++ { - // Insert 10000 rows - if _, err = db.Exec("INSERT INTO foo VALUES (?)", i); err != nil { + go func() { + err = streamOut.Run(context.Background()) require.NoError(t, err) + }() + + time.Sleep(time.Second * 5) + for i := 0; i < 1000; i++ { + // Insert 10000 rows + if _, err = db.Exec("INSERT INTO foo VALUES (?)", i); err != nil { + require.NoError(t, err) + } } - } - assert.Eventually(t, func() bool { - outBatchMut.Lock() - defer outBatchMut.Unlock() - return len(outBatches) == 1000 - }, time.Minute*5, time.Millisecond*100) + assert.Eventually(t, func() bool { + outBatchMut.Lock() + defer outBatchMut.Unlock() + return len(outBatches) == 1000 + }, time.Minute*5, time.Millisecond*100) - require.NoError(t, streamOut.StopWithin(time.Second*10)) + require.NoError(t, streamOut.StopWithin(time.Second*10)) - streamOutBuilder = service.NewStreamBuilder() - require.NoError(t, streamOutBuilder.SetLoggerYAML(`level: INFO`)) - require.NoError(t, streamOutBuilder.AddCacheYAML(cacheConf)) - require.NoError(t, streamOutBuilder.AddInputYAML(template)) + streamOutBuilder = service.NewStreamBuilder() + require.NoError(t, streamOutBuilder.SetLoggerYAML(`level: INFO`)) + require.NoError(t, streamOutBuilder.AddCacheYAML(cacheConf)) + require.NoError(t, streamOutBuilder.AddInputYAML(template)) - outBatches = nil - require.NoError(t, streamOutBuilder.AddBatchConsumerFunc(func(c context.Context, mb service.MessageBatch) error { - msgBytes, err := mb[0].AsBytes() - require.NoError(t, err) - outBatchMut.Lock() - outBatches = append(outBatches, string(msgBytes)) - outBatchMut.Unlock() - return nil - })) - - streamOut, err = streamOutBuilder.Build() - require.NoError(t, err) - - time.Sleep(time.Second) - for i := 1001; i < 2001; i++ { - _, err = db.Exec("INSERT INTO foo VALUES (?)", i) - require.NoError(t, err) - } + outBatches = nil + require.NoError(t, streamOutBuilder.AddBatchConsumerFunc(func(c context.Context, mb service.MessageBatch) error { + msgBytes, err := mb[0].AsBytes() + require.NoError(t, err) + outBatchMut.Lock() + outBatches = append(outBatches, string(msgBytes)) + outBatchMut.Unlock() + return nil + })) - go func() { - err = streamOut.Run(context.Background()) + streamOut, err = streamOutBuilder.Build() require.NoError(t, err) - }() - assert.Eventually(t, func() bool { - outBatchMut.Lock() - defer outBatchMut.Unlock() - return len(outBatches) == 1000 - }, time.Minute*5, time.Millisecond*100) + time.Sleep(time.Second) + for i := 1001; i < 2001; i++ { + _, err = db.Exec("INSERT INTO foo VALUES (?)", i) + require.NoError(t, err) + } + + go func() { + err = streamOut.Run(context.Background()) + require.NoError(t, err) + }() - require.NoError(t, streamOut.StopWithin(time.Second*10)) + assert.Eventually(t, func() bool { + outBatchMut.Lock() + defer outBatchMut.Unlock() + return len(outBatches) == 1000 + }, time.Minute*5, time.Millisecond*100) + + require.NoError(t, streamOut.StopWithin(time.Second*10)) + } } From 2e9b57c93758e81be57290f536e60713af0fa1a4 Mon Sep 17 00:00:00 2001 From: Vladyslav Len Date: Mon, 18 Nov 2024 02:57:09 +0100 Subject: [PATCH 07/45] chore(): implemented snapshot support --- internal/impl/mysql/event.go | 2 +- internal/impl/mysql/input_mysql_stream.go | 229 ++++++++++++++--- internal/impl/mysql/integration_test.go | 289 +++++++++++++++++++++- internal/impl/mysql/snapshot.go | 190 +++++++++++++- 4 files changed, 670 insertions(+), 40 deletions(-) diff --git a/internal/impl/mysql/event.go b/internal/impl/mysql/event.go index 2ad5dba70e..823b08e1f9 100644 --- a/internal/impl/mysql/event.go +++ b/internal/impl/mysql/event.go @@ -25,7 +25,7 @@ const ( type MessageType string const ( - MessageTypeEvent MessageType = "snapshot" + MessageTypeSnapshot MessageType = "snapshot" MessageTypeStreaming MessageType = "streaming" ) diff --git a/internal/impl/mysql/input_mysql_stream.go b/internal/impl/mysql/input_mysql_stream.go index b88dee1707..e557def023 100644 --- a/internal/impl/mysql/input_mysql_stream.go +++ b/internal/impl/mysql/input_mysql_stream.go @@ -10,6 +10,7 @@ package mysql import ( "context" + "database/sql" "encoding/json" "errors" "fmt" @@ -17,21 +18,27 @@ import ( "time" "github.com/Jeffail/checkpoint" + "github.com/Jeffail/shutdown" "github.com/go-mysql-org/go-mysql/canal" mysqlReplications "github.com/go-mysql-org/go-mysql/mysql" "github.com/go-mysql-org/go-mysql/replication" "github.com/go-sql-driver/mysql" "github.com/redpanda-data/benthos/v4/public/service" + "golang.org/x/sync/errgroup" ) const ( - fieldMySQLDSN = "dsn" - fieldMySQLTables = "tables" - fieldStreamSnapshot = "stream_snapshot" - fieldBatching = "batching" - fieldCheckpointKey = "checkpoint_key" - fieldCheckpointLimit = "checkpoint_limit" - fieldFlavor = "flavor" + fieldMySQLDSN = "dsn" + fieldMySQLTables = "tables" + fieldStreamSnapshot = "stream_snapshot" + fieldMaxSnapshotParallelTables = "max_snapshot_parallel_tables" + fieldSnapshotMaxBatchSize = "snapshot_max_batch_size" + fieldBatching = "batching" + fieldCheckpointKey = "checkpoint_key" + fieldCheckpointLimit = "checkpoint_limit" + fieldFlavor = "flavor" + + shutdownTimeout = 5 * time.Second ) var mysqlStreamConfigSpec = service.NewConfigSpec(). @@ -48,8 +55,14 @@ var mysqlStreamConfigSpec = service.NewConfigSpec(). service.NewStringField(fieldFlavor). Description("The flavor of MySQL to connect to."). Example("mysql"), + service.NewBoolField(fieldMaxSnapshotParallelTables). + Description("Int specifies a number of tables to be streamed in parallel when taking a snapshot. If set to true, the connector will stream all tables in parallel. Otherwise, it will stream tables one by one."). + Default(1), + service.NewIntField(fieldSnapshotMaxBatchSize). + Description("The maximum number of rows to be streamed in a single batch when taking a snapshot."). + Default(1000), service.NewBoolField(fieldStreamSnapshot). - Description("If set to true, the connector will perform a backup to get a snapshot of the database. Otherwise, it will start from the current binlog position."), + Description("If set to true, the connector will query all the existing data as a part of snapshot procerss. Otherwise, it will start from the current binlog position."), service.NewAutoRetryNacksToggleField(), service.NewIntField(fieldCheckpointLimit). Description("The maximum number of messages that can be processed at a given time. Increasing this limit enables parallel processing and batching at the output level. Any given LSN will not be acknowledged unless all messages under that offset are delivered in order to preserve at least once delivery guarantees."). @@ -77,29 +90,40 @@ type mysqlStreamInput struct { flavor string streamSnapshot bool - rawMessageEvents chan MessageEvent - msgChan chan asyncMessage - batching service.BatchPolicy - batchPolicy *service.Batcher - checkPointLimit int - lastGtid *string + rawMessageEvents chan MessageEvent + snapshotMessageEvents chan MessageEvent + msgChan chan asyncMessage + batching service.BatchPolicy + batchPolicy *service.Batcher + checkPointLimit int + lastGtid *string logger *service.Logger res *service.Resources - streaCtx context.Context - cp *checkpoint.Capped[*int64] + streamCtx context.Context + errors chan error + cp *checkpoint.Capped[*int64] + + snapshot *Snapshot + shutSig *shutdown.Signaller + snapshotMaxParallelTables int + fieldSnapshotMaxBatchSize int } const binLogCacheKey = "mysql_binlog_position" func newMySQLStreamInput(conf *service.ParsedConfig, res *service.Resources) (s service.BatchInput, err error) { streamInput := mysqlStreamInput{ - logger: res.Logger(), - rawMessageEvents: make(chan MessageEvent), - msgChan: make(chan asyncMessage), - res: res, - streaCtx: context.Background(), + logger: res.Logger(), + rawMessageEvents: make(chan MessageEvent), + snapshotMessageEvents: make(chan MessageEvent), + msgChan: make(chan asyncMessage), + res: res, + streamCtx: context.Background(), + + errors: make(chan error, 1), + shutSig: shutdown.NewSignaller(), } var batching service.BatchPolicy @@ -125,6 +149,14 @@ func newMySQLStreamInput(conf *service.ParsedConfig, res *service.Resources) (s return nil, err } + if streamInput.snapshotMaxParallelTables, err = conf.FieldInt(fieldMaxSnapshotParallelTables); err != nil { + return nil, err + } + + if streamInput.fieldSnapshotMaxBatchSize, err = conf.FieldInt(fieldSnapshotMaxBatchSize); err != nil { + return nil, err + } + if streamInput.checkPointLimit, err = conf.FieldInt(fieldCheckpointLimit); err != nil { return nil, err } @@ -216,18 +248,28 @@ func (i *mysqlStreamInput) Connect(ctx context.Context) error { c.AddDumpTables(i.mysqlConfig.DBName, i.tables...) i.canal = c + + db, err := sql.Open("mysql", i.dsn) + if err != nil { + return err + } + + // create snapshot instance + snapshot := NewSnapshot(ctx, i.logger, db) + i.snapshot = snapshot + go i.startMySQLSync(ctx) return nil } -func (i *mysqlStreamInput) readMessages(ctx context.Context) error { +func (i *mysqlStreamInput) readMessages(ctx context.Context) { var nextTimedBatchChan <-chan time.Time var latestPos *mysqlReplications.Position - for { + for !i.shutSig.IsHasStoppedSignalled() { select { case <-ctx.Done(): - return nil + return case <-nextTimedBatchChan: nextTimedBatchChan = nil flushedBatch, err := i.batchPolicy.Flush(ctx) @@ -242,7 +284,7 @@ func (i *mysqlStreamInput) readMessages(ctx context.Context) error { case me := <-i.rawMessageEvents: row, err := json.Marshal(me.Row) if err != nil { - return err + return } mb := service.NewMessage(row) @@ -269,6 +311,10 @@ func (i *mysqlStreamInput) readMessages(ctx context.Context) error { nextTimedBatchChan = time.After(d) } } + case err := <-i.errors: + i.logger.Warnf("stream error: %s", err) + // If the stream has errored then we should stop and restart processing + return } } } @@ -280,8 +326,103 @@ func (i *mysqlStreamInput) startMySQLSync(ctx context.Context) { // initiate default run for Canal to process snapshot and start incremental sync of binlog if i.streamSnapshot && i.startBinLogPosition == nil { // Doesn't work at the moment - if err := i.canal.Run(); err != nil { - panic(err) + startPos, err := i.snapshot.prepareSnapshot(ctx) + if err != nil { + i.errors <- err + return + } + + defer i.snapshot.releaseSnapshot(ctx) + i.logger.Debugf("Starting snapshot while holding binglog pos on: %v", startPos) + var wg errgroup.Group + wg.SetLimit(i.snapshotMaxParallelTables) + + for _, table := range i.tables { + wg.Go(func() (err error) { + rowsCount, err := i.snapshot.getRowsCount(table) + if err != nil { + return err + } + + i.logger.Debugf("Rows count for table %s is %d", table, rowsCount) + + tablePks, err := i.snapshot.getTablePrimaryKeys(table) + if err != nil { + return err + } + + i.logger.Debugf("Primary keys for table %s %v", table, tablePks) + + var numRowsProcessed int + lastSeenPksValues := map[string]any{} + for _, pk := range tablePks { + lastSeenPksValues[pk] = nil + } + + for numRowsProcessed < rowsCount { + var batchRows *sql.Rows + if numRowsProcessed == 0 { + batchRows, err = i.snapshot.querySnapshotTable(table, tablePks, nil, i.fieldSnapshotMaxBatchSize) + if err != nil { + return err + } + } else { + batchRows, err = i.snapshot.querySnapshotTable(table, tablePks, &lastSeenPksValues, i.fieldSnapshotMaxBatchSize) + if err != nil { + return err + } + } + + for batchRows.Next() { + numRowsProcessed++ + + columns, err := batchRows.Columns() + if err != nil { + batchRows.Close() + return err + } + + values := make([]interface{}, len(columns)) + valuePtrs := make([]interface{}, len(columns)) + for i := range values { + valuePtrs[i] = &values[i] + } + + if err := batchRows.Scan(valuePtrs...); err != nil { + batchRows.Close() + return err + } + + row := map[string]any{} + for idx, value := range values { + row[columns[idx]] = value + if _, ok := lastSeenPksValues[columns[idx]]; ok { + lastSeenPksValues[columns[idx]] = value + } + } + + // build message + i.rawMessageEvents <- MessageEvent{ + Row: row, + Operation: MessageOperationInsert, + Type: MessageTypeSnapshot, + Table: table, + Position: nil, + } + } + } + + return nil + }) + } + + if err = wg.Wait(); err != nil { + i.errors <- fmt.Errorf("snapshot processing failed: %w", err) + } + + i.logger.Infof("Snapshot is done...Running CDC from BingLog: %s on pos: %d", startPos.Name, startPos.Pos) + if err := i.canal.RunFrom(*startPos); err != nil { + i.errors <- fmt.Errorf("failed to start streaming: %v", err) } } else { coords, _ := i.canal.GetMasterPos() @@ -292,18 +433,23 @@ func (i *mysqlStreamInput) startMySQLSync(ctx context.Context) { i.currentLogPosition = &coords if err := i.canal.RunFrom(coords); err != nil { - panic(err) + i.errors <- fmt.Errorf("failed to start streaming: %v", err) } } } -func (i *mysqlStreamInput) flushBatch(ctx context.Context, checkpointer *checkpoint.Capped[*int64], msg service.MessageBatch, binLogPos *mysqlReplications.Position) bool { - if msg == nil { +func (i *mysqlStreamInput) flushBatch(ctx context.Context, checkpointer *checkpoint.Capped[*int64], batch service.MessageBatch, binLogPos *mysqlReplications.Position) bool { + if batch == nil { return true } - posInInt := int64(binLogPos.Pos) - resolveFn, err := checkpointer.Track(ctx, &posInInt, int64(len(msg))) + var intPos *int64 + if binLogPos != nil { + posInInt := int64(binLogPos.Pos) + intPos = &posInInt + } + + resolveFn, err := checkpointer.Track(ctx, intPos, int64(len(batch))) if err != nil { if ctx.Err() == nil { i.logger.Errorf("Failed to checkpoint offset: %v\n", err) @@ -311,15 +457,21 @@ func (i *mysqlStreamInput) flushBatch(ctx context.Context, checkpointer *checkpo return false } + lastMsg := batch[len(batch)-1] + select { case i.msgChan <- asyncMessage{ - msg: msg, + msg: batch, ackFn: func(ctx context.Context, res error) error { maxOffset := resolveFn() if maxOffset == nil { return nil } + if msgType, ok := lastMsg.MetaGet("type"); ok && msgType == "snapshot" { + return nil + } + if err := i.syncBinlogPosition(context.Background()); err != nil { return err } @@ -382,7 +534,7 @@ func (i *mysqlStreamInput) syncBinlogPosition(ctx context.Context) error { i.logger.Errorf("Failed to store binlog position: %v", cErr) } }); err != nil { - fmt.Println("Access cache error", err) + i.logger.Errorf("Access cache error %v", err) return err } @@ -403,6 +555,8 @@ func (i *mysqlStreamInput) ReadBatch(ctx context.Context) (service.MessageBatch, return nil, nil, service.ErrNotConnected } return m.msg, m.ackFn, nil + case <-i.shutSig.HasStoppedChan(): + return nil, nil, service.ErrNotConnected case <-ctx.Done(): } @@ -410,8 +564,11 @@ func (i *mysqlStreamInput) ReadBatch(ctx context.Context) (service.MessageBatch, } func (i *mysqlStreamInput) Close(ctx context.Context) error { + fmt.Println("Close has been called") + i.shutSig.TriggerHardStop() i.canal.SyncedPosition() i.canal.Close() + i.snapshot.releaseSnapshot(ctx) return nil } @@ -421,13 +578,13 @@ func (i *mysqlStreamInput) Close(ctx context.Context) error { func (i *mysqlStreamInput) OnRotate(eh *replication.EventHeader, re *replication.RotateEvent) error { i.mutex.Lock() - flushedBatch, err := i.batchPolicy.Flush(i.streaCtx) + flushedBatch, err := i.batchPolicy.Flush(i.streamCtx) if err != nil { i.logger.Debugf("Flush batch error: %w", err) return err } - if ok := i.flushBatch(i.streaCtx, i.cp, flushedBatch, i.currentLogPosition); !ok { + if ok := i.flushBatch(i.streamCtx, i.cp, flushedBatch, i.currentLogPosition); !ok { return errors.New("failed to flush batch") } diff --git a/internal/impl/mysql/integration_test.go b/internal/impl/mysql/integration_test.go index 1350a28118..065a7e502f 100644 --- a/internal/impl/mysql/integration_test.go +++ b/internal/impl/mysql/integration_test.go @@ -12,12 +12,13 @@ import ( "context" "database/sql" "fmt" - "github.com/ory/dockertest/v3" - "github.com/ory/dockertest/v3/docker" "sync" "testing" "time" + "github.com/ory/dockertest/v3" + "github.com/ory/dockertest/v3/docker" + _ "github.com/go-sql-driver/mysql" _ "github.com/redpanda-data/benthos/v4/public/components/io" _ "github.com/redpanda-data/benthos/v4/public/components/pure" @@ -190,3 +191,287 @@ file: require.NoError(t, streamOut.StopWithin(time.Second*10)) } } + +func TestIntegrationMySQLSnapshotAndCDC(t *testing.T) { + pool, err := dockertest.NewPool("") + require.NoError(t, err) + + pool.MaxWait = time.Minute + + // MySQL specific environment variables + resource, err := pool.RunWithOptions(&dockertest.RunOptions{ + Repository: "mysql", + Tag: "8.0", + Env: []string{ + "MYSQL_ROOT_PASSWORD=password", + "MYSQL_DATABASE=testdb", + }, + Cmd: []string{ + "--server-id=1", + "--log-bin=mysql-bin", + "--binlog-format=ROW", + "--binlog-row-image=FULL", + "--log-slave-updates=ON", + }, + ExposedPorts: []string{"3306/tcp"}, + }, func(config *docker.HostConfig) { + // set AutoRemove to true so that stopped container goes away by itself + config.AutoRemove = true + config.RestartPolicy = docker.RestartPolicy{ + Name: "no", + } + }) + require.NoError(t, err) + t.Cleanup(func() { + assert.NoError(t, pool.Purge(resource)) + }) + + port := resource.GetPort("3306/tcp") + dsn := fmt.Sprintf( + "root:password@tcp(localhost:%s)/testdb?parseTime=true&timeout=30s&readTimeout=30s&writeTimeout=30s&multiStatements=true", + port, + ) + + var db *sql.DB + err = pool.Retry(func() error { + var err error + db, err = sql.Open("mysql", dsn) + if err != nil { + return err + } + + db.SetMaxOpenConns(10) + db.SetMaxIdleConns(5) + db.SetConnMaxLifetime(time.Minute * 5) + + if err != nil { + return err + } + + return db.Ping() + }) + require.NoError(t, err) + + // Create table + _, err = db.Exec(` + CREATE TABLE IF NOT EXISTS foo ( + a INT PRIMARY KEY + ) +`) + require.NoError(t, err) + tmpDir := t.TempDir() + + // Insert 1000 rows for initial snapshot streaming + for i := 0; i < 1000; i++ { + _, err = db.Exec("INSERT INTO foo VALUES (?)", i) + require.NoError(t, err) + } + + template := fmt.Sprintf(` +mysql_stream: + dsn: %s + stream_snapshot: true + snapshot_max_batch_size: 500 + checkpoint_key: foocache + tables: + - foo + flavor: mysql +`, dsn) + + cacheConf := fmt.Sprintf(` +label: foocache +file: + directory: %s`, tmpDir) + + streamOutBuilder := service.NewStreamBuilder() + require.NoError(t, streamOutBuilder.SetLoggerYAML(`level: DEBUG`)) + require.NoError(t, streamOutBuilder.AddCacheYAML(cacheConf)) + require.NoError(t, streamOutBuilder.AddInputYAML(template)) + + var outBatches []string + var outBatchMut sync.Mutex + require.NoError(t, streamOutBuilder.AddBatchConsumerFunc(func(c context.Context, mb service.MessageBatch) error { + msgBytes, err := mb[0].AsBytes() + fmt.Println(string(msgBytes)) + require.NoError(t, err) + outBatchMut.Lock() + outBatches = append(outBatches, string(msgBytes)) + outBatchMut.Unlock() + return nil + })) + + streamOut, err := streamOutBuilder.Build() + require.NoError(t, err) + + go func() { + err = streamOut.Run(context.Background()) + require.NoError(t, err) + }() + + time.Sleep(time.Second * 5) + for i := 1000; i < 2000; i++ { + // Insert 10000 rows + if _, err = db.Exec("INSERT INTO foo VALUES (?)", i); err != nil { + require.NoError(t, err) + } + } + + assert.Eventually(t, func() bool { + outBatchMut.Lock() + defer outBatchMut.Unlock() + return len(outBatches) == 2000 + }, time.Minute*5, time.Millisecond*100) + + require.NoError(t, streamOut.StopWithin(time.Second*10)) +} + +func TestIntegrationMySQLCDCWithCompositePrimaryKeys(t *testing.T) { + pool, err := dockertest.NewPool("") + require.NoError(t, err) + + pool.MaxWait = time.Minute + + // MySQL specific environment variables + resource, err := pool.RunWithOptions(&dockertest.RunOptions{ + Repository: "mysql", + Tag: "8.0", + Env: []string{ + "MYSQL_ROOT_PASSWORD=password", + "MYSQL_DATABASE=testdb", + }, + Cmd: []string{ + "--server-id=1", + "--log-bin=mysql-bin", + "--binlog-format=ROW", + "--binlog-row-image=FULL", + "--log-slave-updates=ON", + }, + ExposedPorts: []string{"3306/tcp"}, + }, func(config *docker.HostConfig) { + // set AutoRemove to true so that stopped container goes away by itself + config.AutoRemove = true + config.RestartPolicy = docker.RestartPolicy{ + Name: "no", + } + }) + require.NoError(t, err) + t.Cleanup(func() { + assert.NoError(t, pool.Purge(resource)) + }) + + port := resource.GetPort("3306/tcp") + dsn := fmt.Sprintf( + "root:password@tcp(localhost:%s)/testdb?parseTime=true&timeout=30s&readTimeout=30s&writeTimeout=30s&multiStatements=true", + port, + ) + + var db *sql.DB + err = pool.Retry(func() error { + var err error + db, err = sql.Open("mysql", dsn) + if err != nil { + return err + } + + db.SetMaxOpenConns(10) + db.SetMaxIdleConns(5) + db.SetConnMaxLifetime(time.Minute * 5) + + if err != nil { + return err + } + + return db.Ping() + }) + require.NoError(t, err) + + // Create table + _, err = db.Exec(` + CREATE TABLE IF NOT EXISTS foo ( + a INT, + b INT, + PRIMARY KEY (a, b) + ) +`) + + // Create controll table to ensure we don't stream it + _, err = db.Exec(` + CREATE TABLE IF NOT EXISTS foo_non_streamed ( + a INT, + b INT, + PRIMARY KEY (a, b) + ) +`) + + require.NoError(t, err) + tmpDir := t.TempDir() + + // Insert 1000 rows for initial snapshot streaming + for i := 0; i < 1000; i++ { + _, err = db.Exec("INSERT INTO foo VALUES (?, ?)", i, i) + require.NoError(t, err) + + _, err = db.Exec("INSERT INTO foo_non_streamed VALUES (?, ?)", i, i) + require.NoError(t, err) + } + + template := fmt.Sprintf(` +mysql_stream: + dsn: %s + stream_snapshot: true + snapshot_max_batch_size: 500 + checkpoint_key: foocache + tables: + - foo + flavor: mysql +`, dsn) + + cacheConf := fmt.Sprintf(` +label: foocache +file: + directory: %s`, tmpDir) + + streamOutBuilder := service.NewStreamBuilder() + require.NoError(t, streamOutBuilder.SetLoggerYAML(`level: DEBUG`)) + require.NoError(t, streamOutBuilder.AddCacheYAML(cacheConf)) + require.NoError(t, streamOutBuilder.AddInputYAML(template)) + + var outBatches []string + var outBatchMut sync.Mutex + require.NoError(t, streamOutBuilder.AddBatchConsumerFunc(func(c context.Context, mb service.MessageBatch) error { + msgBytes, err := mb[0].AsBytes() + fmt.Println(string(msgBytes)) + require.NoError(t, err) + outBatchMut.Lock() + outBatches = append(outBatches, string(msgBytes)) + outBatchMut.Unlock() + return nil + })) + + streamOut, err := streamOutBuilder.Build() + require.NoError(t, err) + + go func() { + err = streamOut.Run(context.Background()) + require.NoError(t, err) + }() + + time.Sleep(time.Second * 5) + for i := 1000; i < 2000; i++ { + // Insert 10000 rows + if _, err = db.Exec("INSERT INTO foo VALUES (?, ?)", i, i); err != nil { + require.NoError(t, err) + } + + _, err = db.Exec("INSERT INTO foo_non_streamed VALUES (?, ?)", i, i) + require.NoError(t, err) + } + + assert.Eventually(t, func() bool { + outBatchMut.Lock() + defer outBatchMut.Unlock() + return len(outBatches) == 2000 + }, time.Minute*5, time.Millisecond*100) + + require.NoError(t, streamOut.StopWithin(time.Second*10)) +} diff --git a/internal/impl/mysql/snapshot.go b/internal/impl/mysql/snapshot.go index 51266ad773..fe2bde93fe 100644 --- a/internal/impl/mysql/snapshot.go +++ b/internal/impl/mysql/snapshot.go @@ -8,4 +8,192 @@ package mysql -type Snapshot struct{} +import ( + "context" + "database/sql" + "fmt" + "strings" + + "github.com/go-mysql-org/go-mysql/mysql" + "github.com/redpanda-data/benthos/v4/public/service" +) + +type Snapshot struct { + db *sql.DB + tx *sql.Tx + + lockConn *sql.Conn + snapshotConn *sql.Conn + + logger *service.Logger + ctx context.Context +} + +func NewSnapshot(ctx context.Context, logger *service.Logger, db *sql.DB) *Snapshot { + return &Snapshot{ + db: db, + ctx: ctx, + logger: logger, + } +} + +func (s *Snapshot) prepareSnapshot(ctx context.Context) (*mysql.Position, error) { + var err error + // Create a separate connection for FTWRL + s.lockConn, err = s.db.Conn(ctx) + if err != nil { + return nil, fmt.Errorf("failed to create lock connection: %v", err) + } + + // Create another connection for the snapshot + s.snapshotConn, err = s.db.Conn(ctx) + if err != nil { + return nil, fmt.Errorf("failed to create snapshot connection: %v", err) + } + + // 1. Start a consistent snapshot transaction + s.tx, err = s.snapshotConn.BeginTx(ctx, &sql.TxOptions{ + Isolation: sql.LevelRepeatableRead, + }) + if err != nil { + return nil, fmt.Errorf("failed to start transaction: %v", err) + } + + // Execute START TRANSACTION WITH CONSISTENT SNAPSHOT + if _, err := s.tx.ExecContext(ctx, "START TRANSACTION WITH CONSISTENT SNAPSHOT"); err != nil { + s.tx.Rollback() + return nil, fmt.Errorf("failed to start consistent snapshot: %v", err) + } + + // 2. Acquire global read lock (minimizing lock time) + if _, err := s.lockConn.ExecContext(ctx, "FLUSH TABLES WITH READ LOCK"); err != nil { + s.tx.Rollback() + return nil, fmt.Errorf("failed to acquire global read lock: %v", err) + } + + // 3. Get binary log position (while locked) + pos, err := s.getCurrentBinlogPosition() + if err != nil { + // Make sure to release the lock if we fail + s.lockConn.ExecContext(ctx, "UNLOCK TABLES") + s.tx.Rollback() + return nil, fmt.Errorf("failed to get binlog position: %v", err) + } + + // 4. Release the global read lock immediately + if _, err := s.lockConn.ExecContext(ctx, "UNLOCK TABLES"); err != nil { + s.tx.Rollback() + return nil, fmt.Errorf("failed to release global read lock: %v", err) + } + + return &pos, nil +} + +func (s *Snapshot) getRowsCount(table string) (int, error) { + var count int + if err := s.tx.QueryRowContext(s.ctx, fmt.Sprintf("SELECT COUNT(*) FROM %s", table)).Scan(&count); err != nil { + return 0, fmt.Errorf("failed to get row count: %v", err) + } + return count, nil +} + +func (s *Snapshot) getTablePrimaryKeys(table string) ([]string, error) { + // Get primary key columns for the table + rows, err := s.tx.QueryContext(s.ctx, fmt.Sprintf(` +SELECT COLUMN_NAME +FROM information_schema.KEY_COLUMN_USAGE +WHERE TABLE_NAME = '%s' AND CONSTRAINT_NAME = 'PRIMARY'; + `, table)) + if err != nil { + return nil, fmt.Errorf("failed to get primary key: %v", err) + } + + defer rows.Close() + + var pks []string + for rows.Next() { + var pk string + if err := rows.Scan(&pk); err != nil { + return nil, err + } + + pks = append(pks, pk) + } + + return pks, nil +} + +func (s *Snapshot) querySnapshotTable(table string, pk []string, lastSeenPkVal *map[string]any, limit int) (*sql.Rows, error) { + snapshotQueryParts := []string{ + fmt.Sprintf("SELECT * FROM %s ", table), + } + + if lastSeenPkVal == nil { + snapshotQueryParts = append(snapshotQueryParts, s.buildOrderByClause(pk)) + + snapshotQueryParts = append(snapshotQueryParts, "LIMIT ?") + q := strings.Join(snapshotQueryParts, " ") + s.logger.Infof("Querying snapshot: %s", q) + return s.tx.QueryContext(s.ctx, strings.Join(snapshotQueryParts, " "), limit) + } + + var lastSeenPkVals []any + var placeholders []string + for _, pkCol := range *lastSeenPkVal { + lastSeenPkVals = append(lastSeenPkVals, pkCol) + placeholders = append(placeholders, "?") + } + + snapshotQueryParts = append(snapshotQueryParts, fmt.Sprintf("WHERE (%s) > (%s)", strings.Join(pk, ", "), strings.Join(placeholders, ", "))) + snapshotQueryParts = append(snapshotQueryParts, s.buildOrderByClause(pk)) + snapshotQueryParts = append(snapshotQueryParts, fmt.Sprintf("LIMIT %d", limit)) + q := strings.Join(snapshotQueryParts, " ") + s.logger.Infof("Querying snapshot: %s", q) + return s.tx.QueryContext(s.ctx, q, lastSeenPkVals...) +} + +func (s *Snapshot) buildOrderByClause(pk []string) string { + if len(pk) == 1 { + return fmt.Sprintf("ORDER BY %s", pk[0]) + } + + return fmt.Sprintf("ORDER BY %s", strings.Join(pk, ", ")) +} + +func (s *Snapshot) getCurrentBinlogPosition() (mysql.Position, error) { + var ( + position uint32 + file string + binlogDoDb interface{} + binlogIgnoreDb interface{} + executedGtidSet interface{} + ) + + row := s.snapshotConn.QueryRowContext(context.Background(), "SHOW MASTER STATUS") + if err := row.Scan(&file, &position, &binlogDoDb, &binlogIgnoreDb, &executedGtidSet); err != nil { + return mysql.Position{}, err + } + + return mysql.Position{ + Name: file, + Pos: position, + }, nil +} + +func (s *Snapshot) releaseSnapshot(ctx context.Context) error { + if s.tx != nil { + if err := s.tx.Commit(); err != nil { + return fmt.Errorf("failed to commit transaction: %v", err) + } + } + + if s.lockConn != nil { + s.lockConn.Close() + } + + if s.snapshotConn != nil { + s.snapshotConn.Close() + } + + return nil +} From 901b4feaaf73b315ab35f302cc65f51025923642 Mon Sep 17 00:00:00 2001 From: Vladyslav Len Date: Tue, 19 Nov 2024 19:11:18 +0100 Subject: [PATCH 08/45] chore(): reset dumper config --- internal/impl/mysql/input_mysql_stream.go | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/internal/impl/mysql/input_mysql_stream.go b/internal/impl/mysql/input_mysql_stream.go index e557def023..32813f7b34 100644 --- a/internal/impl/mysql/input_mysql_stream.go +++ b/internal/impl/mysql/input_mysql_stream.go @@ -232,6 +232,9 @@ func (i *mysqlStreamInput) Connect(ctx context.Context) error { canalConfig.User = i.mysqlConfig.User canalConfig.Password = i.mysqlConfig.Passwd canalConfig.Dump.TableDB = i.mysqlConfig.DBName + // resetting dump path since we are doing snapshot manually + // this is required since canal will try to prepare dumper on init stage + canalConfig.Dump.ExecutionPath = "" // Parse and set additional parameters canalConfig.Charset = i.mysqlConfig.Collation @@ -566,8 +569,11 @@ func (i *mysqlStreamInput) ReadBatch(ctx context.Context) (service.MessageBatch, func (i *mysqlStreamInput) Close(ctx context.Context) error { fmt.Println("Close has been called") i.shutSig.TriggerHardStop() - i.canal.SyncedPosition() - i.canal.Close() + if i.canal != nil { + i.canal.SyncedPosition() + i.canal.Close() + } + i.snapshot.releaseSnapshot(ctx) return nil } From f275aa7de9ff06e8bf82ee13bf9f073daa90d6be Mon Sep 17 00:00:00 2001 From: Vladyslav Len Date: Tue, 19 Nov 2024 21:05:40 +0100 Subject: [PATCH 09/45] fix(): table filtering for mysql --- internal/impl/mysql/input_mysql_stream.go | 11 ++++++++++- internal/impl/mysql/integration_test.go | 2 -- 2 files changed, 10 insertions(+), 3 deletions(-) diff --git a/internal/impl/mysql/input_mysql_stream.go b/internal/impl/mysql/input_mysql_stream.go index 32813f7b34..8a031eafb1 100644 --- a/internal/impl/mysql/input_mysql_stream.go +++ b/internal/impl/mysql/input_mysql_stream.go @@ -95,6 +95,7 @@ type mysqlStreamInput struct { msgChan chan asyncMessage batching service.BatchPolicy batchPolicy *service.Batcher + tablesFilterMap map[string]bool checkPointLimit int lastGtid *string @@ -189,6 +190,11 @@ func newMySQLStreamInput(conf *service.ParsedConfig, res *service.Resources) (s i := &streamInput i.cp = checkpoint.NewCapped[*int64](int64(i.checkPointLimit)) + i.tablesFilterMap = map[string]bool{} + for _, table := range i.tables { + i.tablesFilterMap[table] = true + } + res.Logger().Info("Starting MySQL stream input") if batching, err = conf.FieldBatchPolicy(fieldBatching); err != nil { @@ -231,7 +237,6 @@ func (i *mysqlStreamInput) Connect(ctx context.Context) error { canalConfig.Addr = i.mysqlConfig.Addr canalConfig.User = i.mysqlConfig.User canalConfig.Password = i.mysqlConfig.Passwd - canalConfig.Dump.TableDB = i.mysqlConfig.DBName // resetting dump path since we are doing snapshot manually // this is required since canal will try to prepare dumper on init stage canalConfig.Dump.ExecutionPath = "" @@ -602,6 +607,10 @@ func (i *mysqlStreamInput) OnRotate(eh *replication.EventHeader, re *replication } func (i *mysqlStreamInput) OnRow(e *canal.RowsEvent) error { + if _, ok := i.tablesFilterMap[e.Table.Name]; !ok { + return nil + } + switch e.Action { case canal.InsertAction: return i.onMessage(e, ProcessEventParams{initValue: 0, incrementValue: 1}) diff --git a/internal/impl/mysql/integration_test.go b/internal/impl/mysql/integration_test.go index 065a7e502f..67c6469c0a 100644 --- a/internal/impl/mysql/integration_test.go +++ b/internal/impl/mysql/integration_test.go @@ -292,7 +292,6 @@ file: var outBatchMut sync.Mutex require.NoError(t, streamOutBuilder.AddBatchConsumerFunc(func(c context.Context, mb service.MessageBatch) error { msgBytes, err := mb[0].AsBytes() - fmt.Println(string(msgBytes)) require.NoError(t, err) outBatchMut.Lock() outBatches = append(outBatches, string(msgBytes)) @@ -440,7 +439,6 @@ file: var outBatchMut sync.Mutex require.NoError(t, streamOutBuilder.AddBatchConsumerFunc(func(c context.Context, mb service.MessageBatch) error { msgBytes, err := mb[0].AsBytes() - fmt.Println(string(msgBytes)) require.NoError(t, err) outBatchMut.Lock() outBatches = append(outBatches, string(msgBytes)) From 1927dbe79958aa3e6780f1d3439077ed0a7fc353 Mon Sep 17 00:00:00 2001 From: Vladyslav Len Date: Wed, 20 Nov 2024 16:12:44 +0100 Subject: [PATCH 10/45] fix(): golangci-lint --- .golangci.yml | 2 +- internal/impl/mysql/event.go | 14 ++++--- internal/impl/mysql/input_mysql_stream.go | 28 +++++++------- internal/impl/mysql/integration_test.go | 13 +------ internal/impl/mysql/message.go | 16 -------- internal/impl/mysql/snapshot.go | 45 ++++++++++++++++------- internal/impl/timeplus/driver/driver.go | 1 - 7 files changed, 55 insertions(+), 64 deletions(-) delete mode 100644 internal/impl/mysql/message.go diff --git a/.golangci.yml b/.golangci.yml index ca8d151ddf..cc208f6fa1 100644 --- a/.golangci.yml +++ b/.golangci.yml @@ -89,6 +89,6 @@ linters: - tenv - predeclared - mirror - - rowserrcheck + # - rowserrcheck - bodyclose - nolintlint diff --git a/internal/impl/mysql/event.go b/internal/impl/mysql/event.go index 823b08e1f9..949c6da3b6 100644 --- a/internal/impl/mysql/event.go +++ b/internal/impl/mysql/event.go @@ -10,25 +10,29 @@ package mysql import "github.com/go-mysql-org/go-mysql/mysql" -type ProcessEventParams struct { - initValue, incrementValue int -} - +// MessageOperation is a string type specifying message opration type MessageOperation string const ( + // MessageOperationInsert represents insert statement in mysql binlog MessageOperationInsert MessageOperation = "insert" + // MessageOperationUpdate represents update statement in mysql binlog MessageOperationUpdate MessageOperation = "update" + // MessageOperationDelete represents delete statement in mysql binlog MessageOperationDelete MessageOperation = "delete" ) +// MessageType is a base string type defining a type of the message type MessageType string const ( - MessageTypeSnapshot MessageType = "snapshot" + // MessageTypeSnapshot occures when plugin is processing existing snapshot data + MessageTypeSnapshot MessageType = "snapshot" + // MessageTypeStreaming occures when plugin is processing data from the binlog MessageTypeStreaming MessageType = "streaming" ) +// MessageEvent represents a message from mysql cdc plugin type MessageEvent struct { Row map[string]any `json:"row"` Table string `json:"table"` diff --git a/internal/impl/mysql/input_mysql_stream.go b/internal/impl/mysql/input_mysql_stream.go index 8a031eafb1..17ea93d080 100644 --- a/internal/impl/mysql/input_mysql_stream.go +++ b/internal/impl/mysql/input_mysql_stream.go @@ -37,8 +37,6 @@ const ( fieldCheckpointKey = "checkpoint_key" fieldCheckpointLimit = "checkpoint_limit" fieldFlavor = "flavor" - - shutdownTimeout = 5 * time.Second ) var mysqlStreamConfigSpec = service.NewConfigSpec(). @@ -97,7 +95,6 @@ type mysqlStreamInput struct { batchPolicy *service.Batcher tablesFilterMap map[string]bool checkPointLimit int - lastGtid *string logger *service.Logger res *service.Resources @@ -286,7 +283,7 @@ func (i *mysqlStreamInput) readMessages(ctx context.Context) { break } - if ok := i.flushBatch(ctx, i.cp, flushedBatch, latestPos); !ok { + if !i.flushBatch(ctx, i.cp, flushedBatch, latestPos) { break } case me := <-i.rawMessageEvents: @@ -310,7 +307,7 @@ func (i *mysqlStreamInput) readMessages(ctx context.Context) { i.logger.Debugf("Flush batch error: %w", err) break } - if ok := i.flushBatch(ctx, i.cp, flushedBatch, latestPos); !ok { + if !i.flushBatch(ctx, i.cp, flushedBatch, latestPos) { break } } else { @@ -340,7 +337,11 @@ func (i *mysqlStreamInput) startMySQLSync(ctx context.Context) { return } - defer i.snapshot.releaseSnapshot(ctx) + defer func() { + if err = i.snapshot.releaseSnapshot(ctx); err != nil { + i.logger.Errorf("Failed to properly release snapshot %v", err) + } + }() i.logger.Debugf("Starting snapshot while holding binglog pos on: %v", startPos) var wg errgroup.Group wg.SetLimit(i.snapshotMaxParallelTables) @@ -494,12 +495,12 @@ func (i *mysqlStreamInput) flushBatch(ctx context.Context, checkpointer *checkpo return true } -func (i *mysqlStreamInput) onMessage(e *canal.RowsEvent, params ProcessEventParams) error { +func (i *mysqlStreamInput) onMessage(e *canal.RowsEvent, initValue, incrementValue int) error { i.mutex.Lock() i.currentLogPosition.Pos = e.Header.LogPos i.mutex.Unlock() - for pi := params.initValue; pi < len(e.Rows); pi += params.incrementValue { + for pi := initValue; pi < len(e.Rows); pi += incrementValue { message := map[string]any{} for i, v := range e.Rows[pi] { message[e.Table.Columns[i].Name] = v @@ -572,15 +573,12 @@ func (i *mysqlStreamInput) ReadBatch(ctx context.Context) (service.MessageBatch, } func (i *mysqlStreamInput) Close(ctx context.Context) error { - fmt.Println("Close has been called") - i.shutSig.TriggerHardStop() if i.canal != nil { i.canal.SyncedPosition() i.canal.Close() } - i.snapshot.releaseSnapshot(ctx) - return nil + return i.snapshot.releaseSnapshot(ctx) } // ---- Redpanda Connect specific methods end---- @@ -613,11 +611,11 @@ func (i *mysqlStreamInput) OnRow(e *canal.RowsEvent) error { switch e.Action { case canal.InsertAction: - return i.onMessage(e, ProcessEventParams{initValue: 0, incrementValue: 1}) + return i.onMessage(e, 0, 1) case canal.DeleteAction: - return i.onMessage(e, ProcessEventParams{initValue: 0, incrementValue: 1}) + return i.onMessage(e, 0, 1) case canal.UpdateAction: - return i.onMessage(e, ProcessEventParams{initValue: 1, incrementValue: 2}) + return i.onMessage(e, 1, 2) default: return errors.New("invalid rows action") } diff --git a/internal/impl/mysql/integration_test.go b/internal/impl/mysql/integration_test.go index 67c6469c0a..cf7e3db6e2 100644 --- a/internal/impl/mysql/integration_test.go +++ b/internal/impl/mysql/integration_test.go @@ -81,10 +81,6 @@ func TestIntegrationMySQLCDC(t *testing.T) { db.SetMaxIdleConns(5) db.SetConnMaxLifetime(time.Minute * 5) - if err != nil { - return err - } - return db.Ping() }) require.NoError(t, err) @@ -244,10 +240,6 @@ func TestIntegrationMySQLSnapshotAndCDC(t *testing.T) { db.SetMaxIdleConns(5) db.SetConnMaxLifetime(time.Minute * 5) - if err != nil { - return err - } - return db.Ping() }) require.NoError(t, err) @@ -376,10 +368,6 @@ func TestIntegrationMySQLCDCWithCompositePrimaryKeys(t *testing.T) { db.SetMaxIdleConns(5) db.SetConnMaxLifetime(time.Minute * 5) - if err != nil { - return err - } - return db.Ping() }) require.NoError(t, err) @@ -392,6 +380,7 @@ func TestIntegrationMySQLCDCWithCompositePrimaryKeys(t *testing.T) { PRIMARY KEY (a, b) ) `) + require.NoError(t, err) // Create controll table to ensure we don't stream it _, err = db.Exec(` diff --git a/internal/impl/mysql/message.go b/internal/impl/mysql/message.go deleted file mode 100644 index 4f3a5b8d13..0000000000 --- a/internal/impl/mysql/message.go +++ /dev/null @@ -1,16 +0,0 @@ -// Copyright 2024 Redpanda Data, Inc. -// -// Licensed as a Redpanda Enterprise file under the Redpanda Community -// License (the "License"); you may not use this file except in compliance with -// the License. You may obtain a copy of the License at -// -// https://github.com/redpanda-data/connect/v4/blob/main/licenses/rcl.md - -package mysql - -type ChangeMessage struct { - IsStreaming bool `json:"is_streaming"` - Table string `json:"table"` - Event string `json:"event"` - Data map[string]any `json:"data"` -} diff --git a/internal/impl/mysql/snapshot.go b/internal/impl/mysql/snapshot.go index fe2bde93fe..cfa091dd73 100644 --- a/internal/impl/mysql/snapshot.go +++ b/internal/impl/mysql/snapshot.go @@ -18,6 +18,8 @@ import ( "github.com/redpanda-data/benthos/v4/public/service" ) +// Snapshot represents a structure that prepares a transaction +// and creates mysql consistent snapshot inside the transaction type Snapshot struct { db *sql.DB tx *sql.Tx @@ -29,6 +31,7 @@ type Snapshot struct { ctx context.Context } +// NewSnapshot creates new snapshot instance func NewSnapshot(ctx context.Context, logger *service.Logger, db *sql.DB) *Snapshot { return &Snapshot{ db: db, @@ -61,13 +64,18 @@ func (s *Snapshot) prepareSnapshot(ctx context.Context) (*mysql.Position, error) // Execute START TRANSACTION WITH CONSISTENT SNAPSHOT if _, err := s.tx.ExecContext(ctx, "START TRANSACTION WITH CONSISTENT SNAPSHOT"); err != nil { - s.tx.Rollback() + if rErr := s.tx.Rollback(); rErr != nil { + return nil, rErr + } + return nil, fmt.Errorf("failed to start consistent snapshot: %v", err) } // 2. Acquire global read lock (minimizing lock time) if _, err := s.lockConn.ExecContext(ctx, "FLUSH TABLES WITH READ LOCK"); err != nil { - s.tx.Rollback() + if rErr := s.tx.Rollback(); rErr != nil { + return nil, rErr + } return nil, fmt.Errorf("failed to acquire global read lock: %v", err) } @@ -75,14 +83,21 @@ func (s *Snapshot) prepareSnapshot(ctx context.Context) (*mysql.Position, error) pos, err := s.getCurrentBinlogPosition() if err != nil { // Make sure to release the lock if we fail - s.lockConn.ExecContext(ctx, "UNLOCK TABLES") - s.tx.Rollback() + if _, eErr := s.lockConn.ExecContext(ctx, "UNLOCK TABLES"); eErr != nil { + return nil, eErr + } + + if rErr := s.tx.Rollback(); rErr != nil { + return nil, rErr + } return nil, fmt.Errorf("failed to get binlog position: %v", err) } // 4. Release the global read lock immediately if _, err := s.lockConn.ExecContext(ctx, "UNLOCK TABLES"); err != nil { - s.tx.Rollback() + if rErr := s.tx.Rollback(); rErr != nil { + return nil, rErr + } return nil, fmt.Errorf("failed to release global read lock: %v", err) } @@ -91,7 +106,7 @@ func (s *Snapshot) prepareSnapshot(ctx context.Context) (*mysql.Position, error) func (s *Snapshot) getRowsCount(table string) (int, error) { var count int - if err := s.tx.QueryRowContext(s.ctx, fmt.Sprintf("SELECT COUNT(*) FROM %s", table)).Scan(&count); err != nil { + if err := s.tx.QueryRowContext(s.ctx, "SELECT COUNT(*) FROM "+table).Scan(&count); err != nil { return 0, fmt.Errorf("failed to get row count: %v", err) } return count, nil @@ -125,7 +140,7 @@ WHERE TABLE_NAME = '%s' AND CONSTRAINT_NAME = 'PRIMARY'; func (s *Snapshot) querySnapshotTable(table string, pk []string, lastSeenPkVal *map[string]any, limit int) (*sql.Rows, error) { snapshotQueryParts := []string{ - fmt.Sprintf("SELECT * FROM %s ", table), + "SELECT * FROM " + table, } if lastSeenPkVal == nil { @@ -154,23 +169,25 @@ func (s *Snapshot) querySnapshotTable(table string, pk []string, lastSeenPkVal * func (s *Snapshot) buildOrderByClause(pk []string) string { if len(pk) == 1 { - return fmt.Sprintf("ORDER BY %s", pk[0]) + return "ORDER BY " + pk[0] } - return fmt.Sprintf("ORDER BY %s", strings.Join(pk, ", ")) + return "ORDER BY " + strings.Join(pk, ", ") } func (s *Snapshot) getCurrentBinlogPosition() (mysql.Position, error) { var ( - position uint32 - file string - binlogDoDb interface{} - binlogIgnoreDb interface{} + position uint32 + file string + // binlogDoDB, binlogIgnoreDB intentionally non-used + // required to scan response + binlogDoDB interface{} + binlogIgnoreDB interface{} executedGtidSet interface{} ) row := s.snapshotConn.QueryRowContext(context.Background(), "SHOW MASTER STATUS") - if err := row.Scan(&file, &position, &binlogDoDb, &binlogIgnoreDb, &executedGtidSet); err != nil { + if err := row.Scan(&file, &position, &binlogDoDB, &binlogIgnoreDB, &executedGtidSet); err != nil { return mysql.Position{}, err } diff --git a/internal/impl/timeplus/driver/driver.go b/internal/impl/timeplus/driver/driver.go index 30d5e68a68..d3f40ce875 100644 --- a/internal/impl/timeplus/driver/driver.go +++ b/internal/impl/timeplus/driver/driver.go @@ -51,7 +51,6 @@ func (d *driver) Run(sql string) error { d.ctx, d.cancel = context.WithCancel(context.Background()) ckCtx := protonDriver.Context(d.ctx) - //nolint rows, err := d.conn.QueryContext(ckCtx, sql) if err != nil { return err From f074e50f10121d8c7bff0cf62ece3a8c70c7d4e3 Mon Sep 17 00:00:00 2001 From: Vladyslav Len Date: Thu, 21 Nov 2024 13:45:16 +0100 Subject: [PATCH 11/45] chore(): added table validation --- internal/impl/mysql/input_mysql_stream.go | 13 +++ internal/impl/mysql/validate.go | 46 +++++++++++ internal/impl/mysql/validate_test.go | 99 +++++++++++++++++++++++ 3 files changed, 158 insertions(+) create mode 100644 internal/impl/mysql/validate.go create mode 100644 internal/impl/mysql/validate_test.go diff --git a/internal/impl/mysql/input_mysql_stream.go b/internal/impl/mysql/input_mysql_stream.go index 17ea93d080..f384162187 100644 --- a/internal/impl/mysql/input_mysql_stream.go +++ b/internal/impl/mysql/input_mysql_stream.go @@ -37,6 +37,8 @@ const ( fieldCheckpointKey = "checkpoint_key" fieldCheckpointLimit = "checkpoint_limit" fieldFlavor = "flavor" + + shutdownTimeout = 5 * time.Second ) var mysqlStreamConfigSpec = service.NewConfigSpec(). @@ -189,6 +191,9 @@ func newMySQLStreamInput(conf *service.ParsedConfig, res *service.Resources) (s i.tablesFilterMap = map[string]bool{} for _, table := range i.tables { + if err = validateTableName(table); err != nil { + return nil, err + } i.tablesFilterMap[table] = true } @@ -318,6 +323,7 @@ func (i *mysqlStreamInput) readMessages(ctx context.Context) { } case err := <-i.errors: i.logger.Warnf("stream error: %s", err) + i.shutSig.TriggerSoftStop() // If the stream has errored then we should stop and restart processing return } @@ -573,6 +579,13 @@ func (i *mysqlStreamInput) ReadBatch(ctx context.Context) (service.MessageBatch, } func (i *mysqlStreamInput) Close(ctx context.Context) error { + i.shutSig.TriggerSoftStop() + select { + case <-ctx.Done(): + case <-time.After(shutdownTimeout): + case <-i.shutSig.HasStoppedChan(): + } + i.shutSig.TriggerHardStop() if i.canal != nil { i.canal.SyncedPosition() i.canal.Close() diff --git a/internal/impl/mysql/validate.go b/internal/impl/mysql/validate.go new file mode 100644 index 0000000000..b716580d59 --- /dev/null +++ b/internal/impl/mysql/validate.go @@ -0,0 +1,46 @@ +// Copyright 2024 Redpanda Data, Inc. +// +// Licensed as a Redpanda Enterprise file under the Redpanda Community +// License (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// https://github.com/redpanda-data/connect/v4/blob/main/licenses/rcl.md + +package mysql + +import ( + "errors" + "regexp" + "unicode/utf8" +) + +var ( + ErrEmptyTableName = errors.New("empty table name") + ErrInvalidTableLength = errors.New("invalid table length") + ErrInvalidTableStartChar = errors.New("invalid start char in mysql table name") + ErrInvalidTableName = errors.New("invalid table name") +) + +func validateTableName(tableName string) error { + // Check if empty + if tableName == "" { + return ErrEmptyTableName + } + + // Check length + if utf8.RuneCountInString(tableName) > 64 { + return ErrInvalidTableLength + } + + // Check if starts with a valid character + if matched, _ := regexp.MatchString(`^[a-zA-Z_]`, tableName); !matched { + return ErrInvalidTableStartChar + } + + // Check if contains only valid characters + if matched, _ := regexp.MatchString(`^[a-zA-Z0-9_$]+$`, tableName); !matched { + return ErrInvalidTableName + } + + return nil +} diff --git a/internal/impl/mysql/validate_test.go b/internal/impl/mysql/validate_test.go new file mode 100644 index 0000000000..df8bc3de23 --- /dev/null +++ b/internal/impl/mysql/validate_test.go @@ -0,0 +1,99 @@ +// Copyright 2024 Redpanda Data, Inc. +// +// Licensed as a Redpanda Enterprise file under the Redpanda Community +// License (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// https://github.com/redpanda-data/connect/v4/blob/main/licenses/rcl.md + +package mysql + +import ( + "strings" + "testing" +) + +func TestValidateTableName(t *testing.T) { + tests := []struct { + name string + tableName string + expectedErr error + }{ + // Valid cases + { + name: "Valid simple table name", + tableName: "users", + expectedErr: nil, + }, + { + name: "Valid table name with numbers", + tableName: "orders_2024", + expectedErr: nil, + }, + { + name: "Valid table name with underscore prefix", + tableName: "_temp_table", + expectedErr: nil, + }, + { + name: "Valid table name with dollar sign", + tableName: "user$data", + expectedErr: nil, + }, + { + name: "Valid table name with mixed case", + tableName: "UserProfiles", + expectedErr: nil, + }, + + // Invalid cases + { + name: "Empty table name", + tableName: "", + expectedErr: ErrEmptyTableName, + }, + { + name: "Table name starting with number", + tableName: "2users", + expectedErr: ErrInvalidTableStartChar, + }, + { + name: "Table name with special characters", + tableName: "users@table", + expectedErr: ErrInvalidTableName, + }, + { + name: "Table name with spaces", + tableName: "user table", + expectedErr: ErrInvalidTableName, + }, + { + name: "Table name with hyphens", + tableName: "user-table", + expectedErr: ErrInvalidTableName, + }, + { + name: "Too long table name", + tableName: strings.Repeat("a", 65), + expectedErr: ErrInvalidTableLength, + }, + } + + for _, tc := range tests { + t.Run(tc.name, func(t *testing.T) { + err := validateTableName(tc.tableName) + + if tc.expectedErr == nil && err != nil { + t.Errorf("expected no error, got %v", err) + } + + if tc.expectedErr != nil && err == nil { + t.Errorf("expected error %v, got nil", tc.expectedErr) + } + + if tc.expectedErr != nil && err != nil && tc.expectedErr.Error() != err.Error() { + t.Errorf("expected error %v, got %v", tc.expectedErr, err) + } + }) + } +} From 882c8d1073bd0d4da6cec4f7bca8640794451fcb Mon Sep 17 00:00:00 2001 From: Vladyslav Len Date: Thu, 21 Nov 2024 13:52:03 +0100 Subject: [PATCH 12/45] fix(): lint --- internal/impl/mysql/validate.go | 16 ++++++++-------- internal/impl/mysql/validate_test.go | 12 ++++++------ 2 files changed, 14 insertions(+), 14 deletions(-) diff --git a/internal/impl/mysql/validate.go b/internal/impl/mysql/validate.go index b716580d59..6c4ae184e8 100644 --- a/internal/impl/mysql/validate.go +++ b/internal/impl/mysql/validate.go @@ -15,31 +15,31 @@ import ( ) var ( - ErrEmptyTableName = errors.New("empty table name") - ErrInvalidTableLength = errors.New("invalid table length") - ErrInvalidTableStartChar = errors.New("invalid start char in mysql table name") - ErrInvalidTableName = errors.New("invalid table name") + errEmptyTableName = errors.New("empty table name") + errInvalidTableLength = errors.New("invalid table length") + errInvalidTableStartChar = errors.New("invalid start char in mysql table name") + errInvalidTableName = errors.New("invalid table name") ) func validateTableName(tableName string) error { // Check if empty if tableName == "" { - return ErrEmptyTableName + return errEmptyTableName } // Check length if utf8.RuneCountInString(tableName) > 64 { - return ErrInvalidTableLength + return errInvalidTableLength } // Check if starts with a valid character if matched, _ := regexp.MatchString(`^[a-zA-Z_]`, tableName); !matched { - return ErrInvalidTableStartChar + return errInvalidTableStartChar } // Check if contains only valid characters if matched, _ := regexp.MatchString(`^[a-zA-Z0-9_$]+$`, tableName); !matched { - return ErrInvalidTableName + return errInvalidTableName } return nil diff --git a/internal/impl/mysql/validate_test.go b/internal/impl/mysql/validate_test.go index df8bc3de23..0dd304ce18 100644 --- a/internal/impl/mysql/validate_test.go +++ b/internal/impl/mysql/validate_test.go @@ -50,32 +50,32 @@ func TestValidateTableName(t *testing.T) { { name: "Empty table name", tableName: "", - expectedErr: ErrEmptyTableName, + expectedErr: errEmptyTableName, }, { name: "Table name starting with number", tableName: "2users", - expectedErr: ErrInvalidTableStartChar, + expectedErr: errInvalidTableStartChar, }, { name: "Table name with special characters", tableName: "users@table", - expectedErr: ErrInvalidTableName, + expectedErr: errInvalidTableName, }, { name: "Table name with spaces", tableName: "user table", - expectedErr: ErrInvalidTableName, + expectedErr: errInvalidTableName, }, { name: "Table name with hyphens", tableName: "user-table", - expectedErr: ErrInvalidTableName, + expectedErr: errInvalidTableName, }, { name: "Too long table name", tableName: strings.Repeat("a", 65), - expectedErr: ErrInvalidTableLength, + expectedErr: errInvalidTableLength, }, } From 86238d52daae47af98036f87ea24b657f17bdcdb Mon Sep 17 00:00:00 2001 From: Vladyslav Len Date: Thu, 21 Nov 2024 14:03:31 +0100 Subject: [PATCH 13/45] fix(): added integration.CheckSkip(t) to mysql cdc tests --- internal/impl/mysql/integration_test.go | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/internal/impl/mysql/integration_test.go b/internal/impl/mysql/integration_test.go index cf7e3db6e2..c80ccf4143 100644 --- a/internal/impl/mysql/integration_test.go +++ b/internal/impl/mysql/integration_test.go @@ -23,11 +23,13 @@ import ( _ "github.com/redpanda-data/benthos/v4/public/components/io" _ "github.com/redpanda-data/benthos/v4/public/components/pure" "github.com/redpanda-data/benthos/v4/public/service" + "github.com/redpanda-data/benthos/v4/public/service/integration" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" ) func TestIntegrationMySQLCDC(t *testing.T) { + integration.CheckSkip(t) var mysqlTestVersions = []string{"8.0", "9.0", "9.1"} for _, version := range mysqlTestVersions { pool, err := dockertest.NewPool("") @@ -189,6 +191,7 @@ file: } func TestIntegrationMySQLSnapshotAndCDC(t *testing.T) { + integration.CheckSkip(t) pool, err := dockertest.NewPool("") require.NoError(t, err) @@ -317,6 +320,7 @@ file: } func TestIntegrationMySQLCDCWithCompositePrimaryKeys(t *testing.T) { + integration.CheckSkip(t) pool, err := dockertest.NewPool("") require.NoError(t, err) From dadabe0d76aa502845a1bccf817fca14ef4c840b Mon Sep 17 00:00:00 2001 From: Vladyslav Len Date: Wed, 27 Nov 2024 13:28:27 +0100 Subject: [PATCH 14/45] chore(): work on pr notes --- internal/impl/mysql/input_mysql_stream.go | 74 ++++++++++------------- internal/impl/mysql/integration_test.go | 3 - internal/impl/mysql/snapshot.go | 3 +- 3 files changed, 34 insertions(+), 46 deletions(-) diff --git a/internal/impl/mysql/input_mysql_stream.go b/internal/impl/mysql/input_mysql_stream.go index f384162187..16ac639f36 100644 --- a/internal/impl/mysql/input_mysql_stream.go +++ b/internal/impl/mysql/input_mysql_stream.go @@ -36,7 +36,6 @@ const ( fieldBatching = "batching" fieldCheckpointKey = "checkpoint_key" fieldCheckpointLimit = "checkpoint_limit" - fieldFlavor = "flavor" shutdownTimeout = 5 * time.Second ) @@ -51,21 +50,18 @@ var mysqlStreamConfigSpec = service.NewConfigSpec(). Description("A list of tables to stream from the database."). Example([]string{"table1", "table2"}), service.NewStringField(fieldCheckpointKey). - Description("The key to store the last processed binlog position."), - service.NewStringField(fieldFlavor). - Description("The flavor of MySQL to connect to."). - Example("mysql"), - service.NewBoolField(fieldMaxSnapshotParallelTables). + Description("A https://www.docs.redpanda.com/redpanda-connect/components/caches/about[cache resource^] to use for storing the current latest BinLog Position that has been successfully delivered, this allows RedPanda Connect to continue from that BinLog Position upon restart, rather than consume the entire state of the table.\""), + service.NewIntField(fieldMaxSnapshotParallelTables). Description("Int specifies a number of tables to be streamed in parallel when taking a snapshot. If set to true, the connector will stream all tables in parallel. Otherwise, it will stream tables one by one."). Default(1), service.NewIntField(fieldSnapshotMaxBatchSize). Description("The maximum number of rows to be streamed in a single batch when taking a snapshot."). Default(1000), service.NewBoolField(fieldStreamSnapshot). - Description("If set to true, the connector will query all the existing data as a part of snapshot procerss. Otherwise, it will start from the current binlog position."), + Description("If set to true, the connector will query all the existing data as a part of snapshot process. Otherwise, it will start from the current binlog position."), service.NewAutoRetryNacksToggleField(), service.NewIntField(fieldCheckpointLimit). - Description("The maximum number of messages that can be processed at a given time. Increasing this limit enables parallel processing and batching at the output level. Any given LSN will not be acknowledged unless all messages under that offset are delivered in order to preserve at least once delivery guarantees."). + Description("The maximum number of messages that can be processed at a given time. Increasing this limit enables parallel processing and batching at the output level. Any given BinLog Position will not be acknowledged unless all messages under that offset are delivered in order to preserve at least once delivery guarantees."). Default(1024), service.NewBatchPolicyField(fieldBatching), ) @@ -87,7 +83,6 @@ type mysqlStreamInput struct { dsn string tables []string - flavor string streamSnapshot bool rawMessageEvents chan MessageEvent @@ -141,10 +136,6 @@ func newMySQLStreamInput(conf *service.ParsedConfig, res *service.Resources) (s return nil, err } - if streamInput.flavor, err = conf.FieldString(fieldFlavor); err != nil { - return nil, err - } - if streamInput.streamSnapshot, err = conf.FieldBool(fieldStreamSnapshot); err != nil { return nil, err } @@ -163,27 +154,6 @@ func newMySQLStreamInput(conf *service.ParsedConfig, res *service.Resources) (s if streamInput.binLogCache, err = conf.FieldString(fieldCheckpointKey); err != nil { return nil, err - } else { - if err := res.AccessCache(context.Background(), streamInput.binLogCache, func(c service.Cache) { - binLogPositionBytes, cErr := c.Get(context.Background(), binLogCacheKey) - if err != nil { - if !errors.Is(cErr, service.ErrKeyNotFound) { - res.Logger().Errorf("failed to obtain cursor cache item. %v", cErr) - } - return - } - - var storedMySQLBinLogPosition mysqlReplications.Position - if err = json.Unmarshal(binLogPositionBytes, &storedMySQLBinLogPosition); err != nil { - res.Logger().With("error", err.Error()).Error("Failed to unmarshal stored binlog position.") - return - } - - streamInput.startBinLogPosition = &storedMySQLBinLogPosition - }); err != nil { - - res.Logger().With("error", err.Error()).Error("Failed to access cursor cache.") - } } i := &streamInput @@ -235,7 +205,7 @@ func init() { func (i *mysqlStreamInput) Connect(ctx context.Context) error { canalConfig := canal.NewDefaultConfig() - canalConfig.Flavor = i.flavor + canalConfig.Flavor = mysqlReplications.DEFAULT_FLAVOR canalConfig.Addr = i.mysqlConfig.Addr canalConfig.User = i.mysqlConfig.User canalConfig.Password = i.mysqlConfig.Passwd @@ -251,6 +221,26 @@ func (i *mysqlStreamInput) Connect(ctx context.Context) error { canalConfig.ParseTime = true canalConfig.IncludeTableRegex = i.tables + if err := i.res.AccessCache(context.Background(), i.binLogCache, func(c service.Cache) { + binLogPositionBytes, cErr := c.Get(context.Background(), binLogCacheKey) + if cErr != nil { + if !errors.Is(cErr, service.ErrKeyNotFound) { + i.logger.Errorf("failed to obtain cursor cache item. %v", cErr) + } + return + } + + var storedMySQLBinLogPosition mysqlReplications.Position + if err := json.Unmarshal(binLogPositionBytes, &storedMySQLBinLogPosition); err != nil { + i.logger.With("error", err.Error()).Error("Failed to unmarshal stored binlog position.") + return + } + + i.startBinLogPosition = &storedMySQLBinLogPosition + }); err != nil { + i.logger.With("error", err.Error()).Error("Failed to access cursor cache.") + } + c, err := canal.NewCanal(canalConfig) if err != nil { return err @@ -274,9 +264,8 @@ func (i *mysqlStreamInput) Connect(ctx context.Context) error { func (i *mysqlStreamInput) readMessages(ctx context.Context) { var nextTimedBatchChan <-chan time.Time - var latestPos *mysqlReplications.Position - for !i.shutSig.IsHasStoppedSignalled() { + for !i.shutSig.IsSoftStopSignalled() { select { case <-ctx.Done(): return @@ -288,7 +277,7 @@ func (i *mysqlStreamInput) readMessages(ctx context.Context) { break } - if !i.flushBatch(ctx, i.cp, flushedBatch, latestPos) { + if !i.flushBatch(ctx, i.cp, flushedBatch, i.currentLogPosition) { break } case me := <-i.rawMessageEvents: @@ -302,7 +291,9 @@ func (i *mysqlStreamInput) readMessages(ctx context.Context) { mb.MetaSet("table", me.Table) mb.MetaSet("type", string(me.Type)) if me.Position != nil { - latestPos = me.Position + i.mutex.Lock() + i.currentLogPosition = me.Position + i.mutex.Unlock() } if i.batchPolicy.Add(mb) { @@ -312,7 +303,7 @@ func (i *mysqlStreamInput) readMessages(ctx context.Context) { i.logger.Debugf("Flush batch error: %w", err) break } - if !i.flushBatch(ctx, i.cp, flushedBatch, latestPos) { + if !i.flushBatch(ctx, i.cp, flushedBatch, i.currentLogPosition) { break } } else { @@ -336,7 +327,6 @@ func (i *mysqlStreamInput) startMySQLSync(ctx context.Context) { // If we require snapshot streaming && we don't have a binlog position cache // initiate default run for Canal to process snapshot and start incremental sync of binlog if i.streamSnapshot && i.startBinLogPosition == nil { - // Doesn't work at the moment startPos, err := i.snapshot.prepareSnapshot(ctx) if err != nil { i.errors <- err @@ -487,7 +477,7 @@ func (i *mysqlStreamInput) flushBatch(ctx context.Context, checkpointer *checkpo return nil } - if err := i.syncBinlogPosition(context.Background()); err != nil { + if err := i.syncBinlogPosition(ctx); err != nil { return err } diff --git a/internal/impl/mysql/integration_test.go b/internal/impl/mysql/integration_test.go index c80ccf4143..a1a3003700 100644 --- a/internal/impl/mysql/integration_test.go +++ b/internal/impl/mysql/integration_test.go @@ -103,7 +103,6 @@ mysql_stream: checkpoint_key: foocache tables: - foo - flavor: mysql `, dsn) cacheConf := fmt.Sprintf(` @@ -270,7 +269,6 @@ mysql_stream: checkpoint_key: foocache tables: - foo - flavor: mysql `, dsn) cacheConf := fmt.Sprintf(` @@ -415,7 +413,6 @@ mysql_stream: checkpoint_key: foocache tables: - foo - flavor: mysql `, dsn) cacheConf := fmt.Sprintf(` diff --git a/internal/impl/mysql/snapshot.go b/internal/impl/mysql/snapshot.go index cfa091dd73..de1ecaaf64 100644 --- a/internal/impl/mysql/snapshot.go +++ b/internal/impl/mysql/snapshot.go @@ -117,7 +117,8 @@ func (s *Snapshot) getTablePrimaryKeys(table string) ([]string, error) { rows, err := s.tx.QueryContext(s.ctx, fmt.Sprintf(` SELECT COLUMN_NAME FROM information_schema.KEY_COLUMN_USAGE -WHERE TABLE_NAME = '%s' AND CONSTRAINT_NAME = 'PRIMARY'; +WHERE TABLE_NAME = '%s' AND CONSTRAINT_NAME = 'PRIMARY' +ORDER BY ORDINAL_POSITION; `, table)) if err != nil { return nil, fmt.Errorf("failed to get primary key: %v", err) From 193c574a650d7ec361c826b8029acd6386f6ec9d Mon Sep 17 00:00:00 2001 From: Vladyslav Len Date: Thu, 28 Nov 2024 17:10:12 +0100 Subject: [PATCH 15/45] chore(): updated comments and use ctx from shutdown --- internal/impl/mysql/input_mysql_stream.go | 6 ++++-- internal/impl/mysql/snapshot.go | 24 ++++++++++++++++++----- 2 files changed, 23 insertions(+), 7 deletions(-) diff --git a/internal/impl/mysql/input_mysql_stream.go b/internal/impl/mysql/input_mysql_stream.go index 16ac639f36..4e286431a2 100644 --- a/internal/impl/mysql/input_mysql_stream.go +++ b/internal/impl/mysql/input_mysql_stream.go @@ -258,7 +258,7 @@ func (i *mysqlStreamInput) Connect(ctx context.Context) error { snapshot := NewSnapshot(ctx, i.logger, db) i.snapshot = snapshot - go i.startMySQLSync(ctx) + go i.startMySQLSync() return nil } @@ -321,7 +321,9 @@ func (i *mysqlStreamInput) readMessages(ctx context.Context) { } } -func (i *mysqlStreamInput) startMySQLSync(ctx context.Context) { +func (i *mysqlStreamInput) startMySQLSync() { + ctx, _ := i.shutSig.SoftStopCtx(context.Background()) + i.canal.SetEventHandler(i) go i.readMessages(ctx) // If we require snapshot streaming && we don't have a binlog position cache diff --git a/internal/impl/mysql/snapshot.go b/internal/impl/mysql/snapshot.go index de1ecaaf64..ffddcf8e3e 100644 --- a/internal/impl/mysql/snapshot.go +++ b/internal/impl/mysql/snapshot.go @@ -54,7 +54,7 @@ func (s *Snapshot) prepareSnapshot(ctx context.Context) (*mysql.Position, error) return nil, fmt.Errorf("failed to create snapshot connection: %v", err) } - // 1. Start a consistent snapshot transaction + // Start a consistent snapshot transaction s.tx, err = s.snapshotConn.BeginTx(ctx, &sql.TxOptions{ Isolation: sql.LevelRepeatableRead, }) @@ -62,7 +62,14 @@ func (s *Snapshot) prepareSnapshot(ctx context.Context) (*mysql.Position, error) return nil, fmt.Errorf("failed to start transaction: %v", err) } - // Execute START TRANSACTION WITH CONSISTENT SNAPSHOT + /* + START TRANSACTION WITH CONSISTENT SNAPSHOT ensures a consistent view of database state + when reading historical data during CDC initialization. Without it, concurrent writes + could create inconsistencies between binlog position and table snapshots, potentially + missing or duplicating events. The snapshot prevents other transactions from modifying + the data being read, maintaining referential integrity across tables while capturing + the initial state. + */ if _, err := s.tx.ExecContext(ctx, "START TRANSACTION WITH CONSISTENT SNAPSHOT"); err != nil { if rErr := s.tx.Rollback(); rErr != nil { return nil, rErr @@ -71,7 +78,14 @@ func (s *Snapshot) prepareSnapshot(ctx context.Context) (*mysql.Position, error) return nil, fmt.Errorf("failed to start consistent snapshot: %v", err) } - // 2. Acquire global read lock (minimizing lock time) + /* + FLUSH TABLES WITH READ LOCK is executed after CONSISTENT SNAPSHOT to: + 1. Force MySQL to flush all data from memory to disk + 2. Prevent any writes to tables while we read the binlog position + + This lock MUST be released quickly to avoid blocking other connections. Only use it + to capture the binlog coordinates, then release immediately with UNLOCK TABLES. + */ if _, err := s.lockConn.ExecContext(ctx, "FLUSH TABLES WITH READ LOCK"); err != nil { if rErr := s.tx.Rollback(); rErr != nil { return nil, rErr @@ -79,7 +93,7 @@ func (s *Snapshot) prepareSnapshot(ctx context.Context) (*mysql.Position, error) return nil, fmt.Errorf("failed to acquire global read lock: %v", err) } - // 3. Get binary log position (while locked) + // Get binary log position (while locked) pos, err := s.getCurrentBinlogPosition() if err != nil { // Make sure to release the lock if we fail @@ -93,7 +107,7 @@ func (s *Snapshot) prepareSnapshot(ctx context.Context) (*mysql.Position, error) return nil, fmt.Errorf("failed to get binlog position: %v", err) } - // 4. Release the global read lock immediately + // Release the global read lock immediately after getting the binlog position if _, err := s.lockConn.ExecContext(ctx, "UNLOCK TABLES"); err != nil { if rErr := s.tx.Rollback(); rErr != nil { return nil, rErr From 8a8eb8b4ab01d384fa7b2a59f339fd76593802ee Mon Sep 17 00:00:00 2001 From: Vladyslav Len Date: Wed, 4 Dec 2024 16:40:33 +0100 Subject: [PATCH 16/45] chore(): removed rows count && small pr notes --- go.mod | 9 ++- go.sum | 20 +++---- internal/impl/mysql/input_mysql_stream.go | 68 ++++++++++++----------- internal/impl/mysql/integration_test.go | 6 +- internal/impl/mysql/snapshot.go | 19 +++---- 5 files changed, 59 insertions(+), 63 deletions(-) diff --git a/go.mod b/go.mod index f6abf2bb1d..d77833db76 100644 --- a/go.mod +++ b/go.mod @@ -65,7 +65,7 @@ require ( github.com/getsentry/sentry-go v0.28.1 github.com/go-faker/faker/v4 v4.4.2 github.com/go-jose/go-jose/v3 v3.0.3 - github.com/go-mysql-org/go-mysql v1.9.1 + github.com/go-mysql-org/go-mysql v1.10.0 github.com/go-resty/resty/v2 v2.15.3 github.com/go-sql-driver/mysql v1.8.1 github.com/gocql/gocql v1.6.0 @@ -171,7 +171,6 @@ require ( github.com/certifi/gocertifi v0.0.0-20210507211836-431795d63e8d // indirect github.com/cncf/xds/go v0.0.0-20240905190251-b4127c9b8d78 // indirect github.com/containerd/platforms v0.2.1 // indirect - github.com/cznic/mathutil v0.0.0-20181122101859-297441e03548 // indirect github.com/envoyproxy/go-control-plane v0.13.0 // indirect github.com/envoyproxy/protoc-gen-validate v1.1.0 // indirect github.com/hamba/avro/v2 v2.22.2-0.20240625062549-66aad10411d9 // indirect @@ -181,10 +180,10 @@ require ( github.com/modern-go/reflect2 v1.0.2 // indirect github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822 // indirect github.com/onsi/gomega v1.34.2 // indirect - github.com/pingcap/errors v0.11.5-0.20221009092201-b66cddb77c32 // indirect - github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c // indirect + github.com/pingcap/errors v0.11.5-0.20240311024730-e056997136bb // indirect + github.com/pingcap/failpoint v0.0.0-20240528011301-b51a646c7c86 // indirect github.com/pingcap/log v1.1.1-0.20230317032135-a0d097d16e22 // indirect - github.com/pingcap/tidb/pkg/parser v0.0.0-20231103042308-035ad5ccbe67 // indirect + github.com/pingcap/tidb/pkg/parser v0.0.0-20241118164214-4f047be191be // indirect github.com/planetscale/vtprotobuf v0.6.1-0.20240319094008-0393e58bdf10 // indirect github.com/samber/lo v1.47.0 // indirect github.com/siddontang/go v0.0.0-20180604090527-bdc77568d726 // indirect diff --git a/go.sum b/go.sum index 0863e07f45..4ec9efadf9 100644 --- a/go.sum +++ b/go.sum @@ -997,8 +997,6 @@ github.com/creack/pty v1.1.7/go.mod h1:lj5s0c3V2DBrqTV7llrYr5NG6My20zk30Fl46Y7Do github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= github.com/creack/pty v1.1.21 h1:1/QdRyBaHHJP61QkWMXlOIBfsgdDeeKfK8SYVUWJKf0= github.com/creack/pty v1.1.21/go.mod h1:MOBLtS5ELjhRRrroQr9kyvTxUAFNvYEK993ew/Vr4O4= -github.com/cznic/mathutil v0.0.0-20181122101859-297441e03548 h1:iwZdTE0PVqJCos1vaoKsclOGD3ADKpshg3SRtYBbwso= -github.com/cznic/mathutil v0.0.0-20181122101859-297441e03548/go.mod h1:e6NPNENfs9mPDVNRekM7lKScauxd5kXTr1Mfyig6TDM= github.com/danieljoos/wincred v1.2.0 h1:ozqKHaLK0W/ii4KVbbvluM91W2H3Sh0BncbUNPS7jLE= github.com/danieljoos/wincred v1.2.0/go.mod h1:FzQLLMKBFdvu+osBrnFODiv32YGwCfx0SkRa/eYHgec= github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= @@ -1120,8 +1118,8 @@ github.com/go-logr/logr v1.4.2 h1:6pFjapn8bFcIbiKo3XT4j/BhANplGihG6tvd+8rYgrY= github.com/go-logr/logr v1.4.2/go.mod h1:9T104GzyrTigFIr8wt5mBrctHMim0Nb2HLGrmQ40KvY= github.com/go-logr/stdr v1.2.2 h1:hSWxHoqTgW2S2qGc0LTAI563KZ5YKYRhT3MFKZMbjag= github.com/go-logr/stdr v1.2.2/go.mod h1:mMo/vtBO5dYbehREoey6XUKy/eSumjCCveDpRre4VKE= -github.com/go-mysql-org/go-mysql v1.9.1 h1:W2ZKkHkoM4mmkasJCoSYfaE4RQNxXTb6VqiaMpKFrJc= -github.com/go-mysql-org/go-mysql v1.9.1/go.mod h1:+SgFgTlqjqOQoMc98n9oyUWEgn2KkOL1VmXDoq2ONOs= +github.com/go-mysql-org/go-mysql v1.10.0 h1:9iEPrZdHKq6EepUuPONrBA+wc3aL1WLhbUm5w8ryDFg= +github.com/go-mysql-org/go-mysql v1.10.0/go.mod h1:GzFQAI+FqbYAPtsannL0hmZH6zcLzCQbwqopT9bgTt0= github.com/go-ole/go-ole v1.2.6/go.mod h1:pprOEPIfldk/42T2oK7lQ4v4JSDwmV0As9GaiUsvbm0= github.com/go-ole/go-ole v1.3.0 h1:Dt6ye7+vXGIKZ7Xtk4s6/xVdGDQynvom7xCFEdWr6uE= github.com/go-ole/go-ole v1.3.0/go.mod h1:5LS6F96DhAwUc7C+1HLexzMXY1xGRSryjyPPKW6zv78= @@ -1675,15 +1673,14 @@ github.com/pierrec/lz4/v4 v4.1.21/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFu github.com/pinecone-io/go-pinecone v1.0.0 h1:90euw+0EKSgdeE9q7iGSTVmdx9r9+x3mxWkrCCLab+o= github.com/pinecone-io/go-pinecone v1.0.0/go.mod h1:KfJhn4yThX293+fbtrZLnxe2PJYo8557Py062W4FYKk= github.com/pingcap/errors v0.11.0/go.mod h1:Oi8TUi2kEtXXLMJk9l1cGmz20kV3TaQ0usTwv5KuLY8= -github.com/pingcap/errors v0.11.4/go.mod h1:Oi8TUi2kEtXXLMJk9l1cGmz20kV3TaQ0usTwv5KuLY8= -github.com/pingcap/errors v0.11.5-0.20221009092201-b66cddb77c32 h1:m5ZsBa5o/0CkzZXfXLaThzKuR85SnHHetqBCpzQ30h8= -github.com/pingcap/errors v0.11.5-0.20221009092201-b66cddb77c32/go.mod h1:X2r9ueLEUZgtx2cIogM0v4Zj5uvvzhuuiu7Pn8HzMPg= -github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c h1:CgbKAHto5CQgWM9fSBIvaxsJHuGP0uM74HXtv3MyyGQ= -github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c/go.mod h1:4qGtCB0QK0wBzKtFEGDhxXnSnbQApw1gc9siScUl8ew= +github.com/pingcap/errors v0.11.5-0.20240311024730-e056997136bb h1:3pSi4EDG6hg0orE1ndHkXvX6Qdq2cZn8gAPir8ymKZk= +github.com/pingcap/errors v0.11.5-0.20240311024730-e056997136bb/go.mod h1:X2r9ueLEUZgtx2cIogM0v4Zj5uvvzhuuiu7Pn8HzMPg= +github.com/pingcap/failpoint v0.0.0-20240528011301-b51a646c7c86 h1:tdMsjOqUR7YXHoBitzdebTvOjs/swniBTOLy5XiMtuE= +github.com/pingcap/failpoint v0.0.0-20240528011301-b51a646c7c86/go.mod h1:exzhVYca3WRtd6gclGNErRWb1qEgff3LYta0LvRmON4= github.com/pingcap/log v1.1.1-0.20230317032135-a0d097d16e22 h1:2SOzvGvE8beiC1Y4g9Onkvu6UmuBBOeWRGQEjJaT/JY= github.com/pingcap/log v1.1.1-0.20230317032135-a0d097d16e22/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= -github.com/pingcap/tidb/pkg/parser v0.0.0-20231103042308-035ad5ccbe67 h1:m0RZ583HjzG3NweDi4xAcK54NBBPJh+zXp5Fp60dHtw= -github.com/pingcap/tidb/pkg/parser v0.0.0-20231103042308-035ad5ccbe67/go.mod h1:yRkiqLFwIqibYg2P7h4bclHjHcJiIFRLKhGRyBcKYus= +github.com/pingcap/tidb/pkg/parser v0.0.0-20241118164214-4f047be191be h1:t5EkCmZpxLCig5GQA0AZG47aqsuL5GTsJeeUD+Qfies= +github.com/pingcap/tidb/pkg/parser v0.0.0-20241118164214-4f047be191be/go.mod h1:Hju1TEWZvrctQKbztTRwXH7rd41Yq0Pgmq4PrEKcq7o= github.com/pkg/browser v0.0.0-20180916011732-0a3d74bf9ce4/go.mod h1:4OwLy04Bl9Ef3GJJCoec+30X3LQs/0/m4HFRt/2LUSA= github.com/pkg/browser v0.0.0-20240102092130-5ac0b6a4141c h1:+mdjkGKdHQG3305AYmdv1U2eRNDiU2ErMBj1gwrq8eQ= github.com/pkg/browser v0.0.0-20240102092130-5ac0b6a4141c/go.mod h1:7rwL4CYBLnjLxUqIJNnCWiEdr3bn6IUYi15bNlnbCCU= @@ -1791,7 +1788,6 @@ github.com/segmentio/encoding v0.4.0 h1:MEBYvRqiUB2nfR2criEXWqwdY6HJOUrCn5hboVOV github.com/segmentio/encoding v0.4.0/go.mod h1:/d03Cd8PoaDeceuhUUUQWjU0KhWjrmYrWPgtJHYZSnI= github.com/segmentio/ksuid v1.0.4 h1:sBo2BdShXjmcugAMwjugoGUdUV0pcxY5mW4xKRn3v4c= github.com/segmentio/ksuid v1.0.4/go.mod h1:/XUiZBD3kVx5SmUOl55voK5yeAbBNNIed+2O73XgrPE= -github.com/sergi/go-diff v1.1.0/go.mod h1:STckp+ISIX8hZLjrqAeVduY0gWCT9IjLuqbuNXdaHfM= github.com/shirou/gopsutil/v3 v3.24.5 h1:i0t8kL+kQTvpAYToeuiVk3TgDeKOFioZO3Ztz/iZ9pI= github.com/shirou/gopsutil/v3 v3.24.5/go.mod h1:bsoOS1aStSs9ErQ1WWfxllSeS1K5D+U30r2NfcubMVk= github.com/shoenig/go-m1cpu v0.1.6 h1:nxdKQNcEB6vzgA2E2bvzKIYRuNj7XNJ4S/aRSwKzFtM= diff --git a/internal/impl/mysql/input_mysql_stream.go b/internal/impl/mysql/input_mysql_stream.go index 4e286431a2..8002cd17f3 100644 --- a/internal/impl/mysql/input_mysql_stream.go +++ b/internal/impl/mysql/input_mysql_stream.go @@ -72,11 +72,12 @@ type asyncMessage struct { } type mysqlStreamInput struct { + canal.DummyEventHandler + mutex sync.Mutex // canal stands for mysql binlog listener connection - canal *canal.Canal - mysqlConfig *mysql.Config - canal.DummyEventHandler + canal *canal.Canal + mysqlConfig *mysql.Config startBinLogPosition *mysqlReplications.Position currentLogPosition *mysqlReplications.Position binLogCache string @@ -115,7 +116,6 @@ func newMySQLStreamInput(conf *service.ParsedConfig, res *service.Resources) (s snapshotMessageEvents: make(chan MessageEvent), msgChan: make(chan asyncMessage), res: res, - streamCtx: context.Background(), errors: make(chan error, 1), shutSig: shutdown.NewSignaller(), @@ -182,17 +182,19 @@ func newMySQLStreamInput(conf *service.ParsedConfig, res *service.Resources) (s batching.Count = 1 } + i.streamCtx, _ = i.shutSig.SoftStopCtx(context.Background()) + r, err := service.AutoRetryNacksBatchedToggled(conf, i) if err != nil { return nil, err } - return conf.WrapBatchInputExtractTracingSpanMapping("mysql_stream", r) + return conf.WrapBatchInputExtractTracingSpanMapping("mysql_cdc", r) } func init() { err := service.RegisterBatchInput( - "mysql_stream", mysqlStreamConfigSpec, + "mysql_cdc", mysqlStreamConfigSpec, func(conf *service.ParsedConfig, mgr *service.Resources) (service.BatchInput, error) { return newMySQLStreamInput(conf, mgr) }) @@ -335,24 +337,12 @@ func (i *mysqlStreamInput) startMySQLSync() { return } - defer func() { - if err = i.snapshot.releaseSnapshot(ctx); err != nil { - i.logger.Errorf("Failed to properly release snapshot %v", err) - } - }() i.logger.Debugf("Starting snapshot while holding binglog pos on: %v", startPos) var wg errgroup.Group wg.SetLimit(i.snapshotMaxParallelTables) for _, table := range i.tables { wg.Go(func() (err error) { - rowsCount, err := i.snapshot.getRowsCount(table) - if err != nil { - return err - } - - i.logger.Debugf("Rows count for table %s is %d", table, rowsCount) - tablePks, err := i.snapshot.getTablePrimaryKeys(table) if err != nil { return err @@ -366,7 +356,7 @@ func (i *mysqlStreamInput) startMySQLSync() { lastSeenPksValues[pk] = nil } - for numRowsProcessed < rowsCount { + for { var batchRows *sql.Rows if numRowsProcessed == 0 { batchRows, err = i.snapshot.querySnapshotTable(table, tablePks, nil, i.fieldSnapshotMaxBatchSize) @@ -380,8 +370,10 @@ func (i *mysqlStreamInput) startMySQLSync() { } } + var batchRowsCount int for batchRows.Next() { numRowsProcessed++ + batchRowsCount++ columns, err := batchRows.Columns() if err != nil { @@ -417,6 +409,10 @@ func (i *mysqlStreamInput) startMySQLSync() { Position: nil, } } + + if batchRowsCount < i.fieldSnapshotMaxBatchSize { + break + } } return nil @@ -427,18 +423,28 @@ func (i *mysqlStreamInput) startMySQLSync() { i.errors <- fmt.Errorf("snapshot processing failed: %w", err) } + if err = i.snapshot.releaseSnapshot(ctx); err != nil { + i.logger.Errorf("Failed to properly release snapshot %v", err) + } + + i.mutex.Lock() + i.currentLogPosition = startPos + i.mutex.Unlock() + i.logger.Infof("Snapshot is done...Running CDC from BingLog: %s on pos: %d", startPos.Name, startPos.Pos) if err := i.canal.RunFrom(*startPos); err != nil { i.errors <- fmt.Errorf("failed to start streaming: %v", err) } } else { coords, _ := i.canal.GetMasterPos() + i.mutex.Lock() // starting from the last stored binlog position if i.startBinLogPosition != nil { coords = *i.startBinLogPosition } i.currentLogPosition = &coords + i.mutex.Lock() if err := i.canal.RunFrom(coords); err != nil { i.errors <- fmt.Errorf("failed to start streaming: %v", err) } @@ -464,7 +470,7 @@ func (i *mysqlStreamInput) flushBatch(ctx context.Context, checkpointer *checkpo return false } - lastMsg := batch[len(batch)-1] + lastMessage := batch[len(batch)-1] select { case i.msgChan <- asyncMessage{ @@ -475,11 +481,12 @@ func (i *mysqlStreamInput) flushBatch(ctx context.Context, checkpointer *checkpo return nil } - if msgType, ok := lastMsg.MetaGet("type"); ok && msgType == "snapshot" { + // do not call checkpoint if the last message in the batch is a snapshot + if msgType, ok := lastMessage.MetaGet("type"); ok && msgType == "snapshot" { return nil } - if err := i.syncBinlogPosition(ctx); err != nil { + if err := i.syncBinlogPosition(ctx, *maxOffset); err != nil { return err } @@ -516,7 +523,7 @@ func (i *mysqlStreamInput) onMessage(e *canal.RowsEvent, initValue, incrementVal return nil } -func (i *mysqlStreamInput) syncBinlogPosition(ctx context.Context) error { +func (i *mysqlStreamInput) syncBinlogPosition(ctx context.Context, binLogPos *int64) error { i.mutex.Lock() defer i.mutex.Unlock() @@ -525,6 +532,10 @@ func (i *mysqlStreamInput) syncBinlogPosition(ctx context.Context) error { return errors.New("no current binlog position") } + if binLogPos != nil { + i.currentLogPosition.Pos = uint32(*binLogPos) + } + var ( positionInByte []byte err error @@ -583,7 +594,7 @@ func (i *mysqlStreamInput) Close(ctx context.Context) error { i.canal.Close() } - return i.snapshot.releaseSnapshot(ctx) + return i.snapshot.close() } // ---- Redpanda Connect specific methods end---- @@ -613,7 +624,6 @@ func (i *mysqlStreamInput) OnRow(e *canal.RowsEvent) error { if _, ok := i.tablesFilterMap[e.Table.Name]; !ok { return nil } - switch e.Action { case canal.InsertAction: return i.onMessage(e, 0, 1) @@ -626,12 +636,4 @@ func (i *mysqlStreamInput) OnRow(e *canal.RowsEvent) error { } } -func (i *mysqlStreamInput) OnPosSynced(eh *replication.EventHeader, pos mysqlReplications.Position, gtid mysqlReplications.GTIDSet, synced bool) error { - i.mutex.Lock() - i.currentLogPosition = &pos - i.mutex.Unlock() - - return i.syncBinlogPosition(context.Background()) -} - // --- MySQL Canal handler methods end ---- diff --git a/internal/impl/mysql/integration_test.go b/internal/impl/mysql/integration_test.go index a1a3003700..204fb4de12 100644 --- a/internal/impl/mysql/integration_test.go +++ b/internal/impl/mysql/integration_test.go @@ -97,7 +97,7 @@ func TestIntegrationMySQLCDC(t *testing.T) { tmpDir := t.TempDir() template := fmt.Sprintf(` -mysql_stream: +mysql_cdc: dsn: %s stream_snapshot: false checkpoint_key: foocache @@ -262,7 +262,7 @@ func TestIntegrationMySQLSnapshotAndCDC(t *testing.T) { } template := fmt.Sprintf(` -mysql_stream: +mysql_cdc: dsn: %s stream_snapshot: true snapshot_max_batch_size: 500 @@ -406,7 +406,7 @@ func TestIntegrationMySQLCDCWithCompositePrimaryKeys(t *testing.T) { } template := fmt.Sprintf(` -mysql_stream: +mysql_cdc: dsn: %s stream_snapshot: true snapshot_max_batch_size: 500 diff --git a/internal/impl/mysql/snapshot.go b/internal/impl/mysql/snapshot.go index ffddcf8e3e..5fa957783b 100644 --- a/internal/impl/mysql/snapshot.go +++ b/internal/impl/mysql/snapshot.go @@ -58,6 +58,7 @@ func (s *Snapshot) prepareSnapshot(ctx context.Context) (*mysql.Position, error) s.tx, err = s.snapshotConn.BeginTx(ctx, &sql.TxOptions{ Isolation: sql.LevelRepeatableRead, }) + if err != nil { return nil, fmt.Errorf("failed to start transaction: %v", err) } @@ -118,14 +119,6 @@ func (s *Snapshot) prepareSnapshot(ctx context.Context) (*mysql.Position, error) return &pos, nil } -func (s *Snapshot) getRowsCount(table string) (int, error) { - var count int - if err := s.tx.QueryRowContext(s.ctx, "SELECT COUNT(*) FROM "+table).Scan(&count); err != nil { - return 0, fmt.Errorf("failed to get row count: %v", err) - } - return count, nil -} - func (s *Snapshot) getTablePrimaryKeys(table string) ([]string, error) { // Get primary key columns for the table rows, err := s.tx.QueryContext(s.ctx, fmt.Sprintf(` @@ -219,12 +212,18 @@ func (s *Snapshot) releaseSnapshot(ctx context.Context) error { } } + // reset transaction + s.tx = nil + return nil +} + +func (s *Snapshot) close() error { if s.lockConn != nil { - s.lockConn.Close() + return s.lockConn.Close() } if s.snapshotConn != nil { - s.snapshotConn.Close() + return s.snapshotConn.Close() } return nil From 2373dee9990b68b38fb43168d1300cbaf52ac28c Mon Sep 17 00:00:00 2001 From: Tyler Rockwood Date: Tue, 10 Dec 2024 03:37:10 +0000 Subject: [PATCH 17/45] mysqlcdc: rename component --- internal/impl/mysql/input_mysql_stream.go | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/internal/impl/mysql/input_mysql_stream.go b/internal/impl/mysql/input_mysql_stream.go index 8002cd17f3..6c949a378b 100644 --- a/internal/impl/mysql/input_mysql_stream.go +++ b/internal/impl/mysql/input_mysql_stream.go @@ -107,7 +107,7 @@ type mysqlStreamInput struct { fieldSnapshotMaxBatchSize int } -const binLogCacheKey = "mysql_binlog_position" +const binLogCacheKey = "ysql_binlog_position" func newMySQLStreamInput(conf *service.ParsedConfig, res *service.Resources) (s service.BatchInput, err error) { streamInput := mysqlStreamInput{ @@ -167,8 +167,6 @@ func newMySQLStreamInput(conf *service.ParsedConfig, res *service.Resources) (s i.tablesFilterMap[table] = true } - res.Logger().Info("Starting MySQL stream input") - if batching, err = conf.FieldBatchPolicy(fieldBatching); err != nil { return nil, err } else if batching.IsNoop() { From 166f1dc833d3e80fc063aa2520f3503e29678c26 Mon Sep 17 00:00:00 2001 From: Tyler Rockwood Date: Tue, 10 Dec 2024 03:44:24 +0000 Subject: [PATCH 18/45] mysql: seperate cache key vs cache that is used in configs --- internal/impl/mysql/input_mysql_stream.go | 32 ++++++++++++++--------- internal/impl/mysql/integration_test.go | 6 ++--- 2 files changed, 22 insertions(+), 16 deletions(-) diff --git a/internal/impl/mysql/input_mysql_stream.go b/internal/impl/mysql/input_mysql_stream.go index 6c949a378b..da45e5ea89 100644 --- a/internal/impl/mysql/input_mysql_stream.go +++ b/internal/impl/mysql/input_mysql_stream.go @@ -35,6 +35,7 @@ const ( fieldSnapshotMaxBatchSize = "snapshot_max_batch_size" fieldBatching = "batching" fieldCheckpointKey = "checkpoint_key" + fieldCheckpointCache = "checkpoint_cache" fieldCheckpointLimit = "checkpoint_limit" shutdownTimeout = 5 * time.Second @@ -49,8 +50,11 @@ var mysqlStreamConfigSpec = service.NewConfigSpec(). service.NewStringListField(fieldMySQLTables). Description("A list of tables to stream from the database."). Example([]string{"table1", "table2"}), + service.NewStringField(fieldCheckpointCache). + Description("A https://www.docs.redpanda.com/redpanda-connect/components/caches/about[cache resource^] to use for storing the current latest BinLog Position that has been successfully delivered, this allows Redpanda Connect to continue from that BinLog Position upon restart, rather than consume the entire state of the table.\""), service.NewStringField(fieldCheckpointKey). - Description("A https://www.docs.redpanda.com/redpanda-connect/components/caches/about[cache resource^] to use for storing the current latest BinLog Position that has been successfully delivered, this allows RedPanda Connect to continue from that BinLog Position upon restart, rather than consume the entire state of the table.\""), + Description("The key to use to store the snapshot position in `"+fieldCheckpointCache+"`. An alternative key can be provided if multiple CDC inputs share the same cache."). + Default("mysql_binlog_position"), service.NewIntField(fieldMaxSnapshotParallelTables). Description("Int specifies a number of tables to be streamed in parallel when taking a snapshot. If set to true, the connector will stream all tables in parallel. Otherwise, it will stream tables one by one."). Default(1), @@ -81,6 +85,7 @@ type mysqlStreamInput struct { startBinLogPosition *mysqlReplications.Position currentLogPosition *mysqlReplications.Position binLogCache string + binLogCacheKey string dsn string tables []string @@ -107,8 +112,6 @@ type mysqlStreamInput struct { fieldSnapshotMaxBatchSize int } -const binLogCacheKey = "ysql_binlog_position" - func newMySQLStreamInput(conf *service.ParsedConfig, res *service.Resources) (s service.BatchInput, err error) { streamInput := mysqlStreamInput{ logger: res.Logger(), @@ -152,7 +155,13 @@ func newMySQLStreamInput(conf *service.ParsedConfig, res *service.Resources) (s return nil, err } - if streamInput.binLogCache, err = conf.FieldString(fieldCheckpointKey); err != nil { + if streamInput.binLogCache, err = conf.FieldString(fieldCheckpointCache); err != nil { + return nil, err + } + if !conf.Resources().HasCache(streamInput.binLogCache) { + return nil, fmt.Errorf("unknown cache resource: %s", streamInput.binLogCache) + } + if streamInput.binLogCacheKey, err = conf.FieldString(fieldCheckpointKey); err != nil { return nil, err } @@ -221,8 +230,8 @@ func (i *mysqlStreamInput) Connect(ctx context.Context) error { canalConfig.ParseTime = true canalConfig.IncludeTableRegex = i.tables - if err := i.res.AccessCache(context.Background(), i.binLogCache, func(c service.Cache) { - binLogPositionBytes, cErr := c.Get(context.Background(), binLogCacheKey) + if err := i.res.AccessCache(ctx, i.binLogCache, func(c service.Cache) { + binLogPositionBytes, cErr := c.Get(ctx, i.binLogCacheKey) if cErr != nil { if !errors.Is(cErr, service.ErrKeyNotFound) { i.logger.Errorf("failed to obtain cursor cache item. %v", cErr) @@ -358,14 +367,11 @@ func (i *mysqlStreamInput) startMySQLSync() { var batchRows *sql.Rows if numRowsProcessed == 0 { batchRows, err = i.snapshot.querySnapshotTable(table, tablePks, nil, i.fieldSnapshotMaxBatchSize) - if err != nil { - return err - } } else { batchRows, err = i.snapshot.querySnapshotTable(table, tablePks, &lastSeenPksValues, i.fieldSnapshotMaxBatchSize) - if err != nil { - return err - } + } + if err != nil { + return err } var batchRowsCount int @@ -545,7 +551,7 @@ func (i *mysqlStreamInput) syncBinlogPosition(ctx context.Context, binLogPos *in var cErr error if err := i.res.AccessCache(ctx, i.binLogCache, func(c service.Cache) { - cErr = c.Set(ctx, binLogCacheKey, positionInByte, nil) + cErr = c.Set(ctx, i.binLogCacheKey, positionInByte, nil) if cErr != nil { i.logger.Errorf("Failed to store binlog position: %v", cErr) } diff --git a/internal/impl/mysql/integration_test.go b/internal/impl/mysql/integration_test.go index 204fb4de12..8394db3d35 100644 --- a/internal/impl/mysql/integration_test.go +++ b/internal/impl/mysql/integration_test.go @@ -100,7 +100,7 @@ func TestIntegrationMySQLCDC(t *testing.T) { mysql_cdc: dsn: %s stream_snapshot: false - checkpoint_key: foocache + checkpoint_cache: foocache tables: - foo `, dsn) @@ -266,7 +266,7 @@ mysql_cdc: dsn: %s stream_snapshot: true snapshot_max_batch_size: 500 - checkpoint_key: foocache + checkpoint_cache: foocache tables: - foo `, dsn) @@ -410,7 +410,7 @@ mysql_cdc: dsn: %s stream_snapshot: true snapshot_max_batch_size: 500 - checkpoint_key: foocache + checkpoint_cache: foocache tables: - foo `, dsn) From 6643ea45fba95f92df4dd2ab336dca07263210d8 Mon Sep 17 00:00:00 2001 From: Tyler Rockwood Date: Tue, 10 Dec 2024 03:48:24 +0000 Subject: [PATCH 19/45] snapshot: cleanup context usage * use passed in context * don't hold onto a long lived context as struct member (golang anti-pattern) --- internal/impl/mysql/input_mysql_stream.go | 2 +- internal/impl/mysql/snapshot.go | 20 +++++++++----------- 2 files changed, 10 insertions(+), 12 deletions(-) diff --git a/internal/impl/mysql/input_mysql_stream.go b/internal/impl/mysql/input_mysql_stream.go index da45e5ea89..a48c9580a2 100644 --- a/internal/impl/mysql/input_mysql_stream.go +++ b/internal/impl/mysql/input_mysql_stream.go @@ -264,7 +264,7 @@ func (i *mysqlStreamInput) Connect(ctx context.Context) error { } // create snapshot instance - snapshot := NewSnapshot(ctx, i.logger, db) + snapshot := NewSnapshot(i.logger, db) i.snapshot = snapshot go i.startMySQLSync() diff --git a/internal/impl/mysql/snapshot.go b/internal/impl/mysql/snapshot.go index 5fa957783b..3d0199af88 100644 --- a/internal/impl/mysql/snapshot.go +++ b/internal/impl/mysql/snapshot.go @@ -28,14 +28,12 @@ type Snapshot struct { snapshotConn *sql.Conn logger *service.Logger - ctx context.Context } // NewSnapshot creates new snapshot instance -func NewSnapshot(ctx context.Context, logger *service.Logger, db *sql.DB) *Snapshot { +func NewSnapshot(logger *service.Logger, db *sql.DB) *Snapshot { return &Snapshot{ db: db, - ctx: ctx, logger: logger, } } @@ -95,7 +93,7 @@ func (s *Snapshot) prepareSnapshot(ctx context.Context) (*mysql.Position, error) } // Get binary log position (while locked) - pos, err := s.getCurrentBinlogPosition() + pos, err := s.getCurrentBinlogPosition(ctx) if err != nil { // Make sure to release the lock if we fail if _, eErr := s.lockConn.ExecContext(ctx, "UNLOCK TABLES"); eErr != nil { @@ -119,9 +117,9 @@ func (s *Snapshot) prepareSnapshot(ctx context.Context) (*mysql.Position, error) return &pos, nil } -func (s *Snapshot) getTablePrimaryKeys(table string) ([]string, error) { +func (s *Snapshot) getTablePrimaryKeys(ctx context.Context, table string) ([]string, error) { // Get primary key columns for the table - rows, err := s.tx.QueryContext(s.ctx, fmt.Sprintf(` + rows, err := s.tx.QueryContext(ctx, fmt.Sprintf(` SELECT COLUMN_NAME FROM information_schema.KEY_COLUMN_USAGE WHERE TABLE_NAME = '%s' AND CONSTRAINT_NAME = 'PRIMARY' @@ -146,7 +144,7 @@ ORDER BY ORDINAL_POSITION; return pks, nil } -func (s *Snapshot) querySnapshotTable(table string, pk []string, lastSeenPkVal *map[string]any, limit int) (*sql.Rows, error) { +func (s *Snapshot) querySnapshotTable(ctx context.Context, table string, pk []string, lastSeenPkVal *map[string]any, limit int) (*sql.Rows, error) { snapshotQueryParts := []string{ "SELECT * FROM " + table, } @@ -157,7 +155,7 @@ func (s *Snapshot) querySnapshotTable(table string, pk []string, lastSeenPkVal * snapshotQueryParts = append(snapshotQueryParts, "LIMIT ?") q := strings.Join(snapshotQueryParts, " ") s.logger.Infof("Querying snapshot: %s", q) - return s.tx.QueryContext(s.ctx, strings.Join(snapshotQueryParts, " "), limit) + return s.tx.QueryContext(ctx, strings.Join(snapshotQueryParts, " "), limit) } var lastSeenPkVals []any @@ -172,7 +170,7 @@ func (s *Snapshot) querySnapshotTable(table string, pk []string, lastSeenPkVal * snapshotQueryParts = append(snapshotQueryParts, fmt.Sprintf("LIMIT %d", limit)) q := strings.Join(snapshotQueryParts, " ") s.logger.Infof("Querying snapshot: %s", q) - return s.tx.QueryContext(s.ctx, q, lastSeenPkVals...) + return s.tx.QueryContext(ctx, q, lastSeenPkVals...) } func (s *Snapshot) buildOrderByClause(pk []string) string { @@ -183,7 +181,7 @@ func (s *Snapshot) buildOrderByClause(pk []string) string { return "ORDER BY " + strings.Join(pk, ", ") } -func (s *Snapshot) getCurrentBinlogPosition() (mysql.Position, error) { +func (s *Snapshot) getCurrentBinlogPosition(ctx context.Context) (mysql.Position, error) { var ( position uint32 file string @@ -194,7 +192,7 @@ func (s *Snapshot) getCurrentBinlogPosition() (mysql.Position, error) { executedGtidSet interface{} ) - row := s.snapshotConn.QueryRowContext(context.Background(), "SHOW MASTER STATUS") + row := s.snapshotConn.QueryRowContext(ctx, "SHOW MASTER STATUS") if err := row.Scan(&file, &position, &binlogDoDB, &binlogIgnoreDB, &executedGtidSet); err != nil { return mysql.Position{}, err } From 78af575a319ce3229ca1e57efadb60c112c12ff0 Mon Sep 17 00:00:00 2001 From: Tyler Rockwood Date: Tue, 10 Dec 2024 03:54:39 +0000 Subject: [PATCH 20/45] mysql: fold mode into operation We made a similar move with the postgres cdc component --- internal/impl/mysql/event.go | 13 ++----------- internal/impl/mysql/input_mysql_stream.go | 7 +++---- 2 files changed, 5 insertions(+), 15 deletions(-) diff --git a/internal/impl/mysql/event.go b/internal/impl/mysql/event.go index 949c6da3b6..dc68643ec8 100644 --- a/internal/impl/mysql/event.go +++ b/internal/impl/mysql/event.go @@ -14,6 +14,8 @@ import "github.com/go-mysql-org/go-mysql/mysql" type MessageOperation string const ( + // MessageOperationInsert represents read from snapshot + MessageOperationRead MessageOperation = "read" // MessageOperationInsert represents insert statement in mysql binlog MessageOperationInsert MessageOperation = "insert" // MessageOperationUpdate represents update statement in mysql binlog @@ -22,21 +24,10 @@ const ( MessageOperationDelete MessageOperation = "delete" ) -// MessageType is a base string type defining a type of the message -type MessageType string - -const ( - // MessageTypeSnapshot occures when plugin is processing existing snapshot data - MessageTypeSnapshot MessageType = "snapshot" - // MessageTypeStreaming occures when plugin is processing data from the binlog - MessageTypeStreaming MessageType = "streaming" -) - // MessageEvent represents a message from mysql cdc plugin type MessageEvent struct { Row map[string]any `json:"row"` Table string `json:"table"` Operation MessageOperation `json:"operation"` - Type MessageType `json:"type"` Position *mysql.Position `json:"position"` } diff --git a/internal/impl/mysql/input_mysql_stream.go b/internal/impl/mysql/input_mysql_stream.go index a48c9580a2..17ba38f1a6 100644 --- a/internal/impl/mysql/input_mysql_stream.go +++ b/internal/impl/mysql/input_mysql_stream.go @@ -298,11 +298,12 @@ func (i *mysqlStreamInput) readMessages(ctx context.Context) { mb := service.NewMessage(row) mb.MetaSet("operation", string(me.Operation)) mb.MetaSet("table", me.Table) - mb.MetaSet("type", string(me.Type)) if me.Position != nil { i.mutex.Lock() i.currentLogPosition = me.Position i.mutex.Unlock() + // Lexicographically ordered + mb.MetaSet("binlog_position", me.Position.String()) } if i.batchPolicy.Add(mb) { @@ -407,8 +408,7 @@ func (i *mysqlStreamInput) startMySQLSync() { // build message i.rawMessageEvents <- MessageEvent{ Row: row, - Operation: MessageOperationInsert, - Type: MessageTypeSnapshot, + Operation: MessageOperationRead, Table: table, Position: nil, } @@ -518,7 +518,6 @@ func (i *mysqlStreamInput) onMessage(e *canal.RowsEvent, initValue, incrementVal i.rawMessageEvents <- MessageEvent{ Row: message, Operation: MessageOperation(e.Action), - Type: MessageTypeStreaming, Table: e.Table.Name, Position: i.currentLogPosition, } From 786480cb69ecac047f3d9a1cd19c36e73464569b Mon Sep 17 00:00:00 2001 From: Tyler Rockwood Date: Tue, 10 Dec 2024 04:01:18 +0000 Subject: [PATCH 21/45] mysql: fix ack fn --- internal/impl/mysql/input_mysql_stream.go | 25 ++++++++++------------- 1 file changed, 11 insertions(+), 14 deletions(-) diff --git a/internal/impl/mysql/input_mysql_stream.go b/internal/impl/mysql/input_mysql_stream.go index 17ba38f1a6..1e99bf7482 100644 --- a/internal/impl/mysql/input_mysql_stream.go +++ b/internal/impl/mysql/input_mysql_stream.go @@ -351,7 +351,7 @@ func (i *mysqlStreamInput) startMySQLSync() { for _, table := range i.tables { wg.Go(func() (err error) { - tablePks, err := i.snapshot.getTablePrimaryKeys(table) + tablePks, err := i.snapshot.getTablePrimaryKeys(ctx, table) if err != nil { return err } @@ -367,9 +367,9 @@ func (i *mysqlStreamInput) startMySQLSync() { for { var batchRows *sql.Rows if numRowsProcessed == 0 { - batchRows, err = i.snapshot.querySnapshotTable(table, tablePks, nil, i.fieldSnapshotMaxBatchSize) + batchRows, err = i.snapshot.querySnapshotTable(ctx, table, tablePks, nil, i.fieldSnapshotMaxBatchSize) } else { - batchRows, err = i.snapshot.querySnapshotTable(table, tablePks, &lastSeenPksValues, i.fieldSnapshotMaxBatchSize) + batchRows, err = i.snapshot.querySnapshotTable(ctx, table, tablePks, &lastSeenPksValues, i.fieldSnapshotMaxBatchSize) } if err != nil { return err @@ -474,27 +474,24 @@ func (i *mysqlStreamInput) flushBatch(ctx context.Context, checkpointer *checkpo return false } - lastMessage := batch[len(batch)-1] - select { case i.msgChan <- asyncMessage{ msg: batch, ackFn: func(ctx context.Context, res error) error { maxOffset := resolveFn() + // Nothing to commit, this wasn't the latest message if maxOffset == nil { return nil } - - // do not call checkpoint if the last message in the batch is a snapshot - if msgType, ok := lastMessage.MetaGet("type"); ok && msgType == "snapshot" { + offset := *maxOffset + // This has no offset - it's a snapshot message + // TODO(rockwood): We should be storing the primary key for + // each table in the snapshot so we can properly resume the + // primary key scan. + if offset == nil { return nil } - - if err := i.syncBinlogPosition(ctx, *maxOffset); err != nil { - return err - } - - return nil + return i.syncBinlogPosition(ctx, offset) }, }: case <-ctx.Done(): From eead6dc1dae80ade36a6b5c101c1700fcf7a09f4 Mon Sep 17 00:00:00 2001 From: Tyler Rockwood Date: Tue, 10 Dec 2024 04:03:57 +0000 Subject: [PATCH 22/45] mysql: use error return type --- internal/impl/mysql/input_mysql_stream.go | 24 ++++++++++------------- 1 file changed, 10 insertions(+), 14 deletions(-) diff --git a/internal/impl/mysql/input_mysql_stream.go b/internal/impl/mysql/input_mysql_stream.go index 1e99bf7482..e22bd0389a 100644 --- a/internal/impl/mysql/input_mysql_stream.go +++ b/internal/impl/mysql/input_mysql_stream.go @@ -286,7 +286,7 @@ func (i *mysqlStreamInput) readMessages(ctx context.Context) { break } - if !i.flushBatch(ctx, i.cp, flushedBatch, i.currentLogPosition) { + if err := i.flushBatch(ctx, i.cp, flushedBatch, i.currentLogPosition); err != nil { break } case me := <-i.rawMessageEvents: @@ -313,7 +313,7 @@ func (i *mysqlStreamInput) readMessages(ctx context.Context) { i.logger.Debugf("Flush batch error: %w", err) break } - if !i.flushBatch(ctx, i.cp, flushedBatch, i.currentLogPosition) { + if err := i.flushBatch(ctx, i.cp, flushedBatch, i.currentLogPosition); err != nil { break } } else { @@ -455,9 +455,9 @@ func (i *mysqlStreamInput) startMySQLSync() { } } -func (i *mysqlStreamInput) flushBatch(ctx context.Context, checkpointer *checkpoint.Capped[*int64], batch service.MessageBatch, binLogPos *mysqlReplications.Position) bool { - if batch == nil { - return true +func (i *mysqlStreamInput) flushBatch(ctx context.Context, checkpointer *checkpoint.Capped[*int64], batch service.MessageBatch, binLogPos *mysqlReplications.Position) error { + if len(batch) == 0 { + return nil } var intPos *int64 @@ -468,10 +468,7 @@ func (i *mysqlStreamInput) flushBatch(ctx context.Context, checkpointer *checkpo resolveFn, err := checkpointer.Track(ctx, intPos, int64(len(batch))) if err != nil { - if ctx.Err() == nil { - i.logger.Errorf("Failed to checkpoint offset: %v\n", err) - } - return false + return fmt.Errorf("failed to track checkpoint for batch: %w", err) } select { @@ -494,11 +491,10 @@ func (i *mysqlStreamInput) flushBatch(ctx context.Context, checkpointer *checkpo return i.syncBinlogPosition(ctx, offset) }, }: + return nil case <-ctx.Done(): - return false + return ctx.Err() } - - return true } func (i *mysqlStreamInput) onMessage(e *canal.RowsEvent, initValue, incrementValue int) error { @@ -609,8 +605,8 @@ func (i *mysqlStreamInput) OnRotate(eh *replication.EventHeader, re *replication return err } - if ok := i.flushBatch(i.streamCtx, i.cp, flushedBatch, i.currentLogPosition); !ok { - return errors.New("failed to flush batch") + if err := i.flushBatch(i.streamCtx, i.cp, flushedBatch, i.currentLogPosition); err != nil { + return fmt.Errorf("failed to flush batch: %w", err) } i.currentLogPosition.Pos = uint32(re.Position) From 7498891bac0b5fa017ae30d041a5a2bb49a2e81e Mon Sep 17 00:00:00 2001 From: Tyler Rockwood Date: Tue, 10 Dec 2024 04:06:26 +0000 Subject: [PATCH 23/45] mysql: simplify constructor --- internal/impl/mysql/input_mysql_stream.go | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/internal/impl/mysql/input_mysql_stream.go b/internal/impl/mysql/input_mysql_stream.go index e22bd0389a..eaa4e5cc0f 100644 --- a/internal/impl/mysql/input_mysql_stream.go +++ b/internal/impl/mysql/input_mysql_stream.go @@ -200,11 +200,7 @@ func newMySQLStreamInput(conf *service.ParsedConfig, res *service.Resources) (s } func init() { - err := service.RegisterBatchInput( - "mysql_cdc", mysqlStreamConfigSpec, - func(conf *service.ParsedConfig, mgr *service.Resources) (service.BatchInput, error) { - return newMySQLStreamInput(conf, mgr) - }) + err := service.RegisterBatchInput("mysql_cdc", mysqlStreamConfigSpec, newMySQLStreamInput) if err != nil { panic(err) } @@ -598,7 +594,7 @@ func (i *mysqlStreamInput) Close(ctx context.Context) error { // --- MySQL Canal handler methods ---- func (i *mysqlStreamInput) OnRotate(eh *replication.EventHeader, re *replication.RotateEvent) error { - i.mutex.Lock() + i.mutex.Lock() // seems sketch flushedBatch, err := i.batchPolicy.Flush(i.streamCtx) if err != nil { i.logger.Debugf("Flush batch error: %w", err) From 3b3998d6009e587398d79c68b5206ea8345f37da Mon Sep 17 00:00:00 2001 From: Tyler Rockwood Date: Tue, 10 Dec 2024 04:41:29 +0000 Subject: [PATCH 24/45] mysql: cleaner import name --- internal/impl/mysql/input_mysql_stream.go | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/internal/impl/mysql/input_mysql_stream.go b/internal/impl/mysql/input_mysql_stream.go index eaa4e5cc0f..da6a005c0a 100644 --- a/internal/impl/mysql/input_mysql_stream.go +++ b/internal/impl/mysql/input_mysql_stream.go @@ -20,7 +20,7 @@ import ( "github.com/Jeffail/checkpoint" "github.com/Jeffail/shutdown" "github.com/go-mysql-org/go-mysql/canal" - mysqlReplications "github.com/go-mysql-org/go-mysql/mysql" + mysqlcdc "github.com/go-mysql-org/go-mysql/mysql" "github.com/go-mysql-org/go-mysql/replication" "github.com/go-sql-driver/mysql" "github.com/redpanda-data/benthos/v4/public/service" @@ -82,8 +82,8 @@ type mysqlStreamInput struct { // canal stands for mysql binlog listener connection canal *canal.Canal mysqlConfig *mysql.Config - startBinLogPosition *mysqlReplications.Position - currentLogPosition *mysqlReplications.Position + startBinLogPosition *mysqlcdc.Position + currentLogPosition *mysqlcdc.Position binLogCache string binLogCacheKey string @@ -210,7 +210,7 @@ func init() { func (i *mysqlStreamInput) Connect(ctx context.Context) error { canalConfig := canal.NewDefaultConfig() - canalConfig.Flavor = mysqlReplications.DEFAULT_FLAVOR + canalConfig.Flavor = mysqlcdc.DEFAULT_FLAVOR canalConfig.Addr = i.mysqlConfig.Addr canalConfig.User = i.mysqlConfig.User canalConfig.Password = i.mysqlConfig.Passwd @@ -235,7 +235,7 @@ func (i *mysqlStreamInput) Connect(ctx context.Context) error { return } - var storedMySQLBinLogPosition mysqlReplications.Position + var storedMySQLBinLogPosition mysqlcdc.Position if err := json.Unmarshal(binLogPositionBytes, &storedMySQLBinLogPosition); err != nil { i.logger.With("error", err.Error()).Error("Failed to unmarshal stored binlog position.") return @@ -451,7 +451,7 @@ func (i *mysqlStreamInput) startMySQLSync() { } } -func (i *mysqlStreamInput) flushBatch(ctx context.Context, checkpointer *checkpoint.Capped[*int64], batch service.MessageBatch, binLogPos *mysqlReplications.Position) error { +func (i *mysqlStreamInput) flushBatch(ctx context.Context, checkpointer *checkpoint.Capped[*int64], batch service.MessageBatch, binLogPos *mysqlcdc.Position) error { if len(batch) == 0 { return nil } From 586b387943d5169785b649a45c44eb9b4880a342 Mon Sep 17 00:00:00 2001 From: Tyler Rockwood Date: Tue, 10 Dec 2024 04:53:35 +0000 Subject: [PATCH 25/45] mysql: escape table regex --- internal/impl/mysql/event.go | 2 +- internal/impl/mysql/input_mysql_stream.go | 10 +++++++--- internal/impl/mysql/snapshot.go | 2 +- 3 files changed, 9 insertions(+), 5 deletions(-) diff --git a/internal/impl/mysql/event.go b/internal/impl/mysql/event.go index dc68643ec8..499b7c7ff7 100644 --- a/internal/impl/mysql/event.go +++ b/internal/impl/mysql/event.go @@ -14,7 +14,7 @@ import "github.com/go-mysql-org/go-mysql/mysql" type MessageOperation string const ( - // MessageOperationInsert represents read from snapshot + // MessageOperationRead represents read from snapshot MessageOperationRead MessageOperation = "read" // MessageOperationInsert represents insert statement in mysql binlog MessageOperationInsert MessageOperation = "insert" diff --git a/internal/impl/mysql/input_mysql_stream.go b/internal/impl/mysql/input_mysql_stream.go index da6a005c0a..f68f743efa 100644 --- a/internal/impl/mysql/input_mysql_stream.go +++ b/internal/impl/mysql/input_mysql_stream.go @@ -14,6 +14,7 @@ import ( "encoding/json" "errors" "fmt" + "regexp" "sync" "time" @@ -224,7 +225,10 @@ func (i *mysqlStreamInput) Connect(ctx context.Context) error { canalConfig.TLSConfig = i.mysqlConfig.TLS } canalConfig.ParseTime = true - canalConfig.IncludeTableRegex = i.tables + + for _, table := range i.tables { + canalConfig.IncludeTableRegex = append(canalConfig.IncludeTableRegex, regexp.QuoteMeta(table)) + } if err := i.res.AccessCache(ctx, i.binLogCache, func(c service.Cache) { binLogPositionBytes, cErr := c.Get(ctx, i.binLogCacheKey) @@ -382,8 +386,8 @@ func (i *mysqlStreamInput) startMySQLSync() { return err } - values := make([]interface{}, len(columns)) - valuePtrs := make([]interface{}, len(columns)) + values := make([]any, len(columns)) + valuePtrs := make([]any, len(columns)) for i := range values { valuePtrs[i] = &values[i] } diff --git a/internal/impl/mysql/snapshot.go b/internal/impl/mysql/snapshot.go index 3d0199af88..f7022d9913 100644 --- a/internal/impl/mysql/snapshot.go +++ b/internal/impl/mysql/snapshot.go @@ -203,7 +203,7 @@ func (s *Snapshot) getCurrentBinlogPosition(ctx context.Context) (mysql.Position }, nil } -func (s *Snapshot) releaseSnapshot(ctx context.Context) error { +func (s *Snapshot) releaseSnapshot(_ context.Context) error { if s.tx != nil { if err := s.tx.Commit(); err != nil { return fmt.Errorf("failed to commit transaction: %v", err) From c5328df5f112b99db1743ed70c26f7eaaf27f198 Mon Sep 17 00:00:00 2001 From: Tyler Rockwood Date: Tue, 10 Dec 2024 05:24:21 +0000 Subject: [PATCH 26/45] mysql: simplify checkpointer --- internal/impl/mysql/input_mysql_stream.go | 51 ++++++++--------------- 1 file changed, 17 insertions(+), 34 deletions(-) diff --git a/internal/impl/mysql/input_mysql_stream.go b/internal/impl/mysql/input_mysql_stream.go index f68f743efa..1092f48f9c 100644 --- a/internal/impl/mysql/input_mysql_stream.go +++ b/internal/impl/mysql/input_mysql_stream.go @@ -105,7 +105,7 @@ type mysqlStreamInput struct { streamCtx context.Context errors chan error - cp *checkpoint.Capped[*int64] + cp *checkpoint.Capped[*mysqlcdc.Position] snapshot *Snapshot shutSig *shutdown.Signaller @@ -455,18 +455,17 @@ func (i *mysqlStreamInput) startMySQLSync() { } } -func (i *mysqlStreamInput) flushBatch(ctx context.Context, checkpointer *checkpoint.Capped[*int64], batch service.MessageBatch, binLogPos *mysqlcdc.Position) error { +func (i *mysqlStreamInput) flushBatch( + ctx context.Context, + checkpointer *checkpoint.Capped[*mysqlcdc.Position], + batch service.MessageBatch, + binLogPos *mysqlcdc.Position, +) error { if len(batch) == 0 { return nil } - var intPos *int64 - if binLogPos != nil { - posInInt := int64(binLogPos.Pos) - intPos = &posInInt - } - - resolveFn, err := checkpointer.Track(ctx, intPos, int64(len(batch))) + resolveFn, err := checkpointer.Track(ctx, binLogPos, int64(len(batch))) if err != nil { return fmt.Errorf("failed to track checkpoint for batch: %w", err) } @@ -488,7 +487,7 @@ func (i *mysqlStreamInput) flushBatch(ctx context.Context, checkpointer *checkpo if offset == nil { return nil } - return i.syncBinlogPosition(ctx, offset) + return i.syncBinlogPosition(ctx, *offset) }, }: return nil @@ -519,40 +518,26 @@ func (i *mysqlStreamInput) onMessage(e *canal.RowsEvent, initValue, incrementVal return nil } -func (i *mysqlStreamInput) syncBinlogPosition(ctx context.Context, binLogPos *int64) error { +func (i *mysqlStreamInput) syncBinlogPosition(ctx context.Context, binLogPos mysqlcdc.Position) error { i.mutex.Lock() defer i.mutex.Unlock() - - if i.currentLogPosition == nil { - i.logger.Warn("No current bingLog position") - return errors.New("no current binlog position") - } - - if binLogPos != nil { - i.currentLogPosition.Pos = uint32(*binLogPos) - } - var ( positionInByte []byte err error ) - if positionInByte, err = json.Marshal(*i.currentLogPosition); err != nil { - i.logger.Errorf("Failed to marshal binlog position: %v", err) - return err + if positionInByte, err = json.Marshal(binLogPos); err != nil { + return fmt.Errorf("unable to serialize checkpoint: ", err) } - var cErr error if err := i.res.AccessCache(ctx, i.binLogCache, func(c service.Cache) { cErr = c.Set(ctx, i.binLogCacheKey, positionInByte, nil) - if cErr != nil { - i.logger.Errorf("Failed to store binlog position: %v", cErr) - } }); err != nil { - i.logger.Errorf("Access cache error %v", err) - return err + return fmt.Errorf("unable to access cache: %w", err) } - - return cErr + if cErr != nil { + return fmt.Errorf("unable persist checkpoint to cache: %w", cErr) + } + return nil } func (i *mysqlStreamInput) ReadBatch(ctx context.Context) (service.MessageBatch, service.AckFunc, error) { @@ -586,10 +571,8 @@ func (i *mysqlStreamInput) Close(ctx context.Context) error { } i.shutSig.TriggerHardStop() if i.canal != nil { - i.canal.SyncedPosition() i.canal.Close() } - return i.snapshot.close() } From b5fad95a0a03ce128c78904e8531f59aae421616 Mon Sep 17 00:00:00 2001 From: Tyler Rockwood Date: Tue, 10 Dec 2024 15:48:46 +0000 Subject: [PATCH 27/45] mycdc: use lexicographically ordered binlog position --- internal/impl/mysql/event.go | 29 +++++++++++++++- internal/impl/mysql/event_test.go | 42 +++++++++++++++++++++++ internal/impl/mysql/input_mysql_stream.go | 41 ++++++++-------------- 3 files changed, 84 insertions(+), 28 deletions(-) create mode 100644 internal/impl/mysql/event_test.go diff --git a/internal/impl/mysql/event.go b/internal/impl/mysql/event.go index 499b7c7ff7..1542f3eaa8 100644 --- a/internal/impl/mysql/event.go +++ b/internal/impl/mysql/event.go @@ -8,7 +8,13 @@ package mysql -import "github.com/go-mysql-org/go-mysql/mysql" +import ( + "fmt" + "strconv" + "strings" + + "github.com/go-mysql-org/go-mysql/mysql" +) // MessageOperation is a string type specifying message opration type MessageOperation string @@ -31,3 +37,24 @@ type MessageEvent struct { Operation MessageOperation `json:"operation"` Position *mysql.Position `json:"position"` } + +func binlogPositionToString(pos mysql.Position) string { + // Pad the position so this string is lexicographically ordered. + return fmt.Sprintf("%s@%08X", pos.Name, pos.Pos) +} + +func parseBinlogPosition(str string) (pos mysql.Position, err error) { + idx := strings.LastIndexByte(str, '@') + if idx == -1 { + err = fmt.Errorf("invalid binlog string: %s", str) + return + } + pos.Name = str[:idx] + var offset uint64 + offset, err = strconv.ParseUint(str[idx+1:], 16, 32) + pos.Pos = uint32(offset) + if err != nil { + err = fmt.Errorf("invalid binlog string offset: %w", err) + } + return +} diff --git a/internal/impl/mysql/event_test.go b/internal/impl/mysql/event_test.go new file mode 100644 index 0000000000..c516cd70a7 --- /dev/null +++ b/internal/impl/mysql/event_test.go @@ -0,0 +1,42 @@ +// Copyright 2024 Redpanda Data, Inc. +// +// Licensed as a Redpanda Enterprise file under the Redpanda Community +// License (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// https://github.com/redpanda-data/connect/v4/blob/main/licenses/rcl.md + +package mysql + +import ( + "math" + "strconv" + "testing" + + "github.com/go-mysql-org/go-mysql/mysql" + "github.com/stretchr/testify/require" +) + +func TestBinlogString(t *testing.T) { + good := []mysql.Position{ + {Name: "log.0000", Pos: 32}, + {Name: "log@0000", Pos: 32}, + {Name: "log.09999999", Pos: 0}, + {Name: "custom-binlog.9999999", Pos: math.MaxUint32}, + } + for _, expected := range good { + str := binlogPositionToString(expected) + actual, err := parseBinlogPosition(str) + require.NoError(t, err) + require.Equal(t, expected, actual) + } + bad := []string{ + "log.000", + "log.000@" + strconv.FormatUint(math.MaxUint64, 16), + "log.000.FF", + } + for _, str := range bad { + _, err := parseBinlogPosition(str) + require.Error(t, err) + } +} diff --git a/internal/impl/mysql/input_mysql_stream.go b/internal/impl/mysql/input_mysql_stream.go index 1092f48f9c..32ef8cf8fe 100644 --- a/internal/impl/mysql/input_mysql_stream.go +++ b/internal/impl/mysql/input_mysql_stream.go @@ -92,13 +92,12 @@ type mysqlStreamInput struct { tables []string streamSnapshot bool - rawMessageEvents chan MessageEvent - snapshotMessageEvents chan MessageEvent - msgChan chan asyncMessage - batching service.BatchPolicy - batchPolicy *service.Batcher - tablesFilterMap map[string]bool - checkPointLimit int + rawMessageEvents chan MessageEvent + msgChan chan asyncMessage + batching service.BatchPolicy + batchPolicy *service.Batcher + tablesFilterMap map[string]bool + checkPointLimit int logger *service.Logger res *service.Resources @@ -115,11 +114,10 @@ type mysqlStreamInput struct { func newMySQLStreamInput(conf *service.ParsedConfig, res *service.Resources) (s service.BatchInput, err error) { streamInput := mysqlStreamInput{ - logger: res.Logger(), - rawMessageEvents: make(chan MessageEvent), - snapshotMessageEvents: make(chan MessageEvent), - msgChan: make(chan asyncMessage), - res: res, + logger: res.Logger(), + rawMessageEvents: make(chan MessageEvent), + msgChan: make(chan asyncMessage), + res: res, errors: make(chan error, 1), shutSig: shutdown.NewSignaller(), @@ -167,7 +165,7 @@ func newMySQLStreamInput(conf *service.ParsedConfig, res *service.Resources) (s } i := &streamInput - i.cp = checkpoint.NewCapped[*int64](int64(i.checkPointLimit)) + i.cp = checkpoint.NewCapped[*mysqlcdc.Position](int64(i.checkPointLimit)) i.tablesFilterMap = map[string]bool{} for _, table := range i.tables { @@ -302,8 +300,7 @@ func (i *mysqlStreamInput) readMessages(ctx context.Context) { i.mutex.Lock() i.currentLogPosition = me.Position i.mutex.Unlock() - // Lexicographically ordered - mb.MetaSet("binlog_position", me.Position.String()) + mb.MetaSet("binlog_position", binlogPositionToString(*me.Position)) } if i.batchPolicy.Add(mb) { @@ -526,7 +523,7 @@ func (i *mysqlStreamInput) syncBinlogPosition(ctx context.Context, binLogPos mys err error ) if positionInByte, err = json.Marshal(binLogPos); err != nil { - return fmt.Errorf("unable to serialize checkpoint: ", err) + return fmt.Errorf("unable to serialize checkpoint: %w ", err) } var cErr error if err := i.res.AccessCache(ctx, i.binLogCache, func(c service.Cache) { @@ -541,18 +538,8 @@ func (i *mysqlStreamInput) syncBinlogPosition(ctx context.Context, binLogPos mys } func (i *mysqlStreamInput) ReadBatch(ctx context.Context) (service.MessageBatch, service.AckFunc, error) { - i.mutex.Lock() - msgChan := i.msgChan - i.mutex.Unlock() - if msgChan == nil { - return nil, nil, service.ErrNotConnected - } - select { - case m, open := <-msgChan: - if !open { - return nil, nil, service.ErrNotConnected - } + case m := <-i.msgChan: return m.msg, m.ackFn, nil case <-i.shutSig.HasStoppedChan(): return nil, nil, service.ErrNotConnected From 5555c92d63f6f1cb500e3fdad26cabe56ba50c69 Mon Sep 17 00:00:00 2001 From: Tyler Rockwood Date: Tue, 10 Dec 2024 17:31:05 +0000 Subject: [PATCH 28/45] mycdc: draw the rest of the owl Clean up the control flow and error handling to be simpler --- internal/impl/mysql/event.go | 8 +- internal/impl/mysql/event_test.go | 3 +- internal/impl/mysql/input_mysql_stream.go | 518 ++++++++++------------ internal/impl/mysql/snapshot.go | 15 +- 4 files changed, 253 insertions(+), 291 deletions(-) diff --git a/internal/impl/mysql/event.go b/internal/impl/mysql/event.go index 1542f3eaa8..4dbabe8593 100644 --- a/internal/impl/mysql/event.go +++ b/internal/impl/mysql/event.go @@ -16,6 +16,8 @@ import ( "github.com/go-mysql-org/go-mysql/mysql" ) +type Position = mysql.Position + // MessageOperation is a string type specifying message opration type MessageOperation string @@ -35,15 +37,15 @@ type MessageEvent struct { Row map[string]any `json:"row"` Table string `json:"table"` Operation MessageOperation `json:"operation"` - Position *mysql.Position `json:"position"` + Position *Position `json:"position"` } -func binlogPositionToString(pos mysql.Position) string { +func binlogPositionToString(pos Position) string { // Pad the position so this string is lexicographically ordered. return fmt.Sprintf("%s@%08X", pos.Name, pos.Pos) } -func parseBinlogPosition(str string) (pos mysql.Position, err error) { +func parseBinlogPosition(str string) (pos Position, err error) { idx := strings.LastIndexByte(str, '@') if idx == -1 { err = fmt.Errorf("invalid binlog string: %s", str) diff --git a/internal/impl/mysql/event_test.go b/internal/impl/mysql/event_test.go index c516cd70a7..8ddb129830 100644 --- a/internal/impl/mysql/event_test.go +++ b/internal/impl/mysql/event_test.go @@ -13,12 +13,11 @@ import ( "strconv" "testing" - "github.com/go-mysql-org/go-mysql/mysql" "github.com/stretchr/testify/require" ) func TestBinlogString(t *testing.T) { - good := []mysql.Position{ + good := []Position{ {Name: "log.0000", Pos: 32}, {Name: "log@0000", Pos: 32}, {Name: "log.09999999", Pos: 0}, diff --git a/internal/impl/mysql/input_mysql_stream.go b/internal/impl/mysql/input_mysql_stream.go index 32ef8cf8fe..5332c5d080 100644 --- a/internal/impl/mysql/input_mysql_stream.go +++ b/internal/impl/mysql/input_mysql_stream.go @@ -21,7 +21,6 @@ import ( "github.com/Jeffail/checkpoint" "github.com/Jeffail/shutdown" "github.com/go-mysql-org/go-mysql/canal" - mysqlcdc "github.com/go-mysql-org/go-mysql/mysql" "github.com/go-mysql-org/go-mysql/replication" "github.com/go-sql-driver/mysql" "github.com/redpanda-data/benthos/v4/public/service" @@ -29,15 +28,14 @@ import ( ) const ( - fieldMySQLDSN = "dsn" - fieldMySQLTables = "tables" - fieldStreamSnapshot = "stream_snapshot" - fieldMaxSnapshotParallelTables = "max_snapshot_parallel_tables" - fieldSnapshotMaxBatchSize = "snapshot_max_batch_size" - fieldBatching = "batching" - fieldCheckpointKey = "checkpoint_key" - fieldCheckpointCache = "checkpoint_cache" - fieldCheckpointLimit = "checkpoint_limit" + fieldMySQLDSN = "dsn" + fieldMySQLTables = "tables" + fieldStreamSnapshot = "stream_snapshot" + fieldSnapshotMaxBatchSize = "snapshot_max_batch_size" + fieldBatching = "batching" + fieldCheckpointKey = "checkpoint_key" + fieldCheckpointCache = "checkpoint_cache" + fieldCheckpointLimit = "checkpoint_limit" shutdownTimeout = 5 * time.Second ) @@ -56,9 +54,6 @@ var mysqlStreamConfigSpec = service.NewConfigSpec(). service.NewStringField(fieldCheckpointKey). Description("The key to use to store the snapshot position in `"+fieldCheckpointCache+"`. An alternative key can be provided if multiple CDC inputs share the same cache."). Default("mysql_binlog_position"), - service.NewIntField(fieldMaxSnapshotParallelTables). - Description("Int specifies a number of tables to be streamed in parallel when taking a snapshot. If set to true, the connector will stream all tables in parallel. Otherwise, it will stream tables one by one."). - Default(1), service.NewIntField(fieldSnapshotMaxBatchSize). Description("The maximum number of rows to be streamed in a single batch when taking a snapshot."). Default(1000), @@ -81,35 +76,30 @@ type mysqlStreamInput struct { mutex sync.Mutex // canal stands for mysql binlog listener connection - canal *canal.Canal - mysqlConfig *mysql.Config - startBinLogPosition *mysqlcdc.Position - currentLogPosition *mysqlcdc.Position - binLogCache string - binLogCacheKey string + canal *canal.Canal + mysqlConfig *mysql.Config + binLogCache string + binLogCacheKey string + currentBinlogName string dsn string tables []string streamSnapshot bool - rawMessageEvents chan MessageEvent - msgChan chan asyncMessage - batching service.BatchPolicy - batchPolicy *service.Batcher - tablesFilterMap map[string]bool - checkPointLimit int + batching service.BatchPolicy + batchPolicy *service.Batcher + tablesFilterMap map[string]bool + checkPointLimit int + fieldSnapshotMaxBatchSize int logger *service.Logger res *service.Resources - streamCtx context.Context - errors chan error - cp *checkpoint.Capped[*mysqlcdc.Position] + rawMessageEvents chan MessageEvent + msgChan chan asyncMessage + cp *checkpoint.Capped[*Position] - snapshot *Snapshot - shutSig *shutdown.Signaller - snapshotMaxParallelTables int - fieldSnapshotMaxBatchSize int + shutSig *shutdown.Signaller } func newMySQLStreamInput(conf *service.ParsedConfig, res *service.Resources) (s service.BatchInput, err error) { @@ -118,9 +108,6 @@ func newMySQLStreamInput(conf *service.ParsedConfig, res *service.Resources) (s rawMessageEvents: make(chan MessageEvent), msgChan: make(chan asyncMessage), res: res, - - errors: make(chan error, 1), - shutSig: shutdown.NewSignaller(), } var batching service.BatchPolicy @@ -142,10 +129,6 @@ func newMySQLStreamInput(conf *service.ParsedConfig, res *service.Resources) (s return nil, err } - if streamInput.snapshotMaxParallelTables, err = conf.FieldInt(fieldMaxSnapshotParallelTables); err != nil { - return nil, err - } - if streamInput.fieldSnapshotMaxBatchSize, err = conf.FieldInt(fieldSnapshotMaxBatchSize); err != nil { return nil, err } @@ -165,7 +148,7 @@ func newMySQLStreamInput(conf *service.ParsedConfig, res *service.Resources) (s } i := &streamInput - i.cp = checkpoint.NewCapped[*mysqlcdc.Position](int64(i.checkPointLimit)) + i.cp = checkpoint.NewCapped[*Position](int64(i.checkPointLimit)) i.tablesFilterMap = map[string]bool{} for _, table := range i.tables { @@ -188,8 +171,6 @@ func newMySQLStreamInput(conf *service.ParsedConfig, res *service.Resources) (s batching.Count = 1 } - i.streamCtx, _ = i.shutSig.SoftStopCtx(context.Background()) - r, err := service.AutoRetryNacksBatchedToggled(conf, i) if err != nil { return nil, err @@ -209,7 +190,6 @@ func init() { func (i *mysqlStreamInput) Connect(ctx context.Context) error { canalConfig := canal.NewDefaultConfig() - canalConfig.Flavor = mysqlcdc.DEFAULT_FLAVOR canalConfig.Addr = i.mysqlConfig.Addr canalConfig.User = i.mysqlConfig.User canalConfig.Password = i.mysqlConfig.Passwd @@ -222,32 +202,13 @@ func (i *mysqlStreamInput) Connect(ctx context.Context) error { if i.mysqlConfig.TLS != nil { canalConfig.TLSConfig = i.mysqlConfig.TLS } + // Parse time values as time.Time values not strings canalConfig.ParseTime = true for _, table := range i.tables { canalConfig.IncludeTableRegex = append(canalConfig.IncludeTableRegex, regexp.QuoteMeta(table)) } - if err := i.res.AccessCache(ctx, i.binLogCache, func(c service.Cache) { - binLogPositionBytes, cErr := c.Get(ctx, i.binLogCacheKey) - if cErr != nil { - if !errors.Is(cErr, service.ErrKeyNotFound) { - i.logger.Errorf("failed to obtain cursor cache item. %v", cErr) - } - return - } - - var storedMySQLBinLogPosition mysqlcdc.Position - if err := json.Unmarshal(binLogPositionBytes, &storedMySQLBinLogPosition); err != nil { - i.logger.With("error", err.Error()).Error("Failed to unmarshal stored binlog position.") - return - } - - i.startBinLogPosition = &storedMySQLBinLogPosition - }); err != nil { - i.logger.With("error", err.Error()).Error("Failed to access cursor cache.") - } - c, err := canal.NewCanal(canalConfig) if err != nil { return err @@ -261,45 +222,156 @@ func (i *mysqlStreamInput) Connect(ctx context.Context) error { return err } + pos, err := i.getCachedBinlogPosition(ctx) // create snapshot instance - snapshot := NewSnapshot(i.logger, db) - i.snapshot = snapshot + var snapshot *Snapshot + if i.streamSnapshot && pos == nil { + snapshot = NewSnapshot(i.logger, db) + } + + // Reset the shutSig + sig := shutdown.NewSignaller() + i.shutSig = sig + go func() { + ctx, _ := sig.SoftStopCtx(context.Background()) + wg, ctx := errgroup.WithContext(ctx) + wg.Go(func() error { return i.readMessages(ctx) }) + wg.Go(func() error { return i.startMySQLSync(ctx, pos, snapshot) }) + if err := wg.Wait(); err != nil && !errors.Is(err, context.Canceled) { + i.logger.Errorf("error during MySQL CDC: %v", err) + } else { + i.logger.Info("successfully shutdown MySQL CDC stream") + } + sig.TriggerHasStopped() + }() - go i.startMySQLSync() return nil } -func (i *mysqlStreamInput) readMessages(ctx context.Context) { - var nextTimedBatchChan <-chan time.Time +func (i *mysqlStreamInput) startMySQLSync(ctx context.Context, pos *Position, snapshot *Snapshot) error { + // If we are given a snapshot, then we need to read it. + if snapshot != nil { + startPos, err := snapshot.prepareSnapshot(ctx) + if err != nil { + return fmt.Errorf("unable to prepare snapshot: %w", err) + } + if err = i.readSnapshot(ctx, snapshot); err != nil { + return fmt.Errorf("failed reading snapshot: %w", err) + } + if err = snapshot.releaseSnapshot(ctx); err != nil { + return fmt.Errorf("unable to release snapshot: %w", err) + } + pos = startPos + } else if pos == nil { + coords, err := i.canal.GetMasterPos() + if err != nil { + return fmt.Errorf("unable to get start binlog position: %w", err) + } + pos = &coords + } + i.logger.Infof("starting MySQL CDC stream from binlog %s at offset %d", pos.Name, pos.Pos) + i.canal.SetEventHandler(i) + if err := i.canal.RunFrom(*pos); err != nil { + return fmt.Errorf("failed to start streaming: %w", err) + } + return nil +} + +func (i *mysqlStreamInput) readSnapshot(ctx context.Context, snapshot *Snapshot) error { + // TODO(cdc): Process tables in parallel + for _, table := range i.tables { + tablePks, err := snapshot.getTablePrimaryKeys(ctx, table) + if err != nil { + return err + } + i.logger.Tracef("primary keys for table %s: %v", table, tablePks) + var numRowsProcessed int + lastSeenPksValues := map[string]any{} + for _, pk := range tablePks { + lastSeenPksValues[pk] = nil + } + for { + var batchRows *sql.Rows + if numRowsProcessed == 0 { + batchRows, err = snapshot.querySnapshotTable(ctx, table, tablePks, nil, i.fieldSnapshotMaxBatchSize) + } else { + batchRows, err = snapshot.querySnapshotTable(ctx, table, tablePks, &lastSeenPksValues, i.fieldSnapshotMaxBatchSize) + } + if err != nil { + return err + } + var batchRowsCount int + for batchRows.Next() { + numRowsProcessed++ + batchRowsCount++ + + columns, err := batchRows.Columns() + if err != nil { + batchRows.Close() + return err + } + + values := make([]any, len(columns)) + valuePtrs := make([]any, len(columns)) + for i := range values { + valuePtrs[i] = &values[i] + } - for !i.shutSig.IsSoftStopSignalled() { + if err := batchRows.Scan(valuePtrs...); err != nil { + batchRows.Close() + return err + } + + row := map[string]any{} + for idx, value := range values { + row[columns[idx]] = value + if _, ok := lastSeenPksValues[columns[idx]]; ok { + lastSeenPksValues[columns[idx]] = value + } + } + + i.rawMessageEvents <- MessageEvent{ + Row: row, + Operation: MessageOperationRead, + Table: table, + Position: nil, + } + } + // TODO(cdc): Save checkpoint + if batchRowsCount < i.fieldSnapshotMaxBatchSize { + break + } + } + } + return nil +} + +func (i *mysqlStreamInput) readMessages(ctx context.Context) error { + var nextTimedBatchChan <-chan time.Time + for { select { case <-ctx.Done(): - return + return ctx.Err() case <-nextTimedBatchChan: nextTimedBatchChan = nil flushedBatch, err := i.batchPolicy.Flush(ctx) if err != nil { - i.logger.Debugf("Timed flush batch error: %w", err) - break + return fmt.Errorf("timed flush batch error: %w", err) } - if err := i.flushBatch(ctx, i.cp, flushedBatch, i.currentLogPosition); err != nil { - break + if err := i.flushBatch(ctx, i.cp, flushedBatch); err != nil { + return fmt.Errorf("failed to flush periodic batch: %w", err) } case me := <-i.rawMessageEvents: row, err := json.Marshal(me.Row) if err != nil { - return + return fmt.Errorf("failed to serialize row: %w", err) } mb := service.NewMessage(row) mb.MetaSet("operation", string(me.Operation)) mb.MetaSet("table", me.Table) if me.Position != nil { - i.mutex.Lock() - i.currentLogPosition = me.Position - i.mutex.Unlock() mb.MetaSet("binlog_position", binlogPositionToString(*me.Position)) } @@ -307,11 +379,10 @@ func (i *mysqlStreamInput) readMessages(ctx context.Context) { nextTimedBatchChan = nil flushedBatch, err := i.batchPolicy.Flush(ctx) if err != nil { - i.logger.Debugf("Flush batch error: %w", err) - break + return fmt.Errorf("flush batch error: %w", err) } - if err := i.flushBatch(ctx, i.cp, flushedBatch, i.currentLogPosition); err != nil { - break + if err := i.flushBatch(ctx, i.cp, flushedBatch); err != nil { + return fmt.Errorf("failed to flush batch: %w", err) } } else { d, ok := i.batchPolicy.UntilNext() @@ -319,158 +390,39 @@ func (i *mysqlStreamInput) readMessages(ctx context.Context) { nextTimedBatchChan = time.After(d) } } - case err := <-i.errors: - i.logger.Warnf("stream error: %s", err) - i.shutSig.TriggerSoftStop() - // If the stream has errored then we should stop and restart processing - return - } - } -} - -func (i *mysqlStreamInput) startMySQLSync() { - ctx, _ := i.shutSig.SoftStopCtx(context.Background()) - - i.canal.SetEventHandler(i) - go i.readMessages(ctx) - // If we require snapshot streaming && we don't have a binlog position cache - // initiate default run for Canal to process snapshot and start incremental sync of binlog - if i.streamSnapshot && i.startBinLogPosition == nil { - startPos, err := i.snapshot.prepareSnapshot(ctx) - if err != nil { - i.errors <- err - return - } - - i.logger.Debugf("Starting snapshot while holding binglog pos on: %v", startPos) - var wg errgroup.Group - wg.SetLimit(i.snapshotMaxParallelTables) - - for _, table := range i.tables { - wg.Go(func() (err error) { - tablePks, err := i.snapshot.getTablePrimaryKeys(ctx, table) - if err != nil { - return err - } - - i.logger.Debugf("Primary keys for table %s %v", table, tablePks) - - var numRowsProcessed int - lastSeenPksValues := map[string]any{} - for _, pk := range tablePks { - lastSeenPksValues[pk] = nil - } - - for { - var batchRows *sql.Rows - if numRowsProcessed == 0 { - batchRows, err = i.snapshot.querySnapshotTable(ctx, table, tablePks, nil, i.fieldSnapshotMaxBatchSize) - } else { - batchRows, err = i.snapshot.querySnapshotTable(ctx, table, tablePks, &lastSeenPksValues, i.fieldSnapshotMaxBatchSize) - } - if err != nil { - return err - } - - var batchRowsCount int - for batchRows.Next() { - numRowsProcessed++ - batchRowsCount++ - - columns, err := batchRows.Columns() - if err != nil { - batchRows.Close() - return err - } - - values := make([]any, len(columns)) - valuePtrs := make([]any, len(columns)) - for i := range values { - valuePtrs[i] = &values[i] - } - - if err := batchRows.Scan(valuePtrs...); err != nil { - batchRows.Close() - return err - } - - row := map[string]any{} - for idx, value := range values { - row[columns[idx]] = value - if _, ok := lastSeenPksValues[columns[idx]]; ok { - lastSeenPksValues[columns[idx]] = value - } - } - - // build message - i.rawMessageEvents <- MessageEvent{ - Row: row, - Operation: MessageOperationRead, - Table: table, - Position: nil, - } - } - - if batchRowsCount < i.fieldSnapshotMaxBatchSize { - break - } - } - - return nil - }) - } - - if err = wg.Wait(); err != nil { - i.errors <- fmt.Errorf("snapshot processing failed: %w", err) - } - - if err = i.snapshot.releaseSnapshot(ctx); err != nil { - i.logger.Errorf("Failed to properly release snapshot %v", err) - } - - i.mutex.Lock() - i.currentLogPosition = startPos - i.mutex.Unlock() - - i.logger.Infof("Snapshot is done...Running CDC from BingLog: %s on pos: %d", startPos.Name, startPos.Pos) - if err := i.canal.RunFrom(*startPos); err != nil { - i.errors <- fmt.Errorf("failed to start streaming: %v", err) - } - } else { - coords, _ := i.canal.GetMasterPos() - i.mutex.Lock() - // starting from the last stored binlog position - if i.startBinLogPosition != nil { - coords = *i.startBinLogPosition - } - - i.currentLogPosition = &coords - i.mutex.Lock() - if err := i.canal.RunFrom(coords); err != nil { - i.errors <- fmt.Errorf("failed to start streaming: %v", err) } } } func (i *mysqlStreamInput) flushBatch( ctx context.Context, - checkpointer *checkpoint.Capped[*mysqlcdc.Position], + checkpointer *checkpoint.Capped[*Position], batch service.MessageBatch, - binLogPos *mysqlcdc.Position, ) error { if len(batch) == 0 { return nil } + lastMsg := batch[len(batch)-1] + strPosition, ok := lastMsg.MetaGet("binlog_position") + var binLogPos *Position + if ok { + pos, err := parseBinlogPosition(strPosition) + if err != nil { + return err + } + binLogPos = &pos + } + resolveFn, err := checkpointer.Track(ctx, binLogPos, int64(len(batch))) if err != nil { return fmt.Errorf("failed to track checkpoint for batch: %w", err) } - - select { - case i.msgChan <- asyncMessage{ + msg := asyncMessage{ msg: batch, ackFn: func(ctx context.Context, res error) error { + i.mutex.Lock() + defer i.mutex.Unlock() maxOffset := resolveFn() // Nothing to commit, this wasn't the latest message if maxOffset == nil { @@ -478,65 +430,23 @@ func (i *mysqlStreamInput) flushBatch( } offset := *maxOffset // This has no offset - it's a snapshot message - // TODO(rockwood): We should be storing the primary key for + // TODO(cdc): We should be storing the primary key for // each table in the snapshot so we can properly resume the // primary key scan. if offset == nil { return nil } - return i.syncBinlogPosition(ctx, *offset) + return i.setCachedBinlogPosition(ctx, *offset) }, - }: + } + select { + case i.msgChan <- msg: return nil case <-ctx.Done(): return ctx.Err() } } -func (i *mysqlStreamInput) onMessage(e *canal.RowsEvent, initValue, incrementValue int) error { - i.mutex.Lock() - i.currentLogPosition.Pos = e.Header.LogPos - i.mutex.Unlock() - - for pi := initValue; pi < len(e.Rows); pi += incrementValue { - message := map[string]any{} - for i, v := range e.Rows[pi] { - message[e.Table.Columns[i].Name] = v - } - - i.rawMessageEvents <- MessageEvent{ - Row: message, - Operation: MessageOperation(e.Action), - Table: e.Table.Name, - Position: i.currentLogPosition, - } - } - - return nil -} - -func (i *mysqlStreamInput) syncBinlogPosition(ctx context.Context, binLogPos mysqlcdc.Position) error { - i.mutex.Lock() - defer i.mutex.Unlock() - var ( - positionInByte []byte - err error - ) - if positionInByte, err = json.Marshal(binLogPos); err != nil { - return fmt.Errorf("unable to serialize checkpoint: %w ", err) - } - var cErr error - if err := i.res.AccessCache(ctx, i.binLogCache, func(c service.Cache) { - cErr = c.Set(ctx, i.binLogCacheKey, positionInByte, nil) - }); err != nil { - return fmt.Errorf("unable to access cache: %w", err) - } - if cErr != nil { - return fmt.Errorf("unable persist checkpoint to cache: %w", cErr) - } - return nil -} - func (i *mysqlStreamInput) ReadBatch(ctx context.Context) (service.MessageBatch, service.AckFunc, error) { select { case m := <-i.msgChan: @@ -545,11 +455,13 @@ func (i *mysqlStreamInput) ReadBatch(ctx context.Context) (service.MessageBatch, return nil, nil, service.ErrNotConnected case <-ctx.Done(): } - return nil, nil, ctx.Err() } func (i *mysqlStreamInput) Close(ctx context.Context) error { + if i.shutSig == nil { + return nil // Never connected + } i.shutSig.TriggerSoftStop() select { case <-ctx.Done(): @@ -560,29 +472,62 @@ func (i *mysqlStreamInput) Close(ctx context.Context) error { if i.canal != nil { i.canal.Close() } - return i.snapshot.close() + select { + case <-ctx.Done(): + case <-time.After(shutdownTimeout): + case <-i.shutSig.HasStoppedChan(): + } + return nil } -// ---- Redpanda Connect specific methods end---- +// ---- input methods end ---- -// --- MySQL Canal handler methods ---- +// ---- cache methods start ---- -func (i *mysqlStreamInput) OnRotate(eh *replication.EventHeader, re *replication.RotateEvent) error { - i.mutex.Lock() // seems sketch - flushedBatch, err := i.batchPolicy.Flush(i.streamCtx) - if err != nil { - i.logger.Debugf("Flush batch error: %w", err) - return err +func (i *mysqlStreamInput) getCachedBinlogPosition(ctx context.Context) (*Position, error) { + var ( + cacheVal []byte + cErr error + ) + if err := i.res.AccessCache(ctx, i.binLogCache, func(c service.Cache) { + cacheVal, cErr = c.Get(ctx, i.binLogCacheKey) + }); err != nil { + return nil, fmt.Errorf("unable to access cache: %w", err) + } + if cErr != nil { + return nil, fmt.Errorf("unable persist checkpoint to cache: %w", cErr) } + if cacheVal == nil { + return nil, nil + } + pos, err := parseBinlogPosition(string(cacheVal)) + return &pos, err +} - if err := i.flushBatch(i.streamCtx, i.cp, flushedBatch, i.currentLogPosition); err != nil { - return fmt.Errorf("failed to flush batch: %w", err) +func (i *mysqlStreamInput) setCachedBinlogPosition(ctx context.Context, binLogPos Position) error { + var cErr error + if err := i.res.AccessCache(ctx, i.binLogCache, func(c service.Cache) { + cErr = c.Set( + ctx, + i.binLogCacheKey, + []byte(binlogPositionToString(binLogPos)), + nil, + ) + }); err != nil { + return fmt.Errorf("unable to access cache: %w", err) } + if cErr != nil { + return fmt.Errorf("unable persist checkpoint to cache: %w", cErr) + } + return nil +} + +// ---- cache methods end ---- - i.currentLogPosition.Pos = uint32(re.Position) - i.currentLogPosition.Name = string(re.NextLogName) - i.mutex.Unlock() +// --- MySQL Canal handler methods ---- +func (i *mysqlStreamInput) OnRotate(eh *replication.EventHeader, re *replication.RotateEvent) error { + i.currentBinlogName = string(re.NextLogName) return nil } @@ -590,6 +535,7 @@ func (i *mysqlStreamInput) OnRow(e *canal.RowsEvent) error { if _, ok := i.tablesFilterMap[e.Table.Name]; !ok { return nil } + i.logger.Infof("got rows (action=%s, rows=%d)", e.Action, len(e.Rows)) switch e.Action { case canal.InsertAction: return i.onMessage(e, 0, 1) @@ -602,4 +548,20 @@ func (i *mysqlStreamInput) OnRow(e *canal.RowsEvent) error { } } +func (i *mysqlStreamInput) onMessage(e *canal.RowsEvent, initValue, incrementValue int) error { + for pi := initValue; pi < len(e.Rows); pi += incrementValue { + message := map[string]any{} + for i, v := range e.Rows[pi] { + message[e.Table.Columns[i].Name] = v + } + i.rawMessageEvents <- MessageEvent{ + Row: message, + Operation: MessageOperation(e.Action), + Table: e.Table.Name, + Position: &Position{Name: i.currentBinlogName, Pos: e.Header.LogPos}, + } + } + return nil +} + // --- MySQL Canal handler methods end ---- diff --git a/internal/impl/mysql/snapshot.go b/internal/impl/mysql/snapshot.go index f7022d9913..d95a9246a6 100644 --- a/internal/impl/mysql/snapshot.go +++ b/internal/impl/mysql/snapshot.go @@ -14,7 +14,6 @@ import ( "fmt" "strings" - "github.com/go-mysql-org/go-mysql/mysql" "github.com/redpanda-data/benthos/v4/public/service" ) @@ -38,7 +37,7 @@ func NewSnapshot(logger *service.Logger, db *sql.DB) *Snapshot { } } -func (s *Snapshot) prepareSnapshot(ctx context.Context) (*mysql.Position, error) { +func (s *Snapshot) prepareSnapshot(ctx context.Context) (*Position, error) { var err error // Create a separate connection for FTWRL s.lockConn, err = s.db.Conn(ctx) @@ -181,23 +180,23 @@ func (s *Snapshot) buildOrderByClause(pk []string) string { return "ORDER BY " + strings.Join(pk, ", ") } -func (s *Snapshot) getCurrentBinlogPosition(ctx context.Context) (mysql.Position, error) { +func (s *Snapshot) getCurrentBinlogPosition(ctx context.Context) (Position, error) { var ( position uint32 file string // binlogDoDB, binlogIgnoreDB intentionally non-used // required to scan response - binlogDoDB interface{} - binlogIgnoreDB interface{} - executedGtidSet interface{} + binlogDoDB any + binlogIgnoreDB any + executedGtidSet any ) row := s.snapshotConn.QueryRowContext(ctx, "SHOW MASTER STATUS") if err := row.Scan(&file, &position, &binlogDoDB, &binlogIgnoreDB, &executedGtidSet); err != nil { - return mysql.Position{}, err + return Position{}, err } - return mysql.Position{ + return Position{ Name: file, Pos: position, }, nil From dbbad4a51ac215f3ae88c7dbb68b2b79d3dc02a3 Mon Sep 17 00:00:00 2001 From: Tyler Rockwood Date: Tue, 10 Dec 2024 17:53:14 +0000 Subject: [PATCH 29/45] mycdc: make the linter happy --- internal/impl/mysql/event.go | 8 +++--- internal/impl/mysql/event_test.go | 2 +- internal/impl/mysql/input_mysql_stream.go | 31 +++++++++++++++-------- internal/impl/mysql/snapshot.go | 16 ++++++------ 4 files changed, 34 insertions(+), 23 deletions(-) diff --git a/internal/impl/mysql/event.go b/internal/impl/mysql/event.go index 4dbabe8593..a2f7fd0748 100644 --- a/internal/impl/mysql/event.go +++ b/internal/impl/mysql/event.go @@ -16,7 +16,7 @@ import ( "github.com/go-mysql-org/go-mysql/mysql" ) -type Position = mysql.Position +type position = mysql.Position // MessageOperation is a string type specifying message opration type MessageOperation string @@ -37,15 +37,15 @@ type MessageEvent struct { Row map[string]any `json:"row"` Table string `json:"table"` Operation MessageOperation `json:"operation"` - Position *Position `json:"position"` + Position *position `json:"position"` } -func binlogPositionToString(pos Position) string { +func binlogPositionToString(pos position) string { // Pad the position so this string is lexicographically ordered. return fmt.Sprintf("%s@%08X", pos.Name, pos.Pos) } -func parseBinlogPosition(str string) (pos Position, err error) { +func parseBinlogPosition(str string) (pos position, err error) { idx := strings.LastIndexByte(str, '@') if idx == -1 { err = fmt.Errorf("invalid binlog string: %s", str) diff --git a/internal/impl/mysql/event_test.go b/internal/impl/mysql/event_test.go index 8ddb129830..f6b3a5bfd3 100644 --- a/internal/impl/mysql/event_test.go +++ b/internal/impl/mysql/event_test.go @@ -17,7 +17,7 @@ import ( ) func TestBinlogString(t *testing.T) { - good := []Position{ + good := []position{ {Name: "log.0000", Pos: 32}, {Name: "log@0000", Pos: 32}, {Name: "log.09999999", Pos: 0}, diff --git a/internal/impl/mysql/input_mysql_stream.go b/internal/impl/mysql/input_mysql_stream.go index 5332c5d080..7981f9dfc3 100644 --- a/internal/impl/mysql/input_mysql_stream.go +++ b/internal/impl/mysql/input_mysql_stream.go @@ -97,7 +97,7 @@ type mysqlStreamInput struct { rawMessageEvents chan MessageEvent msgChan chan asyncMessage - cp *checkpoint.Capped[*Position] + cp *checkpoint.Capped[*position] shutSig *shutdown.Signaller } @@ -148,7 +148,7 @@ func newMySQLStreamInput(conf *service.ParsedConfig, res *service.Resources) (s } i := &streamInput - i.cp = checkpoint.NewCapped[*Position](int64(i.checkPointLimit)) + i.cp = checkpoint.NewCapped[*position](int64(i.checkPointLimit)) i.tablesFilterMap = map[string]bool{} for _, table := range i.tables { @@ -204,6 +204,7 @@ func (i *mysqlStreamInput) Connect(ctx context.Context) error { } // Parse time values as time.Time values not strings canalConfig.ParseTime = true + // canalConfig.Logger for _, table := range i.tables { canalConfig.IncludeTableRegex = append(canalConfig.IncludeTableRegex, regexp.QuoteMeta(table)) @@ -223,7 +224,10 @@ func (i *mysqlStreamInput) Connect(ctx context.Context) error { } pos, err := i.getCachedBinlogPosition(ctx) - // create snapshot instance + if err != nil { + return fmt.Errorf("unable to get cached binlog position: %w", err) + } + // create snapshot instance if we were requested and haven't finished it before. var snapshot *Snapshot if i.streamSnapshot && pos == nil { snapshot = NewSnapshot(i.logger, db) @@ -248,19 +252,25 @@ func (i *mysqlStreamInput) Connect(ctx context.Context) error { return nil } -func (i *mysqlStreamInput) startMySQLSync(ctx context.Context, pos *Position, snapshot *Snapshot) error { +func (i *mysqlStreamInput) startMySQLSync(ctx context.Context, pos *position, snapshot *Snapshot) error { // If we are given a snapshot, then we need to read it. if snapshot != nil { startPos, err := snapshot.prepareSnapshot(ctx) if err != nil { + _ = snapshot.close() return fmt.Errorf("unable to prepare snapshot: %w", err) } if err = i.readSnapshot(ctx, snapshot); err != nil { + _ = snapshot.close() return fmt.Errorf("failed reading snapshot: %w", err) } if err = snapshot.releaseSnapshot(ctx); err != nil { + _ = snapshot.close() return fmt.Errorf("unable to release snapshot: %w", err) } + if err = snapshot.close(); err != nil { + return fmt.Errorf("unable to close snapshot: %w", err) + } pos = startPos } else if pos == nil { coords, err := i.canal.GetMasterPos() @@ -270,6 +280,7 @@ func (i *mysqlStreamInput) startMySQLSync(ctx context.Context, pos *Position, sn pos = &coords } i.logger.Infof("starting MySQL CDC stream from binlog %s at offset %d", pos.Name, pos.Pos) + i.currentBinlogName = pos.Name i.canal.SetEventHandler(i) if err := i.canal.RunFrom(*pos); err != nil { return fmt.Errorf("failed to start streaming: %w", err) @@ -396,7 +407,7 @@ func (i *mysqlStreamInput) readMessages(ctx context.Context) error { func (i *mysqlStreamInput) flushBatch( ctx context.Context, - checkpointer *checkpoint.Capped[*Position], + checkpointer *checkpoint.Capped[*position], batch service.MessageBatch, ) error { if len(batch) == 0 { @@ -405,7 +416,7 @@ func (i *mysqlStreamInput) flushBatch( lastMsg := batch[len(batch)-1] strPosition, ok := lastMsg.MetaGet("binlog_position") - var binLogPos *Position + var binLogPos *position if ok { pos, err := parseBinlogPosition(strPosition) if err != nil { @@ -484,7 +495,7 @@ func (i *mysqlStreamInput) Close(ctx context.Context) error { // ---- cache methods start ---- -func (i *mysqlStreamInput) getCachedBinlogPosition(ctx context.Context) (*Position, error) { +func (i *mysqlStreamInput) getCachedBinlogPosition(ctx context.Context) (*position, error) { var ( cacheVal []byte cErr error @@ -504,7 +515,7 @@ func (i *mysqlStreamInput) getCachedBinlogPosition(ctx context.Context) (*Positi return &pos, err } -func (i *mysqlStreamInput) setCachedBinlogPosition(ctx context.Context, binLogPos Position) error { +func (i *mysqlStreamInput) setCachedBinlogPosition(ctx context.Context, binLogPos position) error { var cErr error if err := i.res.AccessCache(ctx, i.binLogCache, func(c service.Cache) { cErr = c.Set( @@ -535,7 +546,7 @@ func (i *mysqlStreamInput) OnRow(e *canal.RowsEvent) error { if _, ok := i.tablesFilterMap[e.Table.Name]; !ok { return nil } - i.logger.Infof("got rows (action=%s, rows=%d)", e.Action, len(e.Rows)) + // i.logger.Infof("got rows (action=%s, rows=%d)", e.Action, len(e.Rows)) switch e.Action { case canal.InsertAction: return i.onMessage(e, 0, 1) @@ -558,7 +569,7 @@ func (i *mysqlStreamInput) onMessage(e *canal.RowsEvent, initValue, incrementVal Row: message, Operation: MessageOperation(e.Action), Table: e.Table.Name, - Position: &Position{Name: i.currentBinlogName, Pos: e.Header.LogPos}, + Position: &position{Name: i.currentBinlogName, Pos: e.Header.LogPos}, } } return nil diff --git a/internal/impl/mysql/snapshot.go b/internal/impl/mysql/snapshot.go index d95a9246a6..4a13b1d6b9 100644 --- a/internal/impl/mysql/snapshot.go +++ b/internal/impl/mysql/snapshot.go @@ -37,7 +37,7 @@ func NewSnapshot(logger *service.Logger, db *sql.DB) *Snapshot { } } -func (s *Snapshot) prepareSnapshot(ctx context.Context) (*Position, error) { +func (s *Snapshot) prepareSnapshot(ctx context.Context) (*position, error) { var err error // Create a separate connection for FTWRL s.lockConn, err = s.db.Conn(ctx) @@ -180,10 +180,10 @@ func (s *Snapshot) buildOrderByClause(pk []string) string { return "ORDER BY " + strings.Join(pk, ", ") } -func (s *Snapshot) getCurrentBinlogPosition(ctx context.Context) (Position, error) { +func (s *Snapshot) getCurrentBinlogPosition(ctx context.Context) (position, error) { var ( - position uint32 - file string + offset uint32 + file string // binlogDoDB, binlogIgnoreDB intentionally non-used // required to scan response binlogDoDB any @@ -192,13 +192,13 @@ func (s *Snapshot) getCurrentBinlogPosition(ctx context.Context) (Position, erro ) row := s.snapshotConn.QueryRowContext(ctx, "SHOW MASTER STATUS") - if err := row.Scan(&file, &position, &binlogDoDB, &binlogIgnoreDB, &executedGtidSet); err != nil { - return Position{}, err + if err := row.Scan(&file, &offset, &binlogDoDB, &binlogIgnoreDB, &executedGtidSet); err != nil { + return position{}, err } - return Position{ + return position{ Name: file, - Pos: position, + Pos: offset, }, nil } From 63c320e68e11bd6d41a0014b3f3f2a121676f2e8 Mon Sep 17 00:00:00 2001 From: Tyler Rockwood Date: Tue, 10 Dec 2024 18:33:16 +0000 Subject: [PATCH 30/45] mycdc: cleanup integration tests --- internal/impl/mysql/integration_test.go | 304 ++++++++---------------- 1 file changed, 96 insertions(+), 208 deletions(-) diff --git a/internal/impl/mysql/integration_test.go b/internal/impl/mysql/integration_test.go index 8394db3d35..60ebb4fcde 100644 --- a/internal/impl/mysql/integration_test.go +++ b/internal/impl/mysql/integration_test.go @@ -28,74 +28,90 @@ import ( "github.com/stretchr/testify/require" ) +type testDB struct { + *sql.DB + + t *testing.T +} + +func (db *testDB) Exec(query string, args ...any) { + _, err := db.DB.Exec(query, args...) + require.NoError(db.t, err) +} + +func setupTestWithMySQLVersion(t *testing.T, version string) (string, *testDB) { + integration.CheckSkip(t) + pool, err := dockertest.NewPool("") + require.NoError(t, err) + + pool.MaxWait = time.Minute + + // MySQL specific environment variables + resource, err := pool.RunWithOptions(&dockertest.RunOptions{ + Repository: "mysql", + Tag: version, + Env: []string{ + "MYSQL_ROOT_PASSWORD=password", + "MYSQL_DATABASE=testdb", + }, + Cmd: []string{ + "--server-id=1", + "--log-bin=mysql-bin", + "--binlog-format=ROW", + "--binlog-row-image=FULL", + "--log-slave-updates=ON", + }, + ExposedPorts: []string{"3306/tcp"}, + }, func(config *docker.HostConfig) { + // set AutoRemove to true so that stopped container goes away by itself + config.AutoRemove = true + config.RestartPolicy = docker.RestartPolicy{ + Name: "no", + } + }) + require.NoError(t, err) + t.Cleanup(func() { + assert.NoError(t, pool.Purge(resource)) + }) + + port := resource.GetPort("3306/tcp") + dsn := fmt.Sprintf( + "root:password@tcp(localhost:%s)/testdb?parseTime=true&timeout=30s&readTimeout=30s&writeTimeout=30s&multiStatements=true", + port, + ) + + var db *sql.DB + err = pool.Retry(func() error { + var err error + db, err = sql.Open("mysql", dsn) + if err != nil { + return err + } + + db.SetMaxOpenConns(10) + db.SetMaxIdleConns(5) + db.SetConnMaxLifetime(time.Minute * 5) + + return db.Ping() + }) + require.NoError(t, err) + t.Cleanup(func() { + assert.NoError(t, db.Close()) + }) + return dsn, &testDB{db, t} +} + func TestIntegrationMySQLCDC(t *testing.T) { integration.CheckSkip(t) var mysqlTestVersions = []string{"8.0", "9.0", "9.1"} for _, version := range mysqlTestVersions { - pool, err := dockertest.NewPool("") - require.NoError(t, err) - - pool.MaxWait = time.Minute - - // MySQL specific environment variables - resource, err := pool.RunWithOptions(&dockertest.RunOptions{ - Repository: "mysql", - Tag: version, - Env: []string{ - "MYSQL_ROOT_PASSWORD=password", - "MYSQL_DATABASE=testdb", - }, - Cmd: []string{ - "--server-id=1", - "--log-bin=mysql-bin", - "--binlog-format=ROW", - "--binlog-row-image=FULL", - "--log-slave-updates=ON", - }, - ExposedPorts: []string{"3306/tcp"}, - }, func(config *docker.HostConfig) { - // set AutoRemove to true so that stopped container goes away by itself - config.AutoRemove = true - config.RestartPolicy = docker.RestartPolicy{ - Name: "no", - } - }) - require.NoError(t, err) - t.Cleanup(func() { - assert.NoError(t, pool.Purge(resource)) - }) - - port := resource.GetPort("3306/tcp") - dsn := fmt.Sprintf( - "root:password@tcp(localhost:%s)/testdb?parseTime=true&timeout=30s&readTimeout=30s&writeTimeout=30s&multiStatements=true", - port, - ) - - var db *sql.DB - err = pool.Retry(func() error { - var err error - db, err = sql.Open("mysql", dsn) - if err != nil { - return err - } - - db.SetMaxOpenConns(10) - db.SetMaxIdleConns(5) - db.SetConnMaxLifetime(time.Minute * 5) - - return db.Ping() - }) - require.NoError(t, err) - + dsn, db := setupTestWithMySQLVersion(t, version) // Create table - _, err = db.Exec(` + db.Exec(` CREATE TABLE IF NOT EXISTS foo ( a INT PRIMARY KEY ) `) - require.NoError(t, err) - tmpDir := t.TempDir() - template := fmt.Sprintf(` mysql_cdc: dsn: %s @@ -108,7 +124,7 @@ mysql_cdc: cacheConf := fmt.Sprintf(` label: foocache file: - directory: %s`, tmpDir) + directory: %s`, t.TempDir()) streamOutBuilder := service.NewStreamBuilder() require.NoError(t, streamOutBuilder.SetLoggerYAML(`level: INFO`)) @@ -137,9 +153,7 @@ file: time.Sleep(time.Second * 5) for i := 0; i < 1000; i++ { // Insert 10000 rows - if _, err = db.Exec("INSERT INTO foo VALUES (?)", i); err != nil { - require.NoError(t, err) - } + db.Exec("INSERT INTO foo VALUES (?)", i) } assert.Eventually(t, func() bool { @@ -170,8 +184,7 @@ file: time.Sleep(time.Second) for i := 1001; i < 2001; i++ { - _, err = db.Exec("INSERT INTO foo VALUES (?)", i) - require.NoError(t, err) + db.Exec("INSERT INTO foo VALUES (?)", i) } go func() { @@ -190,75 +203,16 @@ file: } func TestIntegrationMySQLSnapshotAndCDC(t *testing.T) { - integration.CheckSkip(t) - pool, err := dockertest.NewPool("") - require.NoError(t, err) - - pool.MaxWait = time.Minute - - // MySQL specific environment variables - resource, err := pool.RunWithOptions(&dockertest.RunOptions{ - Repository: "mysql", - Tag: "8.0", - Env: []string{ - "MYSQL_ROOT_PASSWORD=password", - "MYSQL_DATABASE=testdb", - }, - Cmd: []string{ - "--server-id=1", - "--log-bin=mysql-bin", - "--binlog-format=ROW", - "--binlog-row-image=FULL", - "--log-slave-updates=ON", - }, - ExposedPorts: []string{"3306/tcp"}, - }, func(config *docker.HostConfig) { - // set AutoRemove to true so that stopped container goes away by itself - config.AutoRemove = true - config.RestartPolicy = docker.RestartPolicy{ - Name: "no", - } - }) - require.NoError(t, err) - t.Cleanup(func() { - assert.NoError(t, pool.Purge(resource)) - }) - - port := resource.GetPort("3306/tcp") - dsn := fmt.Sprintf( - "root:password@tcp(localhost:%s)/testdb?parseTime=true&timeout=30s&readTimeout=30s&writeTimeout=30s&multiStatements=true", - port, - ) - - var db *sql.DB - err = pool.Retry(func() error { - var err error - db, err = sql.Open("mysql", dsn) - if err != nil { - return err - } - - db.SetMaxOpenConns(10) - db.SetMaxIdleConns(5) - db.SetConnMaxLifetime(time.Minute * 5) - - return db.Ping() - }) - require.NoError(t, err) - + dsn, db := setupTestWithMySQLVersion(t, "8.0") // Create table - _, err = db.Exec(` + db.Exec(` CREATE TABLE IF NOT EXISTS foo ( a INT PRIMARY KEY ) `) - require.NoError(t, err) - tmpDir := t.TempDir() - // Insert 1000 rows for initial snapshot streaming for i := 0; i < 1000; i++ { - _, err = db.Exec("INSERT INTO foo VALUES (?)", i) - require.NoError(t, err) + db.Exec("INSERT INTO foo VALUES (?)", i) } template := fmt.Sprintf(` @@ -274,7 +228,7 @@ mysql_cdc: cacheConf := fmt.Sprintf(` label: foocache file: - directory: %s`, tmpDir) + directory: %s`, t.TempDir()) streamOutBuilder := service.NewStreamBuilder() require.NoError(t, streamOutBuilder.SetLoggerYAML(`level: DEBUG`)) @@ -303,9 +257,7 @@ file: time.Sleep(time.Second * 5) for i := 1000; i < 2000; i++ { // Insert 10000 rows - if _, err = db.Exec("INSERT INTO foo VALUES (?)", i); err != nil { - require.NoError(t, err) - } + db.Exec("INSERT INTO foo VALUES (?)", i) } assert.Eventually(t, func() bool { @@ -318,91 +270,31 @@ file: } func TestIntegrationMySQLCDCWithCompositePrimaryKeys(t *testing.T) { - integration.CheckSkip(t) - pool, err := dockertest.NewPool("") - require.NoError(t, err) - - pool.MaxWait = time.Minute - - // MySQL specific environment variables - resource, err := pool.RunWithOptions(&dockertest.RunOptions{ - Repository: "mysql", - Tag: "8.0", - Env: []string{ - "MYSQL_ROOT_PASSWORD=password", - "MYSQL_DATABASE=testdb", - }, - Cmd: []string{ - "--server-id=1", - "--log-bin=mysql-bin", - "--binlog-format=ROW", - "--binlog-row-image=FULL", - "--log-slave-updates=ON", - }, - ExposedPorts: []string{"3306/tcp"}, - }, func(config *docker.HostConfig) { - // set AutoRemove to true so that stopped container goes away by itself - config.AutoRemove = true - config.RestartPolicy = docker.RestartPolicy{ - Name: "no", - } - }) - require.NoError(t, err) - t.Cleanup(func() { - assert.NoError(t, pool.Purge(resource)) - }) - - port := resource.GetPort("3306/tcp") - dsn := fmt.Sprintf( - "root:password@tcp(localhost:%s)/testdb?parseTime=true&timeout=30s&readTimeout=30s&writeTimeout=30s&multiStatements=true", - port, - ) - - var db *sql.DB - err = pool.Retry(func() error { - var err error - db, err = sql.Open("mysql", dsn) - if err != nil { - return err - } - - db.SetMaxOpenConns(10) - db.SetMaxIdleConns(5) - db.SetConnMaxLifetime(time.Minute * 5) - - return db.Ping() - }) - require.NoError(t, err) - + dsn, db := setupTestWithMySQLVersion(t, "8.0") // Create table - _, err = db.Exec(` + db.Exec(` CREATE TABLE IF NOT EXISTS foo ( a INT, b INT, + v JSON, + size ENUM('x-small', 'small', 'medium', 'large', 'x-large'), PRIMARY KEY (a, b) ) `) - require.NoError(t, err) - - // Create controll table to ensure we don't stream it - _, err = db.Exec(` + // Create control table to ensure we don't stream it + db.Exec(` CREATE TABLE IF NOT EXISTS foo_non_streamed ( a INT, b INT, + v JSON, PRIMARY KEY (a, b) ) `) - require.NoError(t, err) - tmpDir := t.TempDir() - // Insert 1000 rows for initial snapshot streaming for i := 0; i < 1000; i++ { - _, err = db.Exec("INSERT INTO foo VALUES (?, ?)", i, i) - require.NoError(t, err) - - _, err = db.Exec("INSERT INTO foo_non_streamed VALUES (?, ?)", i, i) - require.NoError(t, err) + db.Exec("INSERT INTO foo VALUES (?, ?, ?, ?)", i, i, `{"json":"data"}`, `large`) + db.Exec("INSERT INTO foo_non_streamed VALUES (?, ?, ?)", i, i, `{"json":"data"}`) } template := fmt.Sprintf(` @@ -418,7 +310,7 @@ mysql_cdc: cacheConf := fmt.Sprintf(` label: foocache file: - directory: %s`, tmpDir) + directory: %s`, t.TempDir()) streamOutBuilder := service.NewStreamBuilder() require.NoError(t, streamOutBuilder.SetLoggerYAML(`level: DEBUG`)) @@ -435,6 +327,7 @@ file: outBatchMut.Unlock() return nil })) + fmt.Println(outBatches) streamOut, err := streamOutBuilder.Build() require.NoError(t, err) @@ -447,12 +340,8 @@ file: time.Sleep(time.Second * 5) for i := 1000; i < 2000; i++ { // Insert 10000 rows - if _, err = db.Exec("INSERT INTO foo VALUES (?, ?)", i, i); err != nil { - require.NoError(t, err) - } - - _, err = db.Exec("INSERT INTO foo_non_streamed VALUES (?, ?)", i, i) - require.NoError(t, err) + db.Exec("INSERT INTO foo VALUES (?, ?, ?, ?)", i, i, `{"json":"data"}`, `x-small`) + db.Exec("INSERT INTO foo_non_streamed VALUES (?, ?, ?)", i, i, `{"json":"data"}`) } assert.Eventually(t, func() bool { @@ -460,6 +349,5 @@ file: defer outBatchMut.Unlock() return len(outBatches) == 2000 }, time.Minute*5, time.Millisecond*100) - require.NoError(t, streamOut.StopWithin(time.Second*10)) } From b06d218398c2bb8899078e4e07b687e5dea20972 Mon Sep 17 00:00:00 2001 From: Tyler Rockwood Date: Tue, 10 Dec 2024 18:34:00 +0000 Subject: [PATCH 31/45] mycdc: decode stream message by type Need to do for snapshot phase too --- internal/impl/mysql/input_mysql_stream.go | 52 ++++++++++++++++++----- internal/impl/mysql/integration_test.go | 1 - 2 files changed, 42 insertions(+), 11 deletions(-) diff --git a/internal/impl/mysql/input_mysql_stream.go b/internal/impl/mysql/input_mysql_stream.go index 7981f9dfc3..857ddf4157 100644 --- a/internal/impl/mysql/input_mysql_stream.go +++ b/internal/impl/mysql/input_mysql_stream.go @@ -22,6 +22,7 @@ import ( "github.com/Jeffail/shutdown" "github.com/go-mysql-org/go-mysql/canal" "github.com/go-mysql-org/go-mysql/replication" + "github.com/go-mysql-org/go-mysql/schema" "github.com/go-sql-driver/mysql" "github.com/redpanda-data/benthos/v4/public/service" "golang.org/x/sync/errgroup" @@ -333,8 +334,15 @@ func (i *mysqlStreamInput) readSnapshot(ctx context.Context, snapshot *Snapshot) return err } + types, err := batchRows.ColumnTypes() + if err != nil { + batchRows.Close() + return err + } + row := map[string]any{} for idx, value := range values { + _ = types[idx] // TODO decode based on type row[columns[idx]] = value if _, ok := lastSeenPksValues[columns[idx]]; ok { lastSeenPksValues[columns[idx]] = value @@ -480,12 +488,10 @@ func (i *mysqlStreamInput) Close(ctx context.Context) error { case <-i.shutSig.HasStoppedChan(): } i.shutSig.TriggerHardStop() - if i.canal != nil { - i.canal.Close() - } select { case <-ctx.Done(): case <-time.After(shutdownTimeout): + i.logger.Error("failed to shutdown mysql_cdc within the timeout") case <-i.shutSig.HasStoppedChan(): } return nil @@ -503,12 +509,13 @@ func (i *mysqlStreamInput) getCachedBinlogPosition(ctx context.Context) (*positi if err := i.res.AccessCache(ctx, i.binLogCache, func(c service.Cache) { cacheVal, cErr = c.Get(ctx, i.binLogCacheKey) }); err != nil { - return nil, fmt.Errorf("unable to access cache: %w", err) - } - if cErr != nil { - return nil, fmt.Errorf("unable persist checkpoint to cache: %w", cErr) + return nil, fmt.Errorf("unable to access cache for reading: %w", err) } - if cacheVal == nil { + if errors.Is(cErr, service.ErrKeyNotFound) { + return nil, nil + } else if cErr != nil { + return nil, fmt.Errorf("unable read checkpoint from cache: %w", cErr) + } else if cacheVal == nil { return nil, nil } pos, err := parseBinlogPosition(string(cacheVal)) @@ -525,7 +532,7 @@ func (i *mysqlStreamInput) setCachedBinlogPosition(ctx context.Context, binLogPo nil, ) }); err != nil { - return fmt.Errorf("unable to access cache: %w", err) + return fmt.Errorf("unable to access cache for writing: %w", err) } if cErr != nil { return fmt.Errorf("unable persist checkpoint to cache: %w", cErr) @@ -563,7 +570,32 @@ func (i *mysqlStreamInput) onMessage(e *canal.RowsEvent, initValue, incrementVal for pi := initValue; pi < len(e.Rows); pi += incrementValue { message := map[string]any{} for i, v := range e.Rows[pi] { - message[e.Table.Columns[i].Name] = v + col := e.Table.Columns[i] + switch col.Type { + // TODO(cdc): support more column types + case schema.TYPE_ENUM: + if col.EnumValues != nil { + ordinal, ok := v.(int64) + if !ok { + return fmt.Errorf("expected int value for enum column got: %T", v) + } + if ordinal < 1 || int(ordinal) > len(col.EnumValues) { + return fmt.Errorf("enum ordinal out of range: %d when there are %d variants", ordinal, len(col.EnumValues)) + } + v = col.EnumValues[ordinal-1] + } + case schema.TYPE_JSON: + s, ok := v.(string) + if !ok { + return fmt.Errorf("expected string value for json column got: %T", v) + } + var decoded any + if err := json.Unmarshal([]byte(s), &decoded); err != nil { + return err + } + v = decoded + } + message[col.Name] = v } i.rawMessageEvents <- MessageEvent{ Row: message, diff --git a/internal/impl/mysql/integration_test.go b/internal/impl/mysql/integration_test.go index 60ebb4fcde..c703438242 100644 --- a/internal/impl/mysql/integration_test.go +++ b/internal/impl/mysql/integration_test.go @@ -327,7 +327,6 @@ file: outBatchMut.Unlock() return nil })) - fmt.Println(outBatches) streamOut, err := streamOutBuilder.Build() require.NoError(t, err) From 8ede4a6807f4a0e6e75f8020f6dc09ed0a8a626f Mon Sep 17 00:00:00 2001 From: Tyler Rockwood Date: Tue, 10 Dec 2024 20:42:54 +0000 Subject: [PATCH 32/45] mycdc: add more types --- internal/impl/mysql/input_mysql_stream.go | 64 +++-- internal/impl/mysql/integration_test.go | 285 ++++++++++++++++++++++ 2 files changed, 324 insertions(+), 25 deletions(-) diff --git a/internal/impl/mysql/input_mysql_stream.go b/internal/impl/mysql/input_mysql_stream.go index 857ddf4157..6b93a50dc1 100644 --- a/internal/impl/mysql/input_mysql_stream.go +++ b/internal/impl/mysql/input_mysql_stream.go @@ -342,8 +342,11 @@ func (i *mysqlStreamInput) readSnapshot(ctx context.Context, snapshot *Snapshot) row := map[string]any{} for idx, value := range values { - _ = types[idx] // TODO decode based on type - row[columns[idx]] = value + v, err := mapSnapshotMessageColumn(value, types[idx]) + if err != nil { + return err + } + row[columns[idx]] = v if _, ok := lastSeenPksValues[columns[idx]]; ok { lastSeenPksValues[columns[idx]] = value } @@ -365,6 +368,10 @@ func (i *mysqlStreamInput) readSnapshot(ctx context.Context, snapshot *Snapshot) return nil } +func mapSnapshotMessageColumn(v any, _ *sql.ColumnType) (any, error) { + return v, nil +} + func (i *mysqlStreamInput) readMessages(ctx context.Context) error { var nextTimedBatchChan <-chan time.Time for { @@ -571,29 +578,9 @@ func (i *mysqlStreamInput) onMessage(e *canal.RowsEvent, initValue, incrementVal message := map[string]any{} for i, v := range e.Rows[pi] { col := e.Table.Columns[i] - switch col.Type { - // TODO(cdc): support more column types - case schema.TYPE_ENUM: - if col.EnumValues != nil { - ordinal, ok := v.(int64) - if !ok { - return fmt.Errorf("expected int value for enum column got: %T", v) - } - if ordinal < 1 || int(ordinal) > len(col.EnumValues) { - return fmt.Errorf("enum ordinal out of range: %d when there are %d variants", ordinal, len(col.EnumValues)) - } - v = col.EnumValues[ordinal-1] - } - case schema.TYPE_JSON: - s, ok := v.(string) - if !ok { - return fmt.Errorf("expected string value for json column got: %T", v) - } - var decoded any - if err := json.Unmarshal([]byte(s), &decoded); err != nil { - return err - } - v = decoded + v, err := mapMessageColumn(v, col) + if err != nil { + return err } message[col.Name] = v } @@ -607,4 +594,31 @@ func (i *mysqlStreamInput) onMessage(e *canal.RowsEvent, initValue, incrementVal return nil } +func mapMessageColumn(v any, col schema.TableColumn) (any, error) { + switch col.Type { + // TODO(cdc): support more column types + case schema.TYPE_ENUM: + ordinal, ok := v.(int64) + if !ok { + return nil, fmt.Errorf("expected int value for enum column got: %T", v) + } + if ordinal < 1 || int(ordinal) > len(col.EnumValues) { + return nil, fmt.Errorf("enum ordinal out of range: %d when there are %d variants", ordinal, len(col.EnumValues)) + } + return col.EnumValues[ordinal-1], nil + case schema.TYPE_JSON: + s, ok := v.(string) + if !ok { + return nil, fmt.Errorf("expected string value for json column got: %T", v) + } + var decoded any + if err := json.Unmarshal([]byte(s), &decoded); err != nil { + return nil, err + } + return decoded, nil + default: + return v, nil + } +} + // --- MySQL Canal handler methods end ---- diff --git a/internal/impl/mysql/integration_test.go b/internal/impl/mysql/integration_test.go index c703438242..b016183ce2 100644 --- a/internal/impl/mysql/integration_test.go +++ b/internal/impl/mysql/integration_test.go @@ -350,3 +350,288 @@ file: }, time.Minute*5, time.Millisecond*100) require.NoError(t, streamOut.StopWithin(time.Second*10)) } + +func TestIntegrationMySQLCDCAllTypes(t *testing.T) { + dsn, db := setupTestWithMySQLVersion(t, "8.0") + // Create table + db.Exec(` + CREATE TABLE all_data_types ( + -- Numeric Data Types + tinyint_col TINYINT, + smallint_col SMALLINT, + mediumint_col MEDIUMINT, + int_col INT, + bigint_col BIGINT, + decimal_col DECIMAL(10, 2), + numeric_col NUMERIC(10, 2), + float_col FLOAT, + double_col DOUBLE, + + -- Date and Time Data Types + date_col DATE, + datetime_col DATETIME, + timestamp_col TIMESTAMP, + time_col TIME, + year_col YEAR, + + -- String Data Types + char_col CHAR(10), + varchar_col VARCHAR(255), + binary_col BINARY(10), + varbinary_col VARBINARY(255), + tinyblob_col TINYBLOB, + blob_col BLOB, + mediumblob_col MEDIUMBLOB, + longblob_col LONGBLOB, + tinytext_col TINYTEXT, + text_col TEXT, + mediumtext_col MEDIUMTEXT, + longtext_col LONGTEXT, + enum_col ENUM('option1', 'option2', 'option3'), + set_col SET('a', 'b', 'c', 'd') + + -- TODO(cdc): Spatial Data Types + -- geometry_col GEOMETRY, + -- point_col POINT, + -- linestring_col LINESTRING, + -- polygon_col POLYGON, + -- multipoint_col MULTIPOINT, + -- multilinestring_col MULTILINESTRING, + -- multipolygon_col MULTIPOLYGON, + -- geometrycollection_col GEOMETRYCOLLECTION +); +`) + + db.Exec(` +INSERT INTO all_data_types ( + tinyint_col, + smallint_col, + mediumint_col, + int_col, + bigint_col, + decimal_col, + numeric_col, + float_col, + double_col, + date_col, + datetime_col, + timestamp_col, + time_col, + year_col, + char_col, + varchar_col, + binary_col, + varbinary_col, + tinyblob_col, + blob_col, + mediumblob_col, + longblob_col, + tinytext_col, + text_col, + mediumtext_col, + longtext_col, + enum_col, + set_col +) VALUES ( + 127, -- tinyint_col + 32767, -- smallint_col + 8388607, -- mediumint_col + 2147483647, -- int_col + 9223372036854775807, -- bigint_col + 12345.67, -- decimal_col + 98765.43, -- numeric_col + 3.14, -- float_col + 2.718281828, -- double_col + '2024-12-10', -- date_col + '2024-12-10 15:30:45', -- datetime_col + CURRENT_TIMESTAMP, -- timestamp_col + '15:30:45', -- time_col + 2024, -- year_col + 'char_data', -- char_col + 'varchar_data', -- varchar_col + BINARY('binary'), -- binary_col + BINARY('varbinary'), -- varbinary_col + 'small blob', -- tinyblob_col + 'regular blob', -- blob_col + 'medium blob', -- mediumblob_col + 'large blob', -- longblob_col + 'tiny text', -- tinytext_col + 'regular text', -- text_col + 'medium text', -- mediumtext_col + 'large text', -- longtext_col + 'option1', -- enum_col + 'a,b' -- set_col +); + + `) + + template := fmt.Sprintf(` +mysql_cdc: + dsn: %s + stream_snapshot: true + snapshot_max_batch_size: 500 + checkpoint_cache: memcache + tables: + - foo +`, dsn) + + cacheConf := ` +label: memcache +memory: {} +` + + streamOutBuilder := service.NewStreamBuilder() + require.NoError(t, streamOutBuilder.SetLoggerYAML(`level: DEBUG`)) + require.NoError(t, streamOutBuilder.AddCacheYAML(cacheConf)) + require.NoError(t, streamOutBuilder.AddInputYAML(template)) + + var outBatches []string + var outBatchMut sync.Mutex + require.NoError(t, streamOutBuilder.AddBatchConsumerFunc(func(c context.Context, mb service.MessageBatch) error { + msgBytes, err := mb[0].AsBytes() + require.NoError(t, err) + outBatchMut.Lock() + outBatches = append(outBatches, string(msgBytes)) + outBatchMut.Unlock() + return nil + })) + + streamOut, err := streamOutBuilder.Build() + require.NoError(t, err) + + go func() { + err = streamOut.Run(context.Background()) + require.NoError(t, err) + }() + + time.Sleep(time.Second * 5) + + db.Exec(` + INSERT INTO all_data_types ( + tinyint_col, + smallint_col, + mediumint_col, + int_col, + bigint_col, + decimal_col, + numeric_col, + float_col, + double_col, + date_col, + datetime_col, + timestamp_col, + time_col, + year_col, + char_col, + varchar_col, + binary_col, + varbinary_col, + tinyblob_col, + blob_col, + mediumblob_col, + longblob_col, + tinytext_col, + text_col, + mediumtext_col, + longtext_col, + enum_col, + set_col +) VALUES ( + -128, -- tinyint_col + -32768, -- smallint_col + -8388608, -- mediumint_col + -2147483648, -- int_col + -9223372036854775808, -- bigint_col + 54321.12, -- decimal_col + 87654.21, -- numeric_col + 1.618, -- float_col + 3.141592653, -- double_col + '2023-01-01', -- date_col + '2023-01-01 12:00:00', -- datetime_col + '2023-01-01 12:00:00', -- timestamp_col + '23:59:59', -- time_col + 2023, -- year_col + 'example', -- char_col + 'another_example', -- varchar_col + BINARY('fixed'), -- binary_col + BINARY('dynamic'), -- varbinary_col + 'tiny_blob_value', -- tinyblob_col + 'blob_value', -- blob_col + 'medium_blob_value', -- mediumblob_col + 'long_blob_value', -- longblob_col + 'tiny_text_value', -- tinytext_col + 'text_value', -- text_col + 'medium_text_value', -- mediumtext_col + 'long_text_value', -- longtext_col + 'option2', -- enum_col + 'b,c' -- set_col +);`) + + assert.Eventually(t, func() bool { + outBatchMut.Lock() + defer outBatchMut.Unlock() + return len(outBatches) == 2 + }, time.Minute*5, time.Millisecond*100) + require.NoError(t, streamOut.StopWithin(time.Second*10)) + + require.JSONEq(t, outBatches[0], `{ + "tinyint_col": 127, + "smallint_col": 32767, + "mediumint_col": 8388607, + "int_col": 2147483647, + "bigint_col": 9223372036854775807, + "decimal_col": 12345.67, + "numeric_col": 98765.43, + "float_col": 3.14, + "double_col": 2.718281828, + "date_col": "2024-12-10", + "datetime_col": "2024-12-10 15:30:45", + "timestamp_col": "2024-12-10 15:30:45", + "time_col": "15:30:45", + "year_col": 2024, + "char_col": "char_data", + "varchar_col": "varchar_data", + "binary_col": "binary", + "varbinary_col": "varbinary", + "tinyblob_col": "small blob", + "blob_col": "regular blob", + "mediumblob_col": "medium blob", + "longblob_col": "large blob", + "tinytext_col": "tiny text", + "text_col": "regular text", + "mediumtext_col": "medium text", + "longtext_col": "large text", + "enum_col": "option1", + "set_col": ["a", "b"] +}`) + require.JSONEq(t, outBatches[1], `{ + "tinyint_col": -128, + "smallint_col": -32768, + "mediumint_col": -8388608, + "int_col": -2147483648, + "bigint_col": -9223372036854775808, + "decimal_col": 54321.12, + "numeric_col": 87654.21, + "float_col": 1.618, + "double_col": 3.141592653, + "date_col": "2023-01-01", + "datetime_col": "2023-01-01 12:00:00", + "timestamp_col": "2023-01-01 12:00:00", + "time_col": "23:59:59", + "year_col": 2023, + "char_col": "example", + "varchar_col": "another_example", + "binary_col": "fixed", + "varbinary_col": "dynamic", + "tinyblob_col": "tiny_blob_value", + "blob_col": "blob_value", + "mediumblob_col": "medium_blob_value", + "longblob_col": "long_blob_value", + "tinytext_col": "tiny_text_value", + "text_col": "text_value", + "mediumtext_col": "medium_text_value", + "longtext_col": "long_text_value", + "enum_col": "option2", + "set_col": ["b", "c"] +}`) +} From b0f8ea763f5ba05b6482c44cba1dbb28ecab2f9d Mon Sep 17 00:00:00 2001 From: Tyler Rockwood Date: Fri, 20 Dec 2024 15:15:41 +0000 Subject: [PATCH 33/45] mycdc: fix snapshot cleanup and missing pks --- internal/impl/mysql/input_mysql_stream.go | 9 ++++---- internal/impl/mysql/integration_test.go | 6 +++--- internal/impl/mysql/snapshot.go | 26 +++++++++++++++++------ 3 files changed, 26 insertions(+), 15 deletions(-) diff --git a/internal/impl/mysql/input_mysql_stream.go b/internal/impl/mysql/input_mysql_stream.go index 6b93a50dc1..7ff035ba12 100644 --- a/internal/impl/mysql/input_mysql_stream.go +++ b/internal/impl/mysql/input_mysql_stream.go @@ -219,11 +219,6 @@ func (i *mysqlStreamInput) Connect(ctx context.Context) error { i.canal = c - db, err := sql.Open("mysql", i.dsn) - if err != nil { - return err - } - pos, err := i.getCachedBinlogPosition(ctx) if err != nil { return fmt.Errorf("unable to get cached binlog position: %w", err) @@ -231,6 +226,10 @@ func (i *mysqlStreamInput) Connect(ctx context.Context) error { // create snapshot instance if we were requested and haven't finished it before. var snapshot *Snapshot if i.streamSnapshot && pos == nil { + db, err := sql.Open("mysql", i.dsn) + if err != nil { + return err + } snapshot = NewSnapshot(i.logger, db) } diff --git a/internal/impl/mysql/integration_test.go b/internal/impl/mysql/integration_test.go index b016183ce2..28942979eb 100644 --- a/internal/impl/mysql/integration_test.go +++ b/internal/impl/mysql/integration_test.go @@ -357,7 +357,7 @@ func TestIntegrationMySQLCDCAllTypes(t *testing.T) { db.Exec(` CREATE TABLE all_data_types ( -- Numeric Data Types - tinyint_col TINYINT, + tinyint_col TINYINT PRIMARY KEY, smallint_col SMALLINT, mediumint_col MEDIUMINT, int_col INT, @@ -472,7 +472,7 @@ mysql_cdc: snapshot_max_batch_size: 500 checkpoint_cache: memcache tables: - - foo + - all_data_types `, dsn) cacheConf := ` @@ -571,7 +571,7 @@ memory: {} outBatchMut.Lock() defer outBatchMut.Unlock() return len(outBatches) == 2 - }, time.Minute*5, time.Millisecond*100) + }, time.Second*30, time.Millisecond*100) require.NoError(t, streamOut.StopWithin(time.Second*10)) require.JSONEq(t, outBatches[0], `{ diff --git a/internal/impl/mysql/snapshot.go b/internal/impl/mysql/snapshot.go index 4a13b1d6b9..a5b3407079 100644 --- a/internal/impl/mysql/snapshot.go +++ b/internal/impl/mysql/snapshot.go @@ -136,10 +136,13 @@ ORDER BY ORDINAL_POSITION; if err := rows.Scan(&pk); err != nil { return nil, err } - pks = append(pks, pk) } + if len(pks) == 0 { + return nil, fmt.Errorf("unable to find primary key for table %s - does the table exist and does it have a primary key set?", table) + } + return pks, nil } @@ -215,13 +218,22 @@ func (s *Snapshot) releaseSnapshot(_ context.Context) error { } func (s *Snapshot) close() error { - if s.lockConn != nil { - return s.lockConn.Close() + if s.tx != nil { + if err := s.tx.Rollback(); err != nil { + return fmt.Errorf("unable to rollback transaction: %w", err) + } + s.tx = nil } - - if s.snapshotConn != nil { - return s.snapshotConn.Close() + for _, conn := range []*sql.Conn{s.lockConn, s.snapshotConn} { + if conn == nil { + continue + } + if err := conn.Close(); err != nil { + return fmt.Errorf("unable to close connection: %w", err) + } + } + if err := s.db.Close(); err != nil { + return fmt.Errorf("unable to close db: %w", err) } - return nil } From 30148d0838fef8f65fc7ac4933a4868a5e0d4482 Mon Sep 17 00:00:00 2001 From: Tyler Rockwood Date: Fri, 20 Dec 2024 15:28:56 +0000 Subject: [PATCH 34/45] mycdc: fix shutdown hang --- internal/impl/mysql/input_mysql_stream.go | 17 +++++++++++++---- 1 file changed, 13 insertions(+), 4 deletions(-) diff --git a/internal/impl/mysql/input_mysql_stream.go b/internal/impl/mysql/input_mysql_stream.go index 7ff035ba12..2a43a6f3b3 100644 --- a/internal/impl/mysql/input_mysql_stream.go +++ b/internal/impl/mysql/input_mysql_stream.go @@ -239,6 +239,11 @@ func (i *mysqlStreamInput) Connect(ctx context.Context) error { go func() { ctx, _ := sig.SoftStopCtx(context.Background()) wg, ctx := errgroup.WithContext(ctx) + wg.Go(func() error { + <-ctx.Done() + i.canal.Close() + return nil + }) wg.Go(func() error { return i.readMessages(ctx) }) wg.Go(func() error { return i.startMySQLSync(ctx, pos, snapshot) }) if err := wg.Wait(); err != nil && !errors.Is(err, context.Canceled) { @@ -318,7 +323,7 @@ func (i *mysqlStreamInput) readSnapshot(ctx context.Context, snapshot *Snapshot) columns, err := batchRows.Columns() if err != nil { - batchRows.Close() + _ = batchRows.Close() return err } @@ -329,13 +334,13 @@ func (i *mysqlStreamInput) readSnapshot(ctx context.Context, snapshot *Snapshot) } if err := batchRows.Scan(valuePtrs...); err != nil { - batchRows.Close() + _ = batchRows.Close() return err } types, err := batchRows.ColumnTypes() if err != nil { - batchRows.Close() + _ = batchRows.Close() return err } @@ -351,11 +356,15 @@ func (i *mysqlStreamInput) readSnapshot(ctx context.Context, snapshot *Snapshot) } } - i.rawMessageEvents <- MessageEvent{ + select { + case i.rawMessageEvents <- MessageEvent{ Row: row, Operation: MessageOperationRead, Table: table, Position: nil, + }: + case <-ctx.Done(): + return ctx.Err() } } // TODO(cdc): Save checkpoint From 243eee52358f1ece2b040ef4134c785013d72adb Mon Sep 17 00:00:00 2001 From: Tyler Rockwood Date: Fri, 20 Dec 2024 16:15:03 +0000 Subject: [PATCH 35/45] mycdc: fix snapshot types --- internal/impl/mysql/input_mysql_stream.go | 100 +++++++++++++++++----- internal/impl/mysql/integration_test.go | 20 ++--- 2 files changed, 88 insertions(+), 32 deletions(-) diff --git a/internal/impl/mysql/input_mysql_stream.go b/internal/impl/mysql/input_mysql_stream.go index 2a43a6f3b3..e242ddec0c 100644 --- a/internal/impl/mysql/input_mysql_stream.go +++ b/internal/impl/mysql/input_mysql_stream.go @@ -15,6 +15,7 @@ import ( "errors" "fmt" "regexp" + "strings" "sync" "time" @@ -316,37 +317,34 @@ func (i *mysqlStreamInput) readSnapshot(ctx context.Context, snapshot *Snapshot) if err != nil { return err } + + types, err := batchRows.ColumnTypes() + if err != nil { + _ = batchRows.Close() + return err + } + + values, mappers := prepSnapshotScannerAndMappers(types) + + columns, err := batchRows.Columns() + if err != nil { + _ = batchRows.Close() + return err + } + var batchRowsCount int for batchRows.Next() { numRowsProcessed++ batchRowsCount++ - columns, err := batchRows.Columns() - if err != nil { - _ = batchRows.Close() - return err - } - - values := make([]any, len(columns)) - valuePtrs := make([]any, len(columns)) - for i := range values { - valuePtrs[i] = &values[i] - } - - if err := batchRows.Scan(valuePtrs...); err != nil { - _ = batchRows.Close() - return err - } - - types, err := batchRows.ColumnTypes() - if err != nil { + if err := batchRows.Scan(values...); err != nil { _ = batchRows.Close() return err } row := map[string]any{} for idx, value := range values { - v, err := mapSnapshotMessageColumn(value, types[idx]) + v, err := mappers[idx](value) if err != nil { return err } @@ -376,10 +374,68 @@ func (i *mysqlStreamInput) readSnapshot(ctx context.Context, snapshot *Snapshot) return nil } -func mapSnapshotMessageColumn(v any, _ *sql.ColumnType) (any, error) { - return v, nil +func snapshotValueMapper[T any](v any) (any, error) { + s, ok := v.(*sql.Null[T]) + if !ok { + var e T + return nil, fmt.Errorf("expected %T got %T", e, v) + } + if !s.Valid { + return nil, nil + } + return s.V, nil } +func prepSnapshotScannerAndMappers(cols []*sql.ColumnType) (values []any, mappers []func(any) (any, error)) { + + for _, col := range cols { + var val any + var mapper func(any) (any, error) + switch col.DatabaseTypeName() { + case "BINARY", "VARBINARY", "TINYBLOB", "BLOB", "MEDIUMBLOB", "LONGBLOB": + val = new(sql.Null[[]byte]) + mapper = snapshotValueMapper[[]byte] + case "DATETIME", "TIMESTAMP": + val = new(sql.NullTime) + mapper = func(v any) (any, error) { + s, ok := v.(*sql.NullTime) + if !ok { + return nil, fmt.Errorf("expected %T got %T", time.Time{}, v) + } + if !s.Valid { + return nil, nil + } + return s.Time, nil + } + case "TINYINT", "SMALLINT", "MEDIUMINT", "INT", "BIGINT", "YEAR": + val = new(sql.Null[int]) + mapper = snapshotValueMapper[int] + case "FLOAT", "DOUBLE", "DECIMAL", "NUMERIC": + val = new(sql.Null[float64]) + mapper = snapshotValueMapper[float64] + case "SET": + val = new(sql.Null[string]) + mapper = func(v any) (any, error) { + s, ok := v.(*sql.Null[string]) + if !ok { + return nil, fmt.Errorf("expected %T got %T", "", v) + } + if !s.Valid { + return nil, nil + } + // This might be a little simplistic, we may need to handle escaped values + // here... + return strings.Split(s.V, ","), nil + } + default: + val = new(sql.Null[string]) + mapper = snapshotValueMapper[string] + } + values = append(values, val) + mappers = append(mappers, mapper) + } + return +} func (i *mysqlStreamInput) readMessages(ctx context.Context) error { var nextTimedBatchChan <-chan time.Time for { diff --git a/internal/impl/mysql/integration_test.go b/internal/impl/mysql/integration_test.go index 28942979eb..e5b27b6b1c 100644 --- a/internal/impl/mysql/integration_test.go +++ b/internal/impl/mysql/integration_test.go @@ -444,7 +444,7 @@ INSERT INTO all_data_types ( 2.718281828, -- double_col '2024-12-10', -- date_col '2024-12-10 15:30:45', -- datetime_col - CURRENT_TIMESTAMP, -- timestamp_col + '2024-12-10 15:30:46', -- timestamp_col '15:30:45', -- time_col 2024, -- year_col 'char_data', -- char_col @@ -584,19 +584,19 @@ memory: {} "numeric_col": 98765.43, "float_col": 3.14, "double_col": 2.718281828, - "date_col": "2024-12-10", - "datetime_col": "2024-12-10 15:30:45", - "timestamp_col": "2024-12-10 15:30:45", + "date_col": "2024-12-10T00:00:00Z", + "datetime_col": "2024-12-10T15:30:45Z", + "timestamp_col": "2024-12-10T15:30:46Z", "time_col": "15:30:45", "year_col": 2024, "char_col": "char_data", "varchar_col": "varchar_data", - "binary_col": "binary", - "varbinary_col": "varbinary", - "tinyblob_col": "small blob", - "blob_col": "regular blob", - "mediumblob_col": "medium blob", - "longblob_col": "large blob", + "binary_col": "YmluYXJ5AAAAAA==", + "varbinary_col": "dmFyYmluYXJ5", + "tinyblob_col": "c21hbGwgYmxvYg==", + "blob_col": "cmVndWxhciBibG9i", + "mediumblob_col": "bWVkaXVtIGJsb2I=", + "longblob_col": "bGFyZ2UgYmxvYg==", "tinytext_col": "tiny text", "text_col": "regular text", "mediumtext_col": "medium text", From 87986f8790667db9b993ad66d0b9d457d6cbfec5 Mon Sep 17 00:00:00 2001 From: Tyler Rockwood Date: Fri, 20 Dec 2024 20:09:02 +0000 Subject: [PATCH 36/45] mycdc: handle all types :tada: --- internal/impl/mysql/input_mysql_stream.go | 93 +++++++++++++++++++---- internal/impl/mysql/integration_test.go | 49 +++++++----- 2 files changed, 107 insertions(+), 35 deletions(-) diff --git a/internal/impl/mysql/input_mysql_stream.go b/internal/impl/mysql/input_mysql_stream.go index e242ddec0c..0de203f017 100644 --- a/internal/impl/mysql/input_mysql_stream.go +++ b/internal/impl/mysql/input_mysql_stream.go @@ -387,7 +387,18 @@ func snapshotValueMapper[T any](v any) (any, error) { } func prepSnapshotScannerAndMappers(cols []*sql.ColumnType) (values []any, mappers []func(any) (any, error)) { - + stringMapping := func(mapper func(s string) (any, error)) func(any) (any, error) { + return func(v any) (any, error) { + s, ok := v.(*sql.NullString) + if !ok { + return nil, fmt.Errorf("expected %T got %T", "", v) + } + if !s.Valid { + return nil, nil + } + return mapper(s.String) + } + } for _, col := range cols { var val any var mapper func(any) (any, error) @@ -410,23 +421,31 @@ func prepSnapshotScannerAndMappers(cols []*sql.ColumnType) (values []any, mapper case "TINYINT", "SMALLINT", "MEDIUMINT", "INT", "BIGINT", "YEAR": val = new(sql.Null[int]) mapper = snapshotValueMapper[int] - case "FLOAT", "DOUBLE", "DECIMAL", "NUMERIC": + case "DECIMAL", "NUMERIC": + val = new(sql.NullString) + mapper = stringMapping(func(s string) (any, error) { + return json.Number(s), nil + }) + case "FLOAT", "DOUBLE": val = new(sql.Null[float64]) mapper = snapshotValueMapper[float64] case "SET": - val = new(sql.Null[string]) - mapper = func(v any) (any, error) { - s, ok := v.(*sql.Null[string]) - if !ok { - return nil, fmt.Errorf("expected %T got %T", "", v) - } - if !s.Valid { - return nil, nil - } + val = new(sql.NullString) + mapper = stringMapping(func(s string) (any, error) { // This might be a little simplistic, we may need to handle escaped values // here... - return strings.Split(s.V, ","), nil - } + out := []any{} + for _, elem := range strings.Split(s, ",") { + out = append(out, elem) + } + return out, nil + }) + case "JSON": + val = new(sql.NullString) + mapper = stringMapping(func(s string) (v any, err error) { + err = json.Unmarshal([]byte(s), &v) + return + }) default: val = new(sql.Null[string]) mapper = snapshotValueMapper[string] @@ -660,7 +679,30 @@ func (i *mysqlStreamInput) onMessage(e *canal.RowsEvent, initValue, incrementVal func mapMessageColumn(v any, col schema.TableColumn) (any, error) { switch col.Type { - // TODO(cdc): support more column types + case schema.TYPE_DECIMAL: + s, ok := v.(string) + if !ok { + return nil, fmt.Errorf("expected string value for decimal column got: %T", v) + } + return json.Number(s), nil + case schema.TYPE_SET: + bitset, ok := v.(int64) + if !ok { + return nil, fmt.Errorf("expected int value for set column got: %T", v) + } + out := []any{} + for i, element := range col.SetValues { + if (bitset>>i)&1 == 1 { + out = append(out, element) + } + } + return out, nil + case schema.TYPE_DATE: + date, ok := v.(string) + if !ok { + return nil, fmt.Errorf("expected string value for date column got: %T", v) + } + return time.Parse("2006-01-02", date) case schema.TYPE_ENUM: ordinal, ok := v.(int64) if !ok { @@ -680,6 +722,29 @@ func mapMessageColumn(v any, col schema.TableColumn) (any, error) { return nil, err } return decoded, nil + case schema.TYPE_STRING: + // Blob types should come through as binary, but are marked type 5, + // instead skip them here and have those fallthrough to the binary case. + if !strings.Contains(col.RawType, "blob") { + if s, ok := v.(string); ok { + return s, nil + } + s, ok := v.([]byte) + if !ok { + return nil, fmt.Errorf("unexpected type for STRING column: %T", v) + } + return string(s), nil + } + fallthrough + case schema.TYPE_BINARY: + if s, ok := v.([]byte); ok { + return s, nil + } + s, ok := v.(string) + if !ok { + return nil, fmt.Errorf("unexpected type for BINARY column: %T", v) + } + return []byte(s), nil default: return v, nil } diff --git a/internal/impl/mysql/integration_test.go b/internal/impl/mysql/integration_test.go index e5b27b6b1c..5dca1a1870 100644 --- a/internal/impl/mysql/integration_test.go +++ b/internal/impl/mysql/integration_test.go @@ -362,7 +362,7 @@ func TestIntegrationMySQLCDCAllTypes(t *testing.T) { mediumint_col MEDIUMINT, int_col INT, bigint_col BIGINT, - decimal_col DECIMAL(10, 2), + decimal_col DECIMAL(38, 2), numeric_col NUMERIC(10, 2), float_col FLOAT, double_col DOUBLE, @@ -388,7 +388,8 @@ func TestIntegrationMySQLCDCAllTypes(t *testing.T) { mediumtext_col MEDIUMTEXT, longtext_col LONGTEXT, enum_col ENUM('option1', 'option2', 'option3'), - set_col SET('a', 'b', 'c', 'd') + set_col SET('a', 'b', 'c', 'd'), + json_col JSON -- TODO(cdc): Spatial Data Types -- geometry_col GEOMETRY, @@ -431,14 +432,15 @@ INSERT INTO all_data_types ( mediumtext_col, longtext_col, enum_col, - set_col + set_col, + json_col ) VALUES ( 127, -- tinyint_col 32767, -- smallint_col 8388607, -- mediumint_col 2147483647, -- int_col 9223372036854775807, -- bigint_col - 12345.67, -- decimal_col + 999999999999999999999999999999999999.99, -- decimal_col 98765.43, -- numeric_col 3.14, -- float_col 2.718281828, -- double_col @@ -460,7 +462,8 @@ INSERT INTO all_data_types ( 'medium text', -- mediumtext_col 'large text', -- longtext_col 'option1', -- enum_col - 'a,b' -- set_col + 'a,b', -- set_col + '{"foo":5,"bar":[1,2,3]}' -- json_col ); `) @@ -535,14 +538,15 @@ memory: {} mediumtext_col, longtext_col, enum_col, - set_col + set_col, + json_col ) VALUES ( -128, -- tinyint_col -32768, -- smallint_col -8388608, -- mediumint_col -2147483648, -- int_col -9223372036854775808, -- bigint_col - 54321.12, -- decimal_col + 888888888888888888888888888888888888.88, -- decimal_col 87654.21, -- numeric_col 1.618, -- float_col 3.141592653, -- double_col @@ -564,7 +568,8 @@ memory: {} 'medium_text_value', -- mediumtext_col 'long_text_value', -- longtext_col 'option2', -- enum_col - 'b,c' -- set_col + 'b,c', -- set_col + '{"foo":-1,"bar":[3,2,1]}' -- json_col );`) assert.Eventually(t, func() bool { @@ -580,7 +585,7 @@ memory: {} "mediumint_col": 8388607, "int_col": 2147483647, "bigint_col": 9223372036854775807, - "decimal_col": 12345.67, + "decimal_col": 999999999999999999999999999999999999.99, "numeric_col": 98765.43, "float_col": 3.14, "double_col": 2.718281828, @@ -602,7 +607,8 @@ memory: {} "mediumtext_col": "medium text", "longtext_col": "large text", "enum_col": "option1", - "set_col": ["a", "b"] + "set_col": ["a", "b"], + "json_col": {"foo":5, "bar":[1, 2, 3]} }`) require.JSONEq(t, outBatches[1], `{ "tinyint_col": -128, @@ -610,28 +616,29 @@ memory: {} "mediumint_col": -8388608, "int_col": -2147483648, "bigint_col": -9223372036854775808, - "decimal_col": 54321.12, + "decimal_col": 888888888888888888888888888888888888.88, "numeric_col": 87654.21, "float_col": 1.618, "double_col": 3.141592653, - "date_col": "2023-01-01", - "datetime_col": "2023-01-01 12:00:00", - "timestamp_col": "2023-01-01 12:00:00", + "date_col": "2023-01-01T00:00:00Z", + "datetime_col": "2023-01-01T12:00:00Z", + "timestamp_col": "2023-01-01T12:00:00Z", "time_col": "23:59:59", "year_col": 2023, "char_col": "example", "varchar_col": "another_example", - "binary_col": "fixed", - "varbinary_col": "dynamic", - "tinyblob_col": "tiny_blob_value", - "blob_col": "blob_value", - "mediumblob_col": "medium_blob_value", - "longblob_col": "long_blob_value", + "binary_col": "Zml4ZWQ=", + "varbinary_col": "ZHluYW1pYw==", + "tinyblob_col": "dGlueV9ibG9iX3ZhbHVl", + "blob_col": "YmxvYl92YWx1ZQ==", + "mediumblob_col": "bWVkaXVtX2Jsb2JfdmFsdWU=", + "longblob_col": "bG9uZ19ibG9iX3ZhbHVl", "tinytext_col": "tiny_text_value", "text_col": "text_value", "mediumtext_col": "medium_text_value", "longtext_col": "long_text_value", "enum_col": "option2", - "set_col": ["b", "c"] + "set_col": ["b", "c"], + "json_col": {"foo":-1,"bar":[3,2,1]} }`) } From 0f6efb558ca8b9691df17565e44702839fb5e0c5 Mon Sep 17 00:00:00 2001 From: Tyler Rockwood Date: Fri, 20 Dec 2024 20:35:05 +0000 Subject: [PATCH 37/45] mycdc: cleanup tests for MySQL CDC --- internal/impl/mysql/input_mysql_stream.go | 13 +- internal/impl/mysql/integration_test.go | 169 +++++++++++----------- 2 files changed, 97 insertions(+), 85 deletions(-) diff --git a/internal/impl/mysql/input_mysql_stream.go b/internal/impl/mysql/input_mysql_stream.go index 0de203f017..a8c7463ec1 100644 --- a/internal/impl/mysql/input_mysql_stream.go +++ b/internal/impl/mysql/input_mysql_stream.go @@ -419,8 +419,17 @@ func prepSnapshotScannerAndMappers(cols []*sql.ColumnType) (values []any, mapper return s.Time, nil } case "TINYINT", "SMALLINT", "MEDIUMINT", "INT", "BIGINT", "YEAR": - val = new(sql.Null[int]) - mapper = snapshotValueMapper[int] + val = new(sql.NullInt64) + mapper = func(v any) (any, error) { + s, ok := v.(*sql.NullInt64) + if !ok { + return nil, fmt.Errorf("expected %T got %T", int64(0), v) + } + if !s.Valid { + return nil, nil + } + return int(s.Int64), nil + } case "DECIMAL", "NUMERIC": val = new(sql.NullString) mapper = stringMapping(func(s string) (any, error) { diff --git a/internal/impl/mysql/integration_test.go b/internal/impl/mysql/integration_test.go index 5dca1a1870..cc30254b0b 100644 --- a/internal/impl/mysql/integration_test.go +++ b/internal/impl/mysql/integration_test.go @@ -40,6 +40,7 @@ func (db *testDB) Exec(query string, args ...any) { } func setupTestWithMySQLVersion(t *testing.T, version string) (string, *testDB) { + t.Parallel() integration.CheckSkip(t) pool, err := dockertest.NewPool("") require.NoError(t, err) @@ -105,14 +106,15 @@ func TestIntegrationMySQLCDC(t *testing.T) { integration.CheckSkip(t) var mysqlTestVersions = []string{"8.0", "9.0", "9.1"} for _, version := range mysqlTestVersions { - dsn, db := setupTestWithMySQLVersion(t, version) - // Create table - db.Exec(` + t.Run(version, func(t *testing.T) { + dsn, db := setupTestWithMySQLVersion(t, version) + // Create table + db.Exec(` CREATE TABLE IF NOT EXISTS foo ( a INT PRIMARY KEY ) `) - template := fmt.Sprintf(` + template := fmt.Sprintf(` mysql_cdc: dsn: %s stream_snapshot: false @@ -121,84 +123,85 @@ mysql_cdc: - foo `, dsn) - cacheConf := fmt.Sprintf(` + cacheConf := fmt.Sprintf(` label: foocache file: directory: %s`, t.TempDir()) - streamOutBuilder := service.NewStreamBuilder() - require.NoError(t, streamOutBuilder.SetLoggerYAML(`level: INFO`)) - require.NoError(t, streamOutBuilder.AddCacheYAML(cacheConf)) - require.NoError(t, streamOutBuilder.AddInputYAML(template)) - - var outBatches []string - var outBatchMut sync.Mutex - require.NoError(t, streamOutBuilder.AddBatchConsumerFunc(func(c context.Context, mb service.MessageBatch) error { - msgBytes, err := mb[0].AsBytes() + streamOutBuilder := service.NewStreamBuilder() + require.NoError(t, streamOutBuilder.SetLoggerYAML(`level: INFO`)) + require.NoError(t, streamOutBuilder.AddCacheYAML(cacheConf)) + require.NoError(t, streamOutBuilder.AddInputYAML(template)) + + var outBatches []string + var outBatchMut sync.Mutex + require.NoError(t, streamOutBuilder.AddBatchConsumerFunc(func(c context.Context, mb service.MessageBatch) error { + msgBytes, err := mb[0].AsBytes() + require.NoError(t, err) + outBatchMut.Lock() + outBatches = append(outBatches, string(msgBytes)) + outBatchMut.Unlock() + return nil + })) + + streamOut, err := streamOutBuilder.Build() require.NoError(t, err) - outBatchMut.Lock() - outBatches = append(outBatches, string(msgBytes)) - outBatchMut.Unlock() - return nil - })) - - streamOut, err := streamOutBuilder.Build() - require.NoError(t, err) - go func() { - err = streamOut.Run(context.Background()) + go func() { + err = streamOut.Run(context.Background()) + require.NoError(t, err) + }() + + time.Sleep(time.Second * 5) + for i := 0; i < 1000; i++ { + // Insert 10000 rows + db.Exec("INSERT INTO foo VALUES (?)", i) + } + + assert.Eventually(t, func() bool { + outBatchMut.Lock() + defer outBatchMut.Unlock() + return len(outBatches) == 1000 + }, time.Minute*5, time.Millisecond*100) + + require.NoError(t, streamOut.StopWithin(time.Second*10)) + + streamOutBuilder = service.NewStreamBuilder() + require.NoError(t, streamOutBuilder.SetLoggerYAML(`level: INFO`)) + require.NoError(t, streamOutBuilder.AddCacheYAML(cacheConf)) + require.NoError(t, streamOutBuilder.AddInputYAML(template)) + + outBatches = nil + require.NoError(t, streamOutBuilder.AddBatchConsumerFunc(func(c context.Context, mb service.MessageBatch) error { + msgBytes, err := mb[0].AsBytes() + require.NoError(t, err) + outBatchMut.Lock() + outBatches = append(outBatches, string(msgBytes)) + outBatchMut.Unlock() + return nil + })) + + streamOut, err = streamOutBuilder.Build() require.NoError(t, err) - }() - - time.Sleep(time.Second * 5) - for i := 0; i < 1000; i++ { - // Insert 10000 rows - db.Exec("INSERT INTO foo VALUES (?)", i) - } - assert.Eventually(t, func() bool { - outBatchMut.Lock() - defer outBatchMut.Unlock() - return len(outBatches) == 1000 - }, time.Minute*5, time.Millisecond*100) + time.Sleep(time.Second) + for i := 1001; i < 2001; i++ { + db.Exec("INSERT INTO foo VALUES (?)", i) + } - require.NoError(t, streamOut.StopWithin(time.Second*10)) + go func() { + err = streamOut.Run(context.Background()) + require.NoError(t, err) + }() - streamOutBuilder = service.NewStreamBuilder() - require.NoError(t, streamOutBuilder.SetLoggerYAML(`level: INFO`)) - require.NoError(t, streamOutBuilder.AddCacheYAML(cacheConf)) - require.NoError(t, streamOutBuilder.AddInputYAML(template)) + assert.Eventually(t, func() bool { + outBatchMut.Lock() + defer outBatchMut.Unlock() + return len(outBatches) == 1000 + }, time.Minute*5, time.Millisecond*100) - outBatches = nil - require.NoError(t, streamOutBuilder.AddBatchConsumerFunc(func(c context.Context, mb service.MessageBatch) error { - msgBytes, err := mb[0].AsBytes() - require.NoError(t, err) - outBatchMut.Lock() - outBatches = append(outBatches, string(msgBytes)) - outBatchMut.Unlock() - return nil - })) - - streamOut, err = streamOutBuilder.Build() - require.NoError(t, err) - - time.Sleep(time.Second) - for i := 1001; i < 2001; i++ { - db.Exec("INSERT INTO foo VALUES (?)", i) - } - - go func() { - err = streamOut.Run(context.Background()) - require.NoError(t, err) - }() - - assert.Eventually(t, func() bool { - outBatchMut.Lock() - defer outBatchMut.Unlock() - return len(outBatches) == 1000 - }, time.Minute*5, time.Millisecond*100) - - require.NoError(t, streamOut.StopWithin(time.Second*10)) + require.NoError(t, streamOut.StopWithin(time.Second*10)) + }) } } @@ -273,12 +276,13 @@ func TestIntegrationMySQLCDCWithCompositePrimaryKeys(t *testing.T) { dsn, db := setupTestWithMySQLVersion(t, "8.0") // Create table db.Exec(` - CREATE TABLE IF NOT EXISTS foo ( - a INT, - b INT, - v JSON, - size ENUM('x-small', 'small', 'medium', 'large', 'x-large'), - PRIMARY KEY (a, b) + CREATE TABLE IF NOT EXISTS ` + "`Foo`" + ` ( + ` + "`A`" + ` INT, + ` + "`B`" + ` INT, + PRIMARY KEY ( + ` + "`A`" + `, + ` + "`B`" + ` + ) ) `) // Create control table to ensure we don't stream it @@ -286,15 +290,14 @@ func TestIntegrationMySQLCDCWithCompositePrimaryKeys(t *testing.T) { CREATE TABLE IF NOT EXISTS foo_non_streamed ( a INT, b INT, - v JSON, PRIMARY KEY (a, b) ) `) // Insert 1000 rows for initial snapshot streaming for i := 0; i < 1000; i++ { - db.Exec("INSERT INTO foo VALUES (?, ?, ?, ?)", i, i, `{"json":"data"}`, `large`) - db.Exec("INSERT INTO foo_non_streamed VALUES (?, ?, ?)", i, i, `{"json":"data"}`) + db.Exec("INSERT INTO `Foo` VALUES (?, ?)", i, i) + db.Exec("INSERT INTO foo_non_streamed VALUES (?, ?)", i, i) } template := fmt.Sprintf(` @@ -304,7 +307,7 @@ mysql_cdc: snapshot_max_batch_size: 500 checkpoint_cache: foocache tables: - - foo + - Foo `, dsn) cacheConf := fmt.Sprintf(` @@ -339,8 +342,8 @@ file: time.Sleep(time.Second * 5) for i := 1000; i < 2000; i++ { // Insert 10000 rows - db.Exec("INSERT INTO foo VALUES (?, ?, ?, ?)", i, i, `{"json":"data"}`, `x-small`) - db.Exec("INSERT INTO foo_non_streamed VALUES (?, ?, ?)", i, i, `{"json":"data"}`) + db.Exec("INSERT INTO `Foo` VALUES (?, ?)", i, i) + db.Exec("INSERT INTO foo_non_streamed VALUES (?, ?)", i, i) } assert.Eventually(t, func() bool { From a2dd9cad0fa53e979d9c4ccacd319243cc76a2af Mon Sep 17 00:00:00 2001 From: Tyler Rockwood Date: Fri, 20 Dec 2024 21:08:13 +0000 Subject: [PATCH 38/45] mycdc: handle nil streaming values --- internal/impl/mysql/input_mysql_stream.go | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/internal/impl/mysql/input_mysql_stream.go b/internal/impl/mysql/input_mysql_stream.go index a8c7463ec1..9acdaadb89 100644 --- a/internal/impl/mysql/input_mysql_stream.go +++ b/internal/impl/mysql/input_mysql_stream.go @@ -652,13 +652,13 @@ func (i *mysqlStreamInput) OnRow(e *canal.RowsEvent) error { if _, ok := i.tablesFilterMap[e.Table.Name]; !ok { return nil } - // i.logger.Infof("got rows (action=%s, rows=%d)", e.Action, len(e.Rows)) switch e.Action { case canal.InsertAction: return i.onMessage(e, 0, 1) case canal.DeleteAction: return i.onMessage(e, 0, 1) case canal.UpdateAction: + // Updates send both the new and old data - we only emit the new data. return i.onMessage(e, 1, 2) default: return errors.New("invalid rows action") @@ -687,6 +687,9 @@ func (i *mysqlStreamInput) onMessage(e *canal.RowsEvent, initValue, incrementVal } func mapMessageColumn(v any, col schema.TableColumn) (any, error) { + if v == nil { + return v, nil + } switch col.Type { case schema.TYPE_DECIMAL: s, ok := v.(string) From f34111c41b8a781241d5a7c9226201f665399861 Mon Sep 17 00:00:00 2001 From: Tyler Rockwood Date: Fri, 20 Dec 2024 21:09:00 +0000 Subject: [PATCH 39/45] add changelog entry --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 8dc835bef8..1f124ee6ca 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -16,6 +16,7 @@ All notable changes to this project will be documented in this file. - `snowpipe_streaming` now supports exactly once delivery using `offset_token`. (@rockwotj) - `ollama_chat` now supports tool calling. (@rockwotj) - New `ollama_moderation` which allows using LlamaGuard or ShieldGemma to check if LLM responses are safe. (@rockwotj) +- New `mysql_cdc` input supporting change data capture (CDC) from MySQL. (@rockwotj, @le-vlad) ### Fixed From e118968672af8d7b253252d8f6d663f27e666132 Mon Sep 17 00:00:00 2001 From: Tyler Rockwood Date: Fri, 20 Dec 2024 21:15:40 +0000 Subject: [PATCH 40/45] make linter happy --- internal/impl/mysql/integration_test.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/internal/impl/mysql/integration_test.go b/internal/impl/mysql/integration_test.go index cc30254b0b..ae06d9279e 100644 --- a/internal/impl/mysql/integration_test.go +++ b/internal/impl/mysql/integration_test.go @@ -582,7 +582,7 @@ memory: {} }, time.Second*30, time.Millisecond*100) require.NoError(t, streamOut.StopWithin(time.Second*10)) - require.JSONEq(t, outBatches[0], `{ + require.JSONEq(t, `{ "tinyint_col": 127, "smallint_col": 32767, "mediumint_col": 8388607, @@ -612,8 +612,8 @@ memory: {} "enum_col": "option1", "set_col": ["a", "b"], "json_col": {"foo":5, "bar":[1, 2, 3]} -}`) - require.JSONEq(t, outBatches[1], `{ +}`, outBatches[0]) + require.JSONEq(t, `{ "tinyint_col": -128, "smallint_col": -32768, "mediumint_col": -8388608, @@ -643,5 +643,5 @@ memory: {} "enum_col": "option2", "set_col": ["b", "c"], "json_col": {"foo":-1,"bar":[3,2,1]} -}`) +}`, outBatches[1]) } From e3d2b7d8a31b7dadab3e4d2d82b13d95a4835117 Mon Sep 17 00:00:00 2001 From: Tyler Rockwood Date: Sat, 11 Jan 2025 07:38:40 +0000 Subject: [PATCH 41/45] mycdc: add comment about txn --- internal/impl/mysql/snapshot.go | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/internal/impl/mysql/snapshot.go b/internal/impl/mysql/snapshot.go index a5b3407079..bdd9de4dd8 100644 --- a/internal/impl/mysql/snapshot.go +++ b/internal/impl/mysql/snapshot.go @@ -53,6 +53,7 @@ func (s *Snapshot) prepareSnapshot(ctx context.Context) (*position, error) { // Start a consistent snapshot transaction s.tx, err = s.snapshotConn.BeginTx(ctx, &sql.TxOptions{ + ReadOnly: true, Isolation: sql.LevelRepeatableRead, }) @@ -68,6 +69,10 @@ func (s *Snapshot) prepareSnapshot(ctx context.Context) (*position, error) { the data being read, maintaining referential integrity across tables while capturing the initial state. */ + + // NOTE: this is a little sneaky because we're actually implicitly closing the transaction + // started with `BeginTx` above and replacing it with this one. We have to do this because + // the `database/sql` driver we're using does not support this WITH CONSISTENT SNAPSHOT. if _, err := s.tx.ExecContext(ctx, "START TRANSACTION WITH CONSISTENT SNAPSHOT"); err != nil { if rErr := s.tx.Rollback(); rErr != nil { return nil, rErr From a873c23a392bfc383397690b0db37669b28495ba Mon Sep 17 00:00:00 2001 From: Mihai Todor Date: Sun, 12 Jan 2025 00:11:16 +0000 Subject: [PATCH 42/45] Fix some nitpicks Signed-off-by: Mihai Todor --- internal/impl/mysql/event.go | 2 +- internal/impl/mysql/input_mysql_stream.go | 42 ++++++++++++++--------- internal/impl/mysql/integration_test.go | 2 +- 3 files changed, 27 insertions(+), 19 deletions(-) diff --git a/internal/impl/mysql/event.go b/internal/impl/mysql/event.go index a2f7fd0748..a13470f649 100644 --- a/internal/impl/mysql/event.go +++ b/internal/impl/mysql/event.go @@ -18,7 +18,7 @@ import ( type position = mysql.Position -// MessageOperation is a string type specifying message opration +// MessageOperation is a string type specifying message operation type MessageOperation string const ( diff --git a/internal/impl/mysql/input_mysql_stream.go b/internal/impl/mysql/input_mysql_stream.go index 9acdaadb89..5ac2a0f532 100644 --- a/internal/impl/mysql/input_mysql_stream.go +++ b/internal/impl/mysql/input_mysql_stream.go @@ -44,6 +44,15 @@ const ( var mysqlStreamConfigSpec = service.NewConfigSpec(). Summary("Enables MySQL streaming for RedPanda Connect."). + Description(` +== Metadata + +This input adds the following metadata fields to each message: + +- operation +- table +- binlog_position +`). Fields( service.NewStringField(fieldMySQLDSN). Description("The DSN of the MySQL database to connect to."). @@ -52,7 +61,7 @@ var mysqlStreamConfigSpec = service.NewConfigSpec(). Description("A list of tables to stream from the database."). Example([]string{"table1", "table2"}), service.NewStringField(fieldCheckpointCache). - Description("A https://www.docs.redpanda.com/redpanda-connect/components/caches/about[cache resource^] to use for storing the current latest BinLog Position that has been successfully delivered, this allows Redpanda Connect to continue from that BinLog Position upon restart, rather than consume the entire state of the table.\""), + Description("A https://www.docs.redpanda.com/redpanda-connect/components/caches/about[cache resource^] to use for storing the current latest BinLog Position that has been successfully delivered, this allows Redpanda Connect to continue from that BinLog Position upon restart, rather than consume the entire state of the table."), service.NewStringField(fieldCheckpointKey). Description("The key to use to store the snapshot position in `"+fieldCheckpointCache+"`. An alternative key can be provided if multiple CDC inputs share the same cache."). Default("mysql_binlog_position"), @@ -105,7 +114,7 @@ type mysqlStreamInput struct { } func newMySQLStreamInput(conf *service.ParsedConfig, res *service.Resources) (s service.BatchInput, err error) { - streamInput := mysqlStreamInput{ + i := mysqlStreamInput{ logger: res.Logger(), rawMessageEvents: make(chan MessageEvent), msgChan: make(chan asyncMessage), @@ -114,42 +123,41 @@ func newMySQLStreamInput(conf *service.ParsedConfig, res *service.Resources) (s var batching service.BatchPolicy - if streamInput.dsn, err = conf.FieldString(fieldMySQLDSN); err != nil { + if i.dsn, err = conf.FieldString(fieldMySQLDSN); err != nil { return nil, err } - streamInput.mysqlConfig, err = mysql.ParseDSN(streamInput.dsn) + i.mysqlConfig, err = mysql.ParseDSN(i.dsn) if err != nil { return nil, fmt.Errorf("error parsing mysql DSN: %v", err) } - if streamInput.tables, err = conf.FieldStringList(fieldMySQLTables); err != nil { + if i.tables, err = conf.FieldStringList(fieldMySQLTables); err != nil { return nil, err } - if streamInput.streamSnapshot, err = conf.FieldBool(fieldStreamSnapshot); err != nil { + if i.streamSnapshot, err = conf.FieldBool(fieldStreamSnapshot); err != nil { return nil, err } - if streamInput.fieldSnapshotMaxBatchSize, err = conf.FieldInt(fieldSnapshotMaxBatchSize); err != nil { + if i.fieldSnapshotMaxBatchSize, err = conf.FieldInt(fieldSnapshotMaxBatchSize); err != nil { return nil, err } - if streamInput.checkPointLimit, err = conf.FieldInt(fieldCheckpointLimit); err != nil { + if i.checkPointLimit, err = conf.FieldInt(fieldCheckpointLimit); err != nil { return nil, err } - if streamInput.binLogCache, err = conf.FieldString(fieldCheckpointCache); err != nil { + if i.binLogCache, err = conf.FieldString(fieldCheckpointCache); err != nil { return nil, err } - if !conf.Resources().HasCache(streamInput.binLogCache) { - return nil, fmt.Errorf("unknown cache resource: %s", streamInput.binLogCache) + if !conf.Resources().HasCache(i.binLogCache) { + return nil, fmt.Errorf("unknown cache resource: %s", i.binLogCache) } - if streamInput.binLogCacheKey, err = conf.FieldString(fieldCheckpointKey); err != nil { + if i.binLogCacheKey, err = conf.FieldString(fieldCheckpointKey); err != nil { return nil, err } - i := &streamInput i.cp = checkpoint.NewCapped[*position](int64(i.checkPointLimit)) i.tablesFilterMap = map[string]bool{} @@ -173,7 +181,7 @@ func newMySQLStreamInput(conf *service.ParsedConfig, res *service.Resources) (s batching.Count = 1 } - r, err := service.AutoRetryNacksBatchedToggled(conf, i) + r, err := service.AutoRetryNacksBatchedToggled(conf, &i) if err != nil { return nil, err } @@ -222,14 +230,14 @@ func (i *mysqlStreamInput) Connect(ctx context.Context) error { pos, err := i.getCachedBinlogPosition(ctx) if err != nil { - return fmt.Errorf("unable to get cached binlog position: %w", err) + return fmt.Errorf("unable to get cached binlog position: %s", err) } // create snapshot instance if we were requested and haven't finished it before. var snapshot *Snapshot if i.streamSnapshot && pos == nil { db, err := sql.Open("mysql", i.dsn) if err != nil { - return err + return fmt.Errorf("failed to connect to MySQL server: %s", err) } snapshot = NewSnapshot(i.logger, db) } @@ -248,7 +256,7 @@ func (i *mysqlStreamInput) Connect(ctx context.Context) error { wg.Go(func() error { return i.readMessages(ctx) }) wg.Go(func() error { return i.startMySQLSync(ctx, pos, snapshot) }) if err := wg.Wait(); err != nil && !errors.Is(err, context.Canceled) { - i.logger.Errorf("error during MySQL CDC: %v", err) + i.logger.Errorf("error during MySQL CDC: %s", err) } else { i.logger.Info("successfully shutdown MySQL CDC stream") } diff --git a/internal/impl/mysql/integration_test.go b/internal/impl/mysql/integration_test.go index ae06d9279e..242ea60dca 100644 --- a/internal/impl/mysql/integration_test.go +++ b/internal/impl/mysql/integration_test.go @@ -642,6 +642,6 @@ memory: {} "longtext_col": "long_text_value", "enum_col": "option2", "set_col": ["b", "c"], - "json_col": {"foo":-1,"bar":[3,2,1]} + "json_col": {"foo":-1,"bar":[3,2,1]} }`, outBatches[1]) } From a144500de5ce4d16291eab9c0709e15f885b0e74 Mon Sep 17 00:00:00 2001 From: Mihai Todor Date: Sun, 12 Jan 2025 00:23:55 +0000 Subject: [PATCH 43/45] Add to public components and generate docs Signed-off-by: Mihai Todor --- .../components/pages/inputs/mysql_cdc.adoc | 281 ++++++++++++++++++ internal/impl/mysql/input_mysql_stream.go | 3 + internal/plugins/info.csv | 3 +- public/components/all/package.go | 1 + public/components/cloud/package.go | 1 + public/components/mysql/package.go | 16 + 6 files changed, 304 insertions(+), 1 deletion(-) create mode 100644 docs/modules/components/pages/inputs/mysql_cdc.adoc create mode 100644 public/components/mysql/package.go diff --git a/docs/modules/components/pages/inputs/mysql_cdc.adoc b/docs/modules/components/pages/inputs/mysql_cdc.adoc new file mode 100644 index 0000000000..16bb09675e --- /dev/null +++ b/docs/modules/components/pages/inputs/mysql_cdc.adoc @@ -0,0 +1,281 @@ += mysql_cdc +:type: input +:status: beta +:categories: ["Services"] + + + +//// + 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::[] + + +Enables MySQL streaming for RedPanda Connect. + +Introduced in version 4.45.0. + + +[tabs] +====== +Common:: ++ +-- + +```yml +# Common config fields, showing default values +input: + label: "" + mysql_cdc: + dsn: user:password@tcp(localhost:3306)/database # No default (required) + tables: [] # No default (required) + checkpoint_cache: "" # No default (required) + checkpoint_key: mysql_binlog_position + snapshot_max_batch_size: 1000 + stream_snapshot: false # No default (required) + auto_replay_nacks: true + checkpoint_limit: 1024 + batching: + count: 0 + byte_size: 0 + period: "" + check: "" +``` + +-- +Advanced:: ++ +-- + +```yml +# All config fields, showing default values +input: + label: "" + mysql_cdc: + dsn: user:password@tcp(localhost:3306)/database # No default (required) + tables: [] # No default (required) + checkpoint_cache: "" # No default (required) + checkpoint_key: mysql_binlog_position + snapshot_max_batch_size: 1000 + stream_snapshot: false # No default (required) + auto_replay_nacks: true + checkpoint_limit: 1024 + batching: + count: 0 + byte_size: 0 + period: "" + check: "" + processors: [] # No default (optional) +``` + +-- +====== + +== Metadata + +This input adds the following metadata fields to each message: + +- operation +- table +- binlog_position + + +== Fields + +=== `dsn` + +The DSN of the MySQL database to connect to. + + +*Type*: `string` + + +```yml +# Examples + +dsn: user:password@tcp(localhost:3306)/database +``` + +=== `tables` + +A list of tables to stream from the database. + + +*Type*: `array` + + +```yml +# Examples + +tables: + - table1 + - table2 +``` + +=== `checkpoint_cache` + +A https://www.docs.redpanda.com/redpanda-connect/components/caches/about[cache resource^] to use for storing the current latest BinLog Position that has been successfully delivered, this allows Redpanda Connect to continue from that BinLog Position upon restart, rather than consume the entire state of the table. + + +*Type*: `string` + + +=== `checkpoint_key` + +The key to use to store the snapshot position in `checkpoint_cache`. An alternative key can be provided if multiple CDC inputs share the same cache. + + +*Type*: `string` + +*Default*: `"mysql_binlog_position"` + +=== `snapshot_max_batch_size` + +The maximum number of rows to be streamed in a single batch when taking a snapshot. + + +*Type*: `int` + +*Default*: `1000` + +=== `stream_snapshot` + +If set to true, the connector will query all the existing data as a part of snapshot process. Otherwise, it will start from the current binlog position. + + +*Type*: `bool` + + +=== `auto_replay_nacks` + +Whether messages that are rejected (nacked) at the output level should be automatically replayed indefinitely, eventually resulting in back pressure if the cause of the rejections is persistent. If set to `false` these messages will instead be deleted. Disabling auto replays can greatly improve memory efficiency of high throughput streams as the original shape of the data can be discarded immediately upon consumption and mutation. + + +*Type*: `bool` + +*Default*: `true` + +=== `checkpoint_limit` + +The maximum number of messages that can be processed at a given time. Increasing this limit enables parallel processing and batching at the output level. Any given BinLog Position will not be acknowledged unless all messages under that offset are delivered in order to preserve at least once delivery guarantees. + + +*Type*: `int` + +*Default*: `1024` + +=== `batching` + +Allows you to configure a xref:configuration:batching.adoc[batching policy]. + + +*Type*: `object` + + +```yml +# Examples + +batching: + byte_size: 5000 + count: 0 + period: 1s + +batching: + count: 10 + period: 1s + +batching: + check: this.contains("END BATCH") + count: 0 + period: 1m +``` + +=== `batching.count` + +A number of messages at which the batch should be flushed. If `0` disables count based batching. + + +*Type*: `int` + +*Default*: `0` + +=== `batching.byte_size` + +An amount of bytes at which the batch should be flushed. If `0` disables size based batching. + + +*Type*: `int` + +*Default*: `0` + +=== `batching.period` + +A period in which an incomplete batch should be flushed regardless of its size. + + +*Type*: `string` + +*Default*: `""` + +```yml +# Examples + +period: 1s + +period: 1m + +period: 500ms +``` + +=== `batching.check` + +A xref:guides:bloblang/about.adoc[Bloblang query] that should return a boolean value indicating whether a message should end a batch. + + +*Type*: `string` + +*Default*: `""` + +```yml +# Examples + +check: this.type == "end_of_transaction" +``` + +=== `batching.processors` + +A list of xref:components:processors/about.adoc[processors] to apply to a batch as it is flushed. This allows you to aggregate and archive the batch however you see fit. Please note that all resulting messages are flushed as a single batch, therefore splitting the batch into smaller batches using these processors is a no-op. + + +*Type*: `array` + + +```yml +# Examples + +processors: + - archive: + format: concatenate + +processors: + - archive: + format: lines + +processors: + - archive: + format: json_array +``` + + diff --git a/internal/impl/mysql/input_mysql_stream.go b/internal/impl/mysql/input_mysql_stream.go index 5ac2a0f532..278701bf7b 100644 --- a/internal/impl/mysql/input_mysql_stream.go +++ b/internal/impl/mysql/input_mysql_stream.go @@ -43,6 +43,9 @@ const ( ) var mysqlStreamConfigSpec = service.NewConfigSpec(). + Beta(). + Categories("Services"). + Version("4.45.0"). Summary("Enables MySQL streaming for RedPanda Connect."). Description(` == Metadata diff --git a/internal/plugins/info.csv b/internal/plugins/info.csv index 397eb732c5..782c60d915 100644 --- a/internal/plugins/info.csv +++ b/internal/plugins/info.csv @@ -131,6 +131,7 @@ mqtt ,output ,mqtt ,4.37.0 ,certif msgpack ,processor ,msgpack ,3.59.0 ,community ,n ,n ,n multilevel ,cache ,Multilevel ,0.0.0 ,certified ,n ,y ,y mutation ,processor ,mutation ,4.5.0 ,certified ,n ,y ,y +mysql_cdc ,input ,mysql_cdc ,4.45.0 ,enterprise ,n ,y ,y nanomsg ,input ,nanomsg ,0.0.0 ,community ,n ,n ,n nanomsg ,output ,nanomsg ,0.0.0 ,community ,n ,n ,n nats ,input ,NATS ,0.0.0 ,certified ,n ,y ,y @@ -170,7 +171,7 @@ parquet ,processor ,parquet ,3.62.0 ,commun parquet_decode ,processor ,parquet_decode ,4.4.0 ,certified ,n ,y ,y parquet_encode ,processor ,parquet_encode ,4.4.0 ,certified ,n ,y ,y parse_log ,processor ,parse_log ,0.0.0 ,community ,n ,y ,y -pg_stream ,input ,pg_stream ,0.0.0 ,enterprise ,y ,y ,y +pg_stream ,input ,pg_stream ,4.43.0 ,enterprise ,y ,y ,y pinecone ,output ,pinecone ,4.31.0 ,certified ,n ,y ,y postgres_cdc ,input ,postgres_cdc ,4.43.0 ,enterprise ,n ,y ,y processors ,processor ,processors ,0.0.0 ,certified ,n ,y ,y diff --git a/public/components/all/package.go b/public/components/all/package.go index 0b7b3a6c3e..04ebb22edb 100644 --- a/public/components/all/package.go +++ b/public/components/all/package.go @@ -21,6 +21,7 @@ import ( _ "github.com/redpanda-data/connect/v4/public/components/cohere" _ "github.com/redpanda-data/connect/v4/public/components/gcp/enterprise" _ "github.com/redpanda-data/connect/v4/public/components/kafka/enterprise" + _ "github.com/redpanda-data/connect/v4/public/components/mysql" _ "github.com/redpanda-data/connect/v4/public/components/ollama" _ "github.com/redpanda-data/connect/v4/public/components/openai" _ "github.com/redpanda-data/connect/v4/public/components/postgresql" diff --git a/public/components/cloud/package.go b/public/components/cloud/package.go index 259a140ba4..1823ae3052 100644 --- a/public/components/cloud/package.go +++ b/public/components/cloud/package.go @@ -31,6 +31,7 @@ import ( _ "github.com/redpanda-data/connect/v4/public/components/memcached" _ "github.com/redpanda-data/connect/v4/public/components/mqtt" _ "github.com/redpanda-data/connect/v4/public/components/msgpack" + _ "github.com/redpanda-data/connect/v4/public/components/mysql" _ "github.com/redpanda-data/connect/v4/public/components/nats" _ "github.com/redpanda-data/connect/v4/public/components/openai" _ "github.com/redpanda-data/connect/v4/public/components/opensearch" diff --git a/public/components/mysql/package.go b/public/components/mysql/package.go new file mode 100644 index 0000000000..3fe1517f69 --- /dev/null +++ b/public/components/mysql/package.go @@ -0,0 +1,16 @@ +/* + * Copyright 2024 Redpanda Data, Inc. + * + * Licensed as a Redpanda Enterprise file under the Redpanda Community + * License (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * https://github.com/redpanda-data/redpanda/blob/master/licenses/rcl.md + */ + +package mysql + +import ( + // Bring in the internal plugin definitions. + _ "github.com/redpanda-data/connect/v4/internal/impl/mysql" +) From 673f51a1268e6f1235030a36a5a7a044099f2128 Mon Sep 17 00:00:00 2001 From: Mihai Todor Date: Sun, 12 Jan 2025 01:00:50 +0000 Subject: [PATCH 44/45] Re-enable the rowserrcheck linter MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Seems to work fine on Go 1.23.4 ¯\_(ツ)_/¯ Signed-off-by: Mihai Todor --- .golangci.yml | 2 +- internal/impl/mysql/input_mysql_stream.go | 17 ++++++++++------- internal/impl/mysql/snapshot.go | 4 ++++ internal/impl/timeplus/driver/driver.go | 4 ++++ 4 files changed, 19 insertions(+), 8 deletions(-) diff --git a/.golangci.yml b/.golangci.yml index cc208f6fa1..ca8d151ddf 100644 --- a/.golangci.yml +++ b/.golangci.yml @@ -89,6 +89,6 @@ linters: - tenv - predeclared - mirror - # - rowserrcheck + - rowserrcheck - bodyclose - nolintlint diff --git a/internal/impl/mysql/input_mysql_stream.go b/internal/impl/mysql/input_mysql_stream.go index 278701bf7b..90f325e81a 100644 --- a/internal/impl/mysql/input_mysql_stream.go +++ b/internal/impl/mysql/input_mysql_stream.go @@ -313,11 +313,12 @@ func (i *mysqlStreamInput) readSnapshot(ctx context.Context, snapshot *Snapshot) return err } i.logger.Tracef("primary keys for table %s: %v", table, tablePks) - var numRowsProcessed int lastSeenPksValues := map[string]any{} for _, pk := range tablePks { lastSeenPksValues[pk] = nil } + + var numRowsProcessed int for { var batchRows *sql.Rows if numRowsProcessed == 0 { @@ -326,21 +327,19 @@ func (i *mysqlStreamInput) readSnapshot(ctx context.Context, snapshot *Snapshot) batchRows, err = snapshot.querySnapshotTable(ctx, table, tablePks, &lastSeenPksValues, i.fieldSnapshotMaxBatchSize) } if err != nil { - return err + return fmt.Errorf("failed to execute snapshot table query: %s", err) } types, err := batchRows.ColumnTypes() if err != nil { - _ = batchRows.Close() - return err + return fmt.Errorf("failed to fetch column types: %s", err) } values, mappers := prepSnapshotScannerAndMappers(types) columns, err := batchRows.Columns() if err != nil { - _ = batchRows.Close() - return err + return fmt.Errorf("failed to fetch columns: %s", err) } var batchRowsCount int @@ -349,7 +348,6 @@ func (i *mysqlStreamInput) readSnapshot(ctx context.Context, snapshot *Snapshot) batchRowsCount++ if err := batchRows.Scan(values...); err != nil { - _ = batchRows.Close() return err } @@ -376,6 +374,11 @@ func (i *mysqlStreamInput) readSnapshot(ctx context.Context, snapshot *Snapshot) return ctx.Err() } } + + if err := batchRows.Err(); err != nil { + return fmt.Errorf("failed to iterate snapshot table: %s", err) + } + // TODO(cdc): Save checkpoint if batchRowsCount < i.fieldSnapshotMaxBatchSize { break diff --git a/internal/impl/mysql/snapshot.go b/internal/impl/mysql/snapshot.go index bdd9de4dd8..755ea4723f 100644 --- a/internal/impl/mysql/snapshot.go +++ b/internal/impl/mysql/snapshot.go @@ -144,6 +144,10 @@ ORDER BY ORDINAL_POSITION; pks = append(pks, pk) } + if err := rows.Err(); err != nil { + return nil, fmt.Errorf("failed to iterate table: %s", err) + } + if len(pks) == 0 { return nil, fmt.Errorf("unable to find primary key for table %s - does the table exist and does it have a primary key set?", table) } diff --git a/internal/impl/timeplus/driver/driver.go b/internal/impl/timeplus/driver/driver.go index d3f40ce875..2d2684e694 100644 --- a/internal/impl/timeplus/driver/driver.go +++ b/internal/impl/timeplus/driver/driver.go @@ -56,6 +56,10 @@ func (d *driver) Run(sql string) error { return err } + if err := rows.Err(); err != nil { + return err + } + columnTypes, err := rows.ColumnTypes() if err != nil { return err From d96fcf150ec6571c8a7e3b641f5d6d792bcc6a14 Mon Sep 17 00:00:00 2001 From: Tyler Rockwood Date: Sun, 12 Jan 2025 21:01:26 -0600 Subject: [PATCH 45/45] mycdc: remove some TODOs This is for a future hypothetical feature, let's just remove to simplify for now --- internal/impl/mysql/input_mysql_stream.go | 4 ---- 1 file changed, 4 deletions(-) diff --git a/internal/impl/mysql/input_mysql_stream.go b/internal/impl/mysql/input_mysql_stream.go index 90f325e81a..b2b1173dcf 100644 --- a/internal/impl/mysql/input_mysql_stream.go +++ b/internal/impl/mysql/input_mysql_stream.go @@ -379,7 +379,6 @@ func (i *mysqlStreamInput) readSnapshot(ctx context.Context, snapshot *Snapshot) return fmt.Errorf("failed to iterate snapshot table: %s", err) } - // TODO(cdc): Save checkpoint if batchRowsCount < i.fieldSnapshotMaxBatchSize { break } @@ -562,9 +561,6 @@ func (i *mysqlStreamInput) flushBatch( } offset := *maxOffset // This has no offset - it's a snapshot message - // TODO(cdc): We should be storing the primary key for - // each table in the snapshot so we can properly resume the - // primary key scan. if offset == nil { return nil }