From 5b5a0e4f1222183eb280e91e19a491a97bfefa4b Mon Sep 17 00:00:00 2001 From: Haris Osmanagic Date: Wed, 25 Dec 2024 15:01:04 +0100 Subject: [PATCH 01/60] [WIP] Handle nullable columns --- source/schema/avro.go | 29 +++++++-- source/snapshot/fetch_worker.go | 91 +++++++++++++++++++++++++--- source/snapshot/fetch_worker_test.go | 14 ++--- source/snapshot/iterator.go | 2 +- source_integration_test.go | 13 ++-- test/helper.go | 8 +-- 6 files changed, 125 insertions(+), 32 deletions(-) diff --git a/source/schema/avro.go b/source/schema/avro.go index 2350615..f43ac44 100644 --- a/source/schema/avro.go +++ b/source/schema/avro.go @@ -63,7 +63,7 @@ type avroExtractor struct { // ExtractLogrepl extracts an Avro schema from the given pglogrepl.RelationMessage. // If `fieldNames` are specified, then only the given fields will be included in the schema. -func (a avroExtractor) ExtractLogrepl(schemaName string, rel *pglogrepl.RelationMessage, fieldNames ...string) (*avro.RecordSchema, error) { +func (a *avroExtractor) ExtractLogrepl(schemaName string, rel *pglogrepl.RelationMessage, fieldNames ...string) (*avro.RecordSchema, error) { var fields []pgconn.FieldDescription for i := range rel.Columns { @@ -74,12 +74,13 @@ func (a avroExtractor) ExtractLogrepl(schemaName string, rel *pglogrepl.Relation }) } - return a.Extract(schemaName, fields, fieldNames...) + // todo terribly bad, only a temporary thing + return a.Extract(schemaName, make(map[string]bool), fields, fieldNames...) } // Extract extracts an Avro schema from the given Postgres field descriptions. // If `fieldNames` are specified, then only the given fields will be included in the schema. -func (a *avroExtractor) Extract(schemaName string, fields []pgconn.FieldDescription, fieldNames ...string) (*avro.RecordSchema, error) { +func (a *avroExtractor) Extract(schemaName string, notNullMap map[string]bool, fields []pgconn.FieldDescription, fieldNames ...string) (*avro.RecordSchema, error) { var avroFields []*avro.Field for _, f := range fields { @@ -92,7 +93,7 @@ func (a *avroExtractor) Extract(schemaName string, fields []pgconn.FieldDescript return nil, fmt.Errorf("field %q with OID %d cannot be resolved", f.Name, f.DataTypeOID) } - s, err := a.extractType(t, f.TypeModifier) + s, err := a.extractType(t, f.TypeModifier, notNullMap[f.Name]) if err != nil { return nil, err } @@ -117,7 +118,25 @@ func (a *avroExtractor) Extract(schemaName string, fields []pgconn.FieldDescript return sch, nil } -func (a *avroExtractor) extractType(t *pgtype.Type, typeMod int32) (avro.Schema, error) { +func (a *avroExtractor) extractType(t *pgtype.Type, typeMod int32, notNull bool) (avro.Schema, error) { + baseType, err := a.extractBaseType(t, typeMod) + if err != nil { + return nil, err + } + + if !notNull { + schema, err := avro.NewUnionSchema([]avro.Schema{avro.NewNullSchema(), baseType}) + if err != nil { + return nil, fmt.Errorf("failed to create avro union schema for nullable type %v: %w", baseType, err) + } + + return schema, nil + } + + return baseType, nil +} + +func (a *avroExtractor) extractBaseType(t *pgtype.Type, typeMod int32) (avro.Schema, error) { if ps, ok := a.avroMap[t.Name]; ok { return ps, nil } diff --git a/source/snapshot/fetch_worker.go b/source/snapshot/fetch_worker.go index 18cb904..bf7792e 100644 --- a/source/snapshot/fetch_worker.go +++ b/source/snapshot/fetch_worker.go @@ -92,12 +92,14 @@ type FetchWorker struct { db *pgxpool.Pool out chan<- FetchData - keySchema *cschema.Schema - payloadSchema *cschema.Schema + // notNullMap maps column names to if the column is NOT NULL. + notNullMap map[string]bool + keySchema *cschema.Schema - snapshotEnd int64 - lastRead int64 - cursorName string + payloadSchema *cschema.Schema + snapshotEnd int64 + lastRead int64 + cursorName string } func NewFetchWorker(db *pgxpool.Pool, out chan<- FetchData, c FetchConfig) *FetchWorker { @@ -124,10 +126,81 @@ func NewFetchWorker(db *pgxpool.Pool, out chan<- FetchData, c FetchConfig) *Fetc return f } -// Validate will ensure the config is correct. +// Init will ensure the config is correct. // * Table and keys exist // * Key is a primary key -func (f *FetchWorker) Validate(ctx context.Context) error { +func (f *FetchWorker) Init(ctx context.Context) error { + notNullMap, err := f.getNotNullMap(ctx) + if err != nil { + return fmt.Errorf("could not initialize nullability map: %w", err) + } + f.notNullMap = notNullMap + + err = f.validate(ctx) + if err != nil { + return fmt.Errorf("validation failed: %w", err) + } + + return nil +} + +// getNotNullMap returns a map that contains information about nullability of columns. +// Keys are column names, values are booleans showing if the column is NOT NULL or not. +func (f *FetchWorker) getNotNullMap(ctx context.Context) (map[string]bool, error) { + tableName := f.conf.Table + tx, err := f.db.Begin(ctx) + if err != nil { + return nil, fmt.Errorf("failed to start tx for getting column nullability info: %w", err) + } + defer func() { + if err := tx.Rollback(ctx); err != nil { + sdk.Logger(ctx).Warn(). + Err(err). + Msgf("error on tx rollback for getting column nullability info, cursor name: %q", f.cursorName) + } + }() + + query := ` + SELECT a.attname AS column_name, a.attnotnull AS is_not_null + FROM pg_index i + JOIN pg_attribute a ON a.attrelid = i.indrelid + JOIN pg_class c ON a.attrelid = c.oid + WHERE c.relname = $1 + AND a.attnum > 0 + AND NOT a.attisdropped + ORDER BY a.attnum; + ` + + rows, err := tx.Query(context.Background(), query, tableName) + if err != nil { + return nil, err + } + defer rows.Close() + + // Map to store column nullability and primary key info + nullabilityMap := make(map[string]bool) + + for rows.Next() { + var columnName string + var isNotNull bool + + err := rows.Scan(&columnName, &isNotNull) + if err != nil { + return nil, err + } + + // Store info in nested map + nullabilityMap[columnName] = isNotNull + } + + if err := rows.Err(); err != nil { + return nil, err + } + + return nullabilityMap, nil +} + +func (f *FetchWorker) validate(ctx context.Context) error { if err := f.conf.Validate(); err != nil { return fmt.Errorf("failed to validate config: %w", err) } @@ -473,7 +546,7 @@ func (f *FetchWorker) extractSchemas(ctx context.Context, fields []pgconn.FieldD sdk.Logger(ctx).Debug(). Msgf("extracting payload schema for %v fields in %v", len(fields), f.conf.Table) - avroPayloadSch, err := schema.Avro.Extract(f.conf.Table+"_payload", fields) + avroPayloadSch, err := schema.Avro.Extract(f.conf.Table+"_payload", f.notNullMap, fields) if err != nil { return fmt.Errorf("failed to extract payload schema for table %v: %w", f.conf.Table, err) } @@ -493,7 +566,7 @@ func (f *FetchWorker) extractSchemas(ctx context.Context, fields []pgconn.FieldD sdk.Logger(ctx).Debug(). Msgf("extracting schema for key %v in %v", f.conf.Key, f.conf.Table) - avroKeySch, err := schema.Avro.Extract(f.conf.Table+"_key", fields, f.conf.Key) + avroKeySch, err := schema.Avro.Extract(f.conf.Table+"_key", f.notNullMap, fields, f.conf.Key) if err != nil { return fmt.Errorf("failed to extract key schema for table %v: %w", f.conf.Table, err) } diff --git a/source/snapshot/fetch_worker_test.go b/source/snapshot/fetch_worker_test.go index e9a7595..fcc0f60 100644 --- a/source/snapshot/fetch_worker_test.go +++ b/source/snapshot/fetch_worker_test.go @@ -134,7 +134,7 @@ func Test_FetcherValidate(t *testing.T) { }, } - is.NoErr(f.Validate(ctx)) + is.NoErr(f.Init(ctx)) }) t.Run("table missing", func(t *testing.T) { @@ -147,7 +147,7 @@ func Test_FetcherValidate(t *testing.T) { }, } - err := f.Validate(ctx) + err := f.Init(ctx) is.True(err != nil) is.True(strings.Contains(err.Error(), `table "missing_table" does not exist`)) }) @@ -162,7 +162,7 @@ func Test_FetcherValidate(t *testing.T) { }, } - err := f.Validate(ctx) + err := f.Init(ctx) is.True(err != nil) is.True(strings.Contains(err.Error(), `failed to validate key: key "column3" of type "boolean" is unsupported`)) }) @@ -177,7 +177,7 @@ func Test_FetcherValidate(t *testing.T) { }, } - err := f.Validate(ctx) + err := f.Init(ctx) is.NoErr(err) // no error, only a warning }) @@ -191,7 +191,7 @@ func Test_FetcherValidate(t *testing.T) { }, } - err := f.Validate(ctx) + err := f.Init(ctx) is.True(err != nil) ok := strings.Contains(err.Error(), fmt.Sprintf(`key "missing_key" not present on table %q`, table)) if !ok { @@ -220,7 +220,7 @@ func Test_FetcherRun_Initial(t *testing.T) { ctx = tt.Context(ctx) defer close(out) - if err := f.Validate(ctx); err != nil { + if err := f.Init(ctx); err != nil { return err } return f.Run(ctx) @@ -284,7 +284,7 @@ func Test_FetcherRun_Resume(t *testing.T) { ctx = tt.Context(ctx) defer close(out) - if err := f.Validate(ctx); err != nil { + if err := f.Init(ctx); err != nil { return err } return f.Run(ctx) diff --git a/source/snapshot/iterator.go b/source/snapshot/iterator.go index fbf6317..424f95c 100644 --- a/source/snapshot/iterator.go +++ b/source/snapshot/iterator.go @@ -142,7 +142,7 @@ func (i *Iterator) initFetchers(ctx context.Context) error { FetchSize: i.conf.FetchSize, }) - if err := w.Validate(ctx); err != nil { + if err := w.Init(ctx); err != nil { errs = append(errs, fmt.Errorf("failed to validate table fetcher %q config: %w", t, err)) } diff --git a/source_integration_test.go b/source_integration_test.go index b051b24..32f16b7 100644 --- a/source_integration_test.go +++ b/source_integration_test.go @@ -16,6 +16,7 @@ package postgres import ( "context" + "fmt" "testing" "github.com/conduitio/conduit-connector-postgres/source/logrepl" @@ -23,7 +24,7 @@ import ( "github.com/matryer/is" ) -func TestSource_Open(t *testing.T) { +func TestSource_Read(t *testing.T) { is := is.New(t) ctx := test.Context(t) conn := test.ConnectSimple(ctx, t, test.RepmgrConnString) @@ -47,13 +48,17 @@ func TestSource_Open(t *testing.T) { err = s.Open(ctx, nil) is.NoErr(err) - - defer func() { + t.Cleanup(func() { is.NoErr(logrepl.Cleanup(context.Background(), logrepl.CleanupConfig{ URL: test.RepmgrConnString, SlotName: slotName, PublicationName: publicationName, })) is.NoErr(s.Teardown(ctx)) - }() + }) + + gotRecord, err := s.Read(ctx) + is.NoErr(err) + + fmt.Println(gotRecord) } diff --git a/test/helper.go b/test/helper.go index 9ebc8e1..676bf44 100644 --- a/test/helper.go +++ b/test/helper.go @@ -139,7 +139,6 @@ const testTableCreateQuery = ` column1 varchar(256), column2 integer, column3 boolean, - column4 numeric(16,3), column5 numeric(5) )` @@ -190,11 +189,8 @@ func SetupTestTable(ctx context.Context, t *testing.T, conn Querier) string { }) query = ` - INSERT INTO %s (key, column1, column2, column3, column4, column5) - VALUES ('1', 'foo', 123, false, 12.2, 4), - ('2', 'bar', 456, true, 13.42, 8), - ('3', 'baz', 789, false, null, 9), - ('4', null, null, null, 91.1, null)` + INSERT INTO %s (key) + VALUES ('1')` query = fmt.Sprintf(query, table) _, err = conn.Exec(ctx, query) is.NoErr(err) From 44db32bb7d91ef7d96fc57f5ecdef933119ea037 Mon Sep 17 00:00:00 2001 From: Haris Osmanagic Date: Wed, 25 Dec 2024 15:29:30 +0100 Subject: [PATCH 02/60] improve integration test --- source_integration_test.go | 24 ++++++++++++++++++++---- test/helper.go | 7 +++++-- 2 files changed, 25 insertions(+), 6 deletions(-) diff --git a/source_integration_test.go b/source_integration_test.go index 32f16b7..8cf8366 100644 --- a/source_integration_test.go +++ b/source_integration_test.go @@ -16,7 +16,6 @@ package postgres import ( "context" - "fmt" "testing" "github.com/conduitio/conduit-connector-postgres/source/logrepl" @@ -57,8 +56,25 @@ func TestSource_Read(t *testing.T) { is.NoErr(s.Teardown(ctx)) }) - gotRecord, err := s.Read(ctx) - is.NoErr(err) + for i := 0; i < 4; i++ { + gotRecord, err := s.Read(ctx) + is.NoErr(err) + + is.True(gotRecord.Key != nil) + is.True(gotRecord.Payload.After != nil) + + payloadSchemaSubject, err := gotRecord.Metadata.GetPayloadSchemaSubject() + is.NoErr(err) + is.Equal(tableName+"_payload", payloadSchemaSubject) + payloadSchemaVersion, err := gotRecord.Metadata.GetPayloadSchemaVersion() + is.NoErr(err) + is.Equal(1, payloadSchemaVersion) - fmt.Println(gotRecord) + keySchemaSubject, err := gotRecord.Metadata.GetKeySchemaSubject() + is.NoErr(err) + is.Equal(tableName+"_key", keySchemaSubject) + keySchemaVersion, err := gotRecord.Metadata.GetKeySchemaVersion() + is.NoErr(err) + is.Equal(1, keySchemaVersion) + } } diff --git a/test/helper.go b/test/helper.go index 676bf44..756ebe1 100644 --- a/test/helper.go +++ b/test/helper.go @@ -189,8 +189,11 @@ func SetupTestTable(ctx context.Context, t *testing.T, conn Querier) string { }) query = ` - INSERT INTO %s (key) - VALUES ('1')` + INSERT INTO %s (key, column1, column2, column3) + VALUES ('1', 'foo', 123, false), + ('2', 'bar', 456, true), + ('3', 'baz', 789, false), + ('4', null, null, null)` query = fmt.Sprintf(query, table) _, err = conn.Exec(ctx, query) is.NoErr(err) From df71d711a410714d0c99c40aa0423bd9999fb0ce Mon Sep 17 00:00:00 2001 From: Haris Osmanagic Date: Wed, 25 Dec 2024 18:38:53 +0100 Subject: [PATCH 03/60] improve tests --- source_integration_test.go | 102 +++++++++++++++++++++++++++++-------- test/helper.go | 11 ++-- 2 files changed, 88 insertions(+), 25 deletions(-) diff --git a/source_integration_test.go b/source_integration_test.go index 8cf8366..3ac2306 100644 --- a/source_integration_test.go +++ b/source_integration_test.go @@ -16,8 +16,11 @@ package postgres import ( "context" + "fmt" "testing" + "time" + "github.com/conduitio/conduit-commons/opencdc" "github.com/conduitio/conduit-connector-postgres/source/logrepl" "github.com/conduitio/conduit-connector-postgres/test" "github.com/matryer/is" @@ -25,9 +28,9 @@ import ( func TestSource_Read(t *testing.T) { is := is.New(t) - ctx := test.Context(t) - conn := test.ConnectSimple(ctx, t, test.RepmgrConnString) - tableName := test.SetupTestTable(ctx, t, conn) + ctx, _ := context.WithTimeout(test.Context(t), 3000*time.Second) + + tableName := prepareSourceIntegrationTestTable(ctx, t) slotName := "conduitslot1" publicationName := "conduitpub1" @@ -56,25 +59,84 @@ func TestSource_Read(t *testing.T) { is.NoErr(s.Teardown(ctx)) }) - for i := 0; i < 4; i++ { - gotRecord, err := s.Read(ctx) - is.NoErr(err) + gotRecord, err := s.Read(ctx) + is.NoErr(err) + err = s.Ack(ctx, gotRecord.Position) + is.NoErr(err) + assertRecordOK(is, tableName, gotRecord) - is.True(gotRecord.Key != nil) - is.True(gotRecord.Payload.After != nil) + insertRow(ctx, t, tableName, 2) - payloadSchemaSubject, err := gotRecord.Metadata.GetPayloadSchemaSubject() - is.NoErr(err) - is.Equal(tableName+"_payload", payloadSchemaSubject) - payloadSchemaVersion, err := gotRecord.Metadata.GetPayloadSchemaVersion() - is.NoErr(err) - is.Equal(1, payloadSchemaVersion) + gotRecord, err = s.Read(ctx) + is.NoErr(err) + err = s.Ack(ctx, gotRecord.Position) + is.NoErr(err) + assertRecordOK(is, tableName, gotRecord) +} - keySchemaSubject, err := gotRecord.Metadata.GetKeySchemaSubject() - is.NoErr(err) - is.Equal(tableName+"_key", keySchemaSubject) - keySchemaVersion, err := gotRecord.Metadata.GetKeySchemaVersion() +func assertRecordOK(is *is.I, tableName string, gotRecord opencdc.Record) { + is.True(gotRecord.Key != nil) + is.True(gotRecord.Payload.After != nil) + + payloadSchemaSubject, err := gotRecord.Metadata.GetPayloadSchemaSubject() + is.NoErr(err) + is.Equal(tableName+"_payload", payloadSchemaSubject) + payloadSchemaVersion, err := gotRecord.Metadata.GetPayloadSchemaVersion() + is.NoErr(err) + is.Equal(1, payloadSchemaVersion) + + keySchemaSubject, err := gotRecord.Metadata.GetKeySchemaSubject() + is.NoErr(err) + is.Equal(tableName+"_key", keySchemaSubject) + keySchemaVersion, err := gotRecord.Metadata.GetKeySchemaVersion() + is.NoErr(err) + is.Equal(1, keySchemaVersion) +} + +func prepareSourceIntegrationTestTable(ctx context.Context, t *testing.T) string { + is := is.New(t) + + conn := test.ConnectSimple(ctx, t, test.RepmgrConnString) + table := test.RandomIdentifier(t) + + query := fmt.Sprintf(` + CREATE TABLE %s ( + id bigserial PRIMARY KEY, + key bytea, + column1 varchar(256), + column2 integer, + column3 boolean, + column4 varchar(256) NOT NULL, + column5 integer NOT NULL, + column6 boolean NOT NULL + )`, table) + _, err := conn.Exec(ctx, query) + is.NoErr(err) + + t.Cleanup(func() { + query := `DROP TABLE %s` + query = fmt.Sprintf(query, table) + _, err := conn.Exec(context.Background(), query) is.NoErr(err) - is.Equal(1, keySchemaVersion) - } + }) + + insertRow(ctx, t, table, 1) + + return table +} + +func insertRow(ctx context.Context, t *testing.T, table string, rowNumber int) { + is := is.New(t) + conn := test.ConnectSimple(ctx, t, test.RepmgrConnString) + + query := fmt.Sprintf( + `INSERT INTO %s (key, column1, column2, column3, column4, column5, column6) + VALUES ('%v', null, null, null, 'foo-%v', %d, false)`, + table, + rowNumber, + rowNumber, + 100+rowNumber, + ) + _, err := conn.Exec(ctx, query) + is.NoErr(err) } diff --git a/test/helper.go b/test/helper.go index 756ebe1..9ebc8e1 100644 --- a/test/helper.go +++ b/test/helper.go @@ -139,6 +139,7 @@ const testTableCreateQuery = ` column1 varchar(256), column2 integer, column3 boolean, + column4 numeric(16,3), column5 numeric(5) )` @@ -189,11 +190,11 @@ func SetupTestTable(ctx context.Context, t *testing.T, conn Querier) string { }) query = ` - INSERT INTO %s (key, column1, column2, column3) - VALUES ('1', 'foo', 123, false), - ('2', 'bar', 456, true), - ('3', 'baz', 789, false), - ('4', null, null, null)` + INSERT INTO %s (key, column1, column2, column3, column4, column5) + VALUES ('1', 'foo', 123, false, 12.2, 4), + ('2', 'bar', 456, true, 13.42, 8), + ('3', 'baz', 789, false, null, 9), + ('4', null, null, null, 91.1, null)` query = fmt.Sprintf(query, table) _, err = conn.Exec(ctx, query) is.NoErr(err) From 74e3d99dd96913a813026ba6d5f7bd40f34aeb7f Mon Sep 17 00:00:00 2001 From: Haris Osmanagic Date: Thu, 26 Dec 2024 18:55:20 +0100 Subject: [PATCH 04/60] get table info --- source/common/table_info.go | 116 ++++++++++++++++++++++++++++++++ source/logrepl/cdc.go | 8 ++- source/logrepl/handler.go | 13 +++- source/schema/avro.go | 10 +-- source/snapshot/fetch_worker.go | 79 ++++------------------ 5 files changed, 150 insertions(+), 76 deletions(-) create mode 100644 source/common/table_info.go diff --git a/source/common/table_info.go b/source/common/table_info.go new file mode 100644 index 0000000..f802110 --- /dev/null +++ b/source/common/table_info.go @@ -0,0 +1,116 @@ +// Copyright © 2024 Meroxa, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package common + +import ( + "context" + "fmt" + + sdk "github.com/conduitio/conduit-connector-sdk" + "github.com/jackc/pgx/v5/pgxpool" +) + +type TableInfo struct { + Name string + Columns map[string]*ColumnInfo +} + +func NewTableInfo(tableName string) *TableInfo { + return &TableInfo{ + Name: tableName, + Columns: make(map[string]*ColumnInfo), + } +} + +type ColumnInfo struct { + IsNotNull bool +} + +type TableInfoFetcher struct { + connPool *pgxpool.Pool + tableInfo map[string]*TableInfo +} + +func NewTableInfoFetcher(connPool *pgxpool.Pool) *TableInfoFetcher { + return &TableInfoFetcher{ + connPool: connPool, + tableInfo: make(map[string]*TableInfo), + } +} + +func (i TableInfoFetcher) Refresh(ctx context.Context, tableName string) error { + tx, err := i.connPool.Begin(ctx) + if err != nil { + return fmt.Errorf("failed to start tx for getting table info: %w", err) + } + defer func() { + if err := tx.Rollback(ctx); err != nil { + sdk.Logger(ctx).Warn(). + Err(err). + Msgf("error on tx rollback for getting table info") + } + }() + + query := ` + SELECT a.attname AS column_name, a.attnotnull AS is_not_null + FROM pg_index i + JOIN pg_attribute a ON a.attrelid = i.indrelid + JOIN pg_class c ON a.attrelid = c.oid + WHERE c.relname = $1 + AND a.attnum > 0 + AND NOT a.attisdropped + ORDER BY a.attnum; + ` + + rows, err := tx.Query(context.Background(), query, tableName) + if err != nil { + sdk.Logger(ctx). + Err(err). + Str("query", query). + Msgf("failed to execute table info query") + + return fmt.Errorf("failed to get table info: %w", err) + } + defer rows.Close() + + ti := NewTableInfo(tableName) + for rows.Next() { + var columnName string + var isNotNull bool + + err := rows.Scan(&columnName, &isNotNull) + if err != nil { + return fmt.Errorf("failed to scan table info row: %w", err) + } + + ci := ti.Columns[columnName] + if ci == nil { + ci = &ColumnInfo{} + ti.Columns[columnName] = ci + } + ci.IsNotNull = isNotNull + } + + if err := rows.Err(); err != nil { + return fmt.Errorf("failed to get table info rows: %w", err) + } + + i.tableInfo[tableName] = ti + return nil +} + +func (i TableInfoFetcher) GetTable(name string) *TableInfo { + return i.tableInfo[name] +} diff --git a/source/logrepl/cdc.go b/source/logrepl/cdc.go index 056fec9..d76acf2 100644 --- a/source/logrepl/cdc.go +++ b/source/logrepl/cdc.go @@ -20,6 +20,7 @@ import ( "fmt" "github.com/conduitio/conduit-commons/opencdc" + "github.com/conduitio/conduit-connector-postgres/source/common" "github.com/conduitio/conduit-connector-postgres/source/logrepl/internal" "github.com/conduitio/conduit-connector-postgres/source/position" sdk "github.com/conduitio/conduit-connector-sdk" @@ -65,7 +66,12 @@ func NewCDCIterator(ctx context.Context, pool *pgxpool.Pool, c CDCConfig) (*CDCI } records := make(chan opencdc.Record) - handler := NewCDCHandler(internal.NewRelationSet(), c.TableKeys, records) + handler := NewCDCHandler( + internal.NewRelationSet(), + common.NewTableInfoFetcher(pool), + c.TableKeys, + records, + ) sub, err := internal.CreateSubscription( ctx, diff --git a/source/logrepl/handler.go b/source/logrepl/handler.go index d1a504d..b69655f 100644 --- a/source/logrepl/handler.go +++ b/source/logrepl/handler.go @@ -20,6 +20,7 @@ import ( "github.com/conduitio/conduit-commons/opencdc" cschema "github.com/conduitio/conduit-commons/schema" + "github.com/conduitio/conduit-connector-postgres/source/common" "github.com/conduitio/conduit-connector-postgres/source/logrepl/internal" "github.com/conduitio/conduit-connector-postgres/source/position" "github.com/conduitio/conduit-connector-postgres/source/schema" @@ -36,15 +37,17 @@ type CDCHandler struct { out chan<- opencdc.Record lastTXLSN pglogrepl.LSN + tableInfo *common.TableInfoFetcher keySchemas map[string]cschema.Schema payloadSchemas map[string]cschema.Schema } -func NewCDCHandler(rs *internal.RelationSet, tableKeys map[string]string, out chan<- opencdc.Record) *CDCHandler { +func NewCDCHandler(rs *internal.RelationSet, tableInfo *common.TableInfoFetcher, tableKeys map[string]string, out chan<- opencdc.Record) *CDCHandler { return &CDCHandler{ tableKeys: tableKeys, relationSet: rs, out: out, + tableInfo: tableInfo, keySchemas: make(map[string]cschema.Schema), payloadSchemas: make(map[string]cschema.Schema), } @@ -62,6 +65,10 @@ func (h *CDCHandler) Handle(ctx context.Context, m pglogrepl.Message, lsn pglogr case *pglogrepl.RelationMessage: // We have to add the Relations to our Set so that we can decode our own output h.relationSet.Add(m) + err := h.tableInfo.Refresh(ctx, m.RelationName) + if err != nil { + return 0, fmt.Errorf("failed to refresh table info: %w", err) + } case *pglogrepl.InsertMessage: if err := h.handleInsert(ctx, m, lsn); err != nil { return 0, fmt.Errorf("logrepl handler insert: %w", err) @@ -246,7 +253,7 @@ func (*CDCHandler) buildPosition(lsn pglogrepl.LSN) opencdc.Position { // when usage of avro schema is requested. func (h *CDCHandler) updateAvroSchema(ctx context.Context, rel *pglogrepl.RelationMessage) error { // Payload schema - avroPayloadSch, err := schema.Avro.ExtractLogrepl(rel.RelationName+"_payload", rel) + avroPayloadSch, err := schema.Avro.ExtractLogrepl(rel.RelationName+"_payload", rel, h.tableInfo.GetTable(rel.RelationName)) if err != nil { return fmt.Errorf("failed to extract payload schema: %w", err) } @@ -262,7 +269,7 @@ func (h *CDCHandler) updateAvroSchema(ctx context.Context, rel *pglogrepl.Relati h.payloadSchemas[rel.RelationName] = ps // Key schema - avroKeySch, err := schema.Avro.ExtractLogrepl(rel.RelationName+"_key", rel, h.tableKeys[rel.RelationName]) + avroKeySch, err := schema.Avro.ExtractLogrepl(rel.RelationName+"_key", rel, h.tableInfo.GetTable(rel.RelationName), h.tableKeys[rel.RelationName]) if err != nil { return fmt.Errorf("failed to extract key schema: %w", err) } diff --git a/source/schema/avro.go b/source/schema/avro.go index f43ac44..fb1e3da 100644 --- a/source/schema/avro.go +++ b/source/schema/avro.go @@ -19,6 +19,7 @@ import ( "fmt" "slices" + "github.com/conduitio/conduit-connector-postgres/source/common" "github.com/hamba/avro/v2" "github.com/jackc/pglogrepl" "github.com/jackc/pgx/v5/pgconn" @@ -63,7 +64,7 @@ type avroExtractor struct { // ExtractLogrepl extracts an Avro schema from the given pglogrepl.RelationMessage. // If `fieldNames` are specified, then only the given fields will be included in the schema. -func (a *avroExtractor) ExtractLogrepl(schemaName string, rel *pglogrepl.RelationMessage, fieldNames ...string) (*avro.RecordSchema, error) { +func (a *avroExtractor) ExtractLogrepl(schemaName string, rel *pglogrepl.RelationMessage, tableInfo *common.TableInfo, fieldNames ...string) (*avro.RecordSchema, error) { var fields []pgconn.FieldDescription for i := range rel.Columns { @@ -74,13 +75,12 @@ func (a *avroExtractor) ExtractLogrepl(schemaName string, rel *pglogrepl.Relatio }) } - // todo terribly bad, only a temporary thing - return a.Extract(schemaName, make(map[string]bool), fields, fieldNames...) + return a.Extract(schemaName, tableInfo, fields, fieldNames...) } // Extract extracts an Avro schema from the given Postgres field descriptions. // If `fieldNames` are specified, then only the given fields will be included in the schema. -func (a *avroExtractor) Extract(schemaName string, notNullMap map[string]bool, fields []pgconn.FieldDescription, fieldNames ...string) (*avro.RecordSchema, error) { +func (a *avroExtractor) Extract(schemaName string, tableInfo *common.TableInfo, fields []pgconn.FieldDescription, fieldNames ...string) (*avro.RecordSchema, error) { var avroFields []*avro.Field for _, f := range fields { @@ -93,7 +93,7 @@ func (a *avroExtractor) Extract(schemaName string, notNullMap map[string]bool, f return nil, fmt.Errorf("field %q with OID %d cannot be resolved", f.Name, f.DataTypeOID) } - s, err := a.extractType(t, f.TypeModifier, notNullMap[f.Name]) + s, err := a.extractType(t, f.TypeModifier, tableInfo.Columns[f.Name].IsNotNull) if err != nil { return nil, err } diff --git a/source/snapshot/fetch_worker.go b/source/snapshot/fetch_worker.go index bf7792e..de49fa3 100644 --- a/source/snapshot/fetch_worker.go +++ b/source/snapshot/fetch_worker.go @@ -24,6 +24,7 @@ import ( "github.com/conduitio/conduit-commons/opencdc" cschema "github.com/conduitio/conduit-commons/schema" + "github.com/conduitio/conduit-connector-postgres/source/common" "github.com/conduitio/conduit-connector-postgres/source/position" "github.com/conduitio/conduit-connector-postgres/source/schema" "github.com/conduitio/conduit-connector-postgres/source/types" @@ -93,8 +94,8 @@ type FetchWorker struct { out chan<- FetchData // notNullMap maps column names to if the column is NOT NULL. - notNullMap map[string]bool - keySchema *cschema.Schema + tableInfoFetcher *common.TableInfoFetcher + keySchema *cschema.Schema payloadSchema *cschema.Schema snapshotEnd int64 @@ -104,10 +105,11 @@ type FetchWorker struct { func NewFetchWorker(db *pgxpool.Pool, out chan<- FetchData, c FetchConfig) *FetchWorker { f := &FetchWorker{ - conf: c, - db: db, - out: out, - cursorName: "fetcher_" + strings.ReplaceAll(uuid.NewString(), "-", ""), + conf: c, + db: db, + out: out, + tableInfoFetcher: common.NewTableInfoFetcher(db), + cursorName: "fetcher_" + strings.ReplaceAll(uuid.NewString(), "-", ""), } if f.conf.FetchSize == 0 { @@ -130,11 +132,10 @@ func NewFetchWorker(db *pgxpool.Pool, out chan<- FetchData, c FetchConfig) *Fetc // * Table and keys exist // * Key is a primary key func (f *FetchWorker) Init(ctx context.Context) error { - notNullMap, err := f.getNotNullMap(ctx) + err := f.tableInfoFetcher.Refresh(ctx, f.conf.Table) if err != nil { - return fmt.Errorf("could not initialize nullability map: %w", err) + return fmt.Errorf("failed to refresh table info: %w", err) } - f.notNullMap = notNullMap err = f.validate(ctx) if err != nil { @@ -144,62 +145,6 @@ func (f *FetchWorker) Init(ctx context.Context) error { return nil } -// getNotNullMap returns a map that contains information about nullability of columns. -// Keys are column names, values are booleans showing if the column is NOT NULL or not. -func (f *FetchWorker) getNotNullMap(ctx context.Context) (map[string]bool, error) { - tableName := f.conf.Table - tx, err := f.db.Begin(ctx) - if err != nil { - return nil, fmt.Errorf("failed to start tx for getting column nullability info: %w", err) - } - defer func() { - if err := tx.Rollback(ctx); err != nil { - sdk.Logger(ctx).Warn(). - Err(err). - Msgf("error on tx rollback for getting column nullability info, cursor name: %q", f.cursorName) - } - }() - - query := ` - SELECT a.attname AS column_name, a.attnotnull AS is_not_null - FROM pg_index i - JOIN pg_attribute a ON a.attrelid = i.indrelid - JOIN pg_class c ON a.attrelid = c.oid - WHERE c.relname = $1 - AND a.attnum > 0 - AND NOT a.attisdropped - ORDER BY a.attnum; - ` - - rows, err := tx.Query(context.Background(), query, tableName) - if err != nil { - return nil, err - } - defer rows.Close() - - // Map to store column nullability and primary key info - nullabilityMap := make(map[string]bool) - - for rows.Next() { - var columnName string - var isNotNull bool - - err := rows.Scan(&columnName, &isNotNull) - if err != nil { - return nil, err - } - - // Store info in nested map - nullabilityMap[columnName] = isNotNull - } - - if err := rows.Err(); err != nil { - return nil, err - } - - return nullabilityMap, nil -} - func (f *FetchWorker) validate(ctx context.Context) error { if err := f.conf.Validate(); err != nil { return fmt.Errorf("failed to validate config: %w", err) @@ -546,7 +491,7 @@ func (f *FetchWorker) extractSchemas(ctx context.Context, fields []pgconn.FieldD sdk.Logger(ctx).Debug(). Msgf("extracting payload schema for %v fields in %v", len(fields), f.conf.Table) - avroPayloadSch, err := schema.Avro.Extract(f.conf.Table+"_payload", f.notNullMap, fields) + avroPayloadSch, err := schema.Avro.Extract(f.conf.Table+"_payload", f.tableInfoFetcher.GetTable(f.conf.Table), fields) if err != nil { return fmt.Errorf("failed to extract payload schema for table %v: %w", f.conf.Table, err) } @@ -566,7 +511,7 @@ func (f *FetchWorker) extractSchemas(ctx context.Context, fields []pgconn.FieldD sdk.Logger(ctx).Debug(). Msgf("extracting schema for key %v in %v", f.conf.Key, f.conf.Table) - avroKeySch, err := schema.Avro.Extract(f.conf.Table+"_key", f.notNullMap, fields, f.conf.Key) + avroKeySch, err := schema.Avro.Extract(f.conf.Table+"_key", f.tableInfoFetcher.GetTable(f.conf.Table), fields, f.conf.Key) if err != nil { return fmt.Errorf("failed to extract key schema for table %v: %w", f.conf.Table, err) } From 5924975bb2e6b6da296c6241f741bc3937921091 Mon Sep 17 00:00:00 2001 From: Haris Osmanagic Date: Thu, 26 Dec 2024 19:17:05 +0100 Subject: [PATCH 05/60] fixing tests --- ...{avro_test.go => avro_integration_test.go} | 11 ++++- source/snapshot/fetch_worker_test.go | 45 ++++++++++--------- 2 files changed, 35 insertions(+), 21 deletions(-) rename source/schema/{avro_test.go => avro_integration_test.go} (95%) diff --git a/source/schema/avro_test.go b/source/schema/avro_integration_test.go similarity index 95% rename from source/schema/avro_test.go rename to source/schema/avro_integration_test.go index 448e2a9..345316d 100644 --- a/source/schema/avro_test.go +++ b/source/schema/avro_integration_test.go @@ -23,11 +23,13 @@ import ( "testing" "time" + "github.com/conduitio/conduit-connector-postgres/source/common" "github.com/conduitio/conduit-connector-postgres/source/types" "github.com/conduitio/conduit-connector-postgres/test" "github.com/hamba/avro/v2" "github.com/jackc/pgx/v5/pgconn" "github.com/jackc/pgx/v5/pgtype" + "github.com/jackc/pgx/v5/pgxpool" "github.com/matryer/is" ) @@ -36,7 +38,14 @@ func Test_AvroExtract(t *testing.T) { is := is.New(t) c := test.ConnectSimple(ctx, t, test.RegularConnString) + connPool, err := pgxpool.New(ctx, test.RegularConnString) + is.NoErr(err) + table := setupAvroTestTable(ctx, t, c) + tableInfoFetcher := common.NewTableInfoFetcher(connPool) + err = tableInfoFetcher.Refresh(ctx, table) + is.NoErr(err) + insertAvroTestRow(ctx, t, c, table) rows, err := c.Query(ctx, "SELECT * FROM "+table) @@ -50,7 +59,7 @@ func Test_AvroExtract(t *testing.T) { fields := rows.FieldDescriptions() - sch, err := Avro.Extract(table, fields) + sch, err := Avro.Extract(table, tableInfoFetcher.GetTable(table), fields) is.NoErr(err) t.Run("schema is parsable", func(t *testing.T) { diff --git a/source/snapshot/fetch_worker_test.go b/source/snapshot/fetch_worker_test.go index fcc0f60..e518603 100644 --- a/source/snapshot/fetch_worker_test.go +++ b/source/snapshot/fetch_worker_test.go @@ -126,26 +126,28 @@ func Test_FetcherValidate(t *testing.T) { t.Run("success", func(t *testing.T) { is := is.New(t) - f := FetchWorker{ - db: pool, - conf: FetchConfig{ + f := NewFetchWorker( + pool, + make(chan<- FetchData), + FetchConfig{ Table: table, Key: "id", }, - } + ) is.NoErr(f.Init(ctx)) }) t.Run("table missing", func(t *testing.T) { is := is.New(t) - f := FetchWorker{ - db: pool, - conf: FetchConfig{ + f := NewFetchWorker( + pool, + make(chan<- FetchData), + FetchConfig{ Table: "missing_table", Key: "id", }, - } + ) err := f.Init(ctx) is.True(err != nil) @@ -154,13 +156,14 @@ func Test_FetcherValidate(t *testing.T) { t.Run("key is wrong type", func(t *testing.T) { is := is.New(t) - f := FetchWorker{ - db: pool, - conf: FetchConfig{ + f := NewFetchWorker( + pool, + make(chan<- FetchData), + FetchConfig{ Table: table, Key: "column3", }, - } + ) err := f.Init(ctx) is.True(err != nil) @@ -169,13 +172,14 @@ func Test_FetcherValidate(t *testing.T) { t.Run("key is not pk", func(t *testing.T) { is := is.New(t) - f := FetchWorker{ - db: pool, - conf: FetchConfig{ + f := NewFetchWorker( + pool, + make(chan<- FetchData), + FetchConfig{ Table: table, Key: "column2", }, - } + ) err := f.Init(ctx) is.NoErr(err) // no error, only a warning @@ -183,13 +187,14 @@ func Test_FetcherValidate(t *testing.T) { t.Run("missing key", func(t *testing.T) { is := is.New(t) - f := FetchWorker{ - db: pool, - conf: FetchConfig{ + f := NewFetchWorker( + pool, + make(chan<- FetchData), + FetchConfig{ Table: table, Key: "missing_key", }, - } + ) err := f.Init(ctx) is.True(err != nil) From cc1ecf8117e854b6a22188c089a79444be742fb0 Mon Sep 17 00:00:00 2001 From: Haris Osmanagic Date: Thu, 26 Dec 2024 19:27:39 +0100 Subject: [PATCH 06/60] rename --- .../{fetch_worker_test.go => fetch_worker_integration_test.go} | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename source/snapshot/{fetch_worker_test.go => fetch_worker_integration_test.go} (100%) diff --git a/source/snapshot/fetch_worker_test.go b/source/snapshot/fetch_worker_integration_test.go similarity index 100% rename from source/snapshot/fetch_worker_test.go rename to source/snapshot/fetch_worker_integration_test.go From 0d1533b42874b71889e231108634383c3b2ef304 Mon Sep 17 00:00:00 2001 From: Haris Osmanagic Date: Fri, 27 Dec 2024 14:42:25 +0100 Subject: [PATCH 07/60] updated integration test --- source/common/table_info.go | 12 +++--- source_integration_test.go | 79 ++++++++++++++++++++++++++++++------- 2 files changed, 70 insertions(+), 21 deletions(-) diff --git a/source/common/table_info.go b/source/common/table_info.go index f802110..c016444 100644 --- a/source/common/table_info.go +++ b/source/common/table_info.go @@ -64,13 +64,11 @@ func (i TableInfoFetcher) Refresh(ctx context.Context, tableName string) error { }() query := ` - SELECT a.attname AS column_name, a.attnotnull AS is_not_null - FROM pg_index i - JOIN pg_attribute a ON a.attrelid = i.indrelid - JOIN pg_class c ON a.attrelid = c.oid - WHERE c.relname = $1 - AND a.attnum > 0 - AND NOT a.attisdropped + SELECT a.attname as column_name, a.attnotnull as is_not_null + FROM pg_catalog.pg_attribute a + WHERE a.attrelid = $1::regclass + AND a.attnum > 0 + AND NOT a.attisdropped ORDER BY a.attnum; ` diff --git a/source_integration_test.go b/source_integration_test.go index 3ac2306..dc780a9 100644 --- a/source_integration_test.go +++ b/source_integration_test.go @@ -99,17 +99,35 @@ func prepareSourceIntegrationTestTable(ctx context.Context, t *testing.T) string conn := test.ConnectSimple(ctx, t, test.RepmgrConnString) table := test.RandomIdentifier(t) - query := fmt.Sprintf(` - CREATE TABLE %s ( - id bigserial PRIMARY KEY, - key bytea, - column1 varchar(256), - column2 integer, - column3 boolean, - column4 varchar(256) NOT NULL, - column5 integer NOT NULL, - column6 boolean NOT NULL - )`, table) + query := fmt.Sprintf(`CREATE TABLE %s ( + id bigserial PRIMARY KEY, + col_bytea bytea, + col_bytea_not_null bytea NOT NULL, + col_varchar varchar(10), + col_varchar_not_null varchar(10) NOT NULL, + col_date date, + col_date_not_null date NOT NULL, + col_float4 float4, + col_float4_not_null float4 NOT NULL, + col_float8 float8, + col_float8_not_null float8 NOT NULL, + col_int2 int2, + col_int2_not_null int2 NOT NULL, + col_int4 int4, + col_int4_not_null int4 NOT NULL, + col_int8 int8, + col_int8_not_null int8 NOT NULL, + col_numeric numeric(8,2), + -- col_numeric_not_null numeric(8,2) NOT NULL, + col_text text, + col_text_not_null text NOT NULL, + col_timestamp timestamp, + col_timestamp_not_null timestamp NOT NULL, + col_timestamptz timestamptz, + col_timestamptz_not_null timestamptz NOT NULL, + col_uuid uuid, + col_uuid_not_null uuid NOT NULL +)`, table) _, err := conn.Exec(ctx, query) is.NoErr(err) @@ -130,12 +148,45 @@ func insertRow(ctx context.Context, t *testing.T, table string, rowNumber int) { conn := test.ConnectSimple(ctx, t, test.RepmgrConnString) query := fmt.Sprintf( - `INSERT INTO %s (key, column1, column2, column3, column4, column5, column6) - VALUES ('%v', null, null, null, 'foo-%v', %d, false)`, + `INSERT INTO %s ( + col_bytea_not_null, + col_varchar_not_null, + col_date_not_null, + col_float4_not_null, + col_float8_not_null, + col_int2_not_null, + col_int4_not_null, + col_int8_not_null, + -- col_numeric_not_null, + col_text_not_null, + col_timestamp_not_null, + col_timestamptz_not_null, + col_uuid_not_null + ) VALUES ( + '%s'::bytea, -- col_bytea_not_null + 'foo-%v', -- col_varchar_not_null + now(), -- col_date_not_null + %f, -- col_float4_not_null + %f, -- col_float8_not_null + %d, -- col_int2_not_null + %d, -- col_int4_not_null + %d, -- col_int8_not_null + -- %f, -- col_numeric_not_null + 'bar-%v', -- col_text_not_null + now(), -- col_timestamp_not_null + now(), -- col_timestamptz_not_null + gen_random_uuid() -- col_uuid_not_null + )`, table, + fmt.Sprintf("col_bytea_-%v", rowNumber), rowNumber, + float32(rowNumber)/10, + float64(rowNumber)/10, + rowNumber%32768, + rowNumber, + rowNumber, + float64(100+rowNumber)/10, rowNumber, - 100+rowNumber, ) _, err := conn.Exec(ctx, query) is.NoErr(err) From e961a17bd3a43b3b5e0065b6f507bcf7d1810fca Mon Sep 17 00:00:00 2001 From: Haris Osmanagic Date: Fri, 27 Dec 2024 14:55:38 +0100 Subject: [PATCH 08/60] first validate, then refresh tables --- source/snapshot/fetch_worker.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/source/snapshot/fetch_worker.go b/source/snapshot/fetch_worker.go index de49fa3..9aea011 100644 --- a/source/snapshot/fetch_worker.go +++ b/source/snapshot/fetch_worker.go @@ -132,14 +132,14 @@ func NewFetchWorker(db *pgxpool.Pool, out chan<- FetchData, c FetchConfig) *Fetc // * Table and keys exist // * Key is a primary key func (f *FetchWorker) Init(ctx context.Context) error { - err := f.tableInfoFetcher.Refresh(ctx, f.conf.Table) + err := f.validate(ctx) if err != nil { - return fmt.Errorf("failed to refresh table info: %w", err) + return fmt.Errorf("validation failed: %w", err) } - err = f.validate(ctx) + err = f.tableInfoFetcher.Refresh(ctx, f.conf.Table) if err != nil { - return fmt.Errorf("validation failed: %w", err) + return fmt.Errorf("failed to refresh table info: %w", err) } return nil From afbe85129532d84af3d6178caea72cced85e342e Mon Sep 17 00:00:00 2001 From: Haris Osmanagic Date: Fri, 27 Dec 2024 15:54:07 +0100 Subject: [PATCH 09/60] linter --- source/schema/avro_integration_test.go | 4 ++-- source_integration_test.go | 3 +-- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/source/schema/avro_integration_test.go b/source/schema/avro_integration_test.go index 345316d..40ea3e2 100644 --- a/source/schema/avro_integration_test.go +++ b/source/schema/avro_integration_test.go @@ -24,12 +24,12 @@ import ( "time" "github.com/conduitio/conduit-connector-postgres/source/common" + "github.com/conduitio/conduit-connector-postgres/source/cpool" "github.com/conduitio/conduit-connector-postgres/source/types" "github.com/conduitio/conduit-connector-postgres/test" "github.com/hamba/avro/v2" "github.com/jackc/pgx/v5/pgconn" "github.com/jackc/pgx/v5/pgtype" - "github.com/jackc/pgx/v5/pgxpool" "github.com/matryer/is" ) @@ -38,7 +38,7 @@ func Test_AvroExtract(t *testing.T) { is := is.New(t) c := test.ConnectSimple(ctx, t, test.RegularConnString) - connPool, err := pgxpool.New(ctx, test.RegularConnString) + connPool, err := cpool.New(ctx, test.RegularConnString) is.NoErr(err) table := setupAvroTestTable(ctx, t, c) diff --git a/source_integration_test.go b/source_integration_test.go index dc780a9..a783126 100644 --- a/source_integration_test.go +++ b/source_integration_test.go @@ -18,7 +18,6 @@ import ( "context" "fmt" "testing" - "time" "github.com/conduitio/conduit-commons/opencdc" "github.com/conduitio/conduit-connector-postgres/source/logrepl" @@ -28,7 +27,7 @@ import ( func TestSource_Read(t *testing.T) { is := is.New(t) - ctx, _ := context.WithTimeout(test.Context(t), 3000*time.Second) + ctx := test.Context(t) tableName := prepareSourceIntegrationTestTable(ctx, t) slotName := "conduitslot1" From 53b79f67db8cfe71e72d20a078c646193d6221d6 Mon Sep 17 00:00:00 2001 From: Haris Osmanagic Date: Fri, 27 Dec 2024 20:01:25 +0100 Subject: [PATCH 10/60] support timestamps --- source/logrepl/handler.go | 16 ++++---- source/logrepl/internal/relationset.go | 9 ++-- source/snapshot/fetch_worker.go | 13 ++++-- source/types/types.go | 33 ++++++++++++++- source_integration_test.go | 57 ++++++++++++++++++++++++-- 5 files changed, 108 insertions(+), 20 deletions(-) diff --git a/source/logrepl/handler.go b/source/logrepl/handler.go index b69655f..d2a0cb5 100644 --- a/source/logrepl/handler.go +++ b/source/logrepl/handler.go @@ -107,15 +107,15 @@ func (h *CDCHandler) handleInsert( return fmt.Errorf("failed getting relation %v: %w", msg.RelationID, err) } - newValues, err := h.relationSet.Values(msg.RelationID, msg.Tuple) - if err != nil { - return fmt.Errorf("failed to decode new values: %w", err) - } - if err := h.updateAvroSchema(ctx, rel); err != nil { return fmt.Errorf("failed to update avro schema: %w", err) } + newValues, err := h.relationSet.Values(msg.RelationID, msg.Tuple, h.tableInfo.GetTable(rel.RelationName)) + if err != nil { + return fmt.Errorf("failed to decode new values: %w", err) + } + rec := sdk.Util.Source.NewRecordCreate( h.buildPosition(lsn), h.buildRecordMetadata(rel), @@ -139,7 +139,7 @@ func (h *CDCHandler) handleUpdate( return err } - newValues, err := h.relationSet.Values(msg.RelationID, msg.NewTuple) + newValues, err := h.relationSet.Values(msg.RelationID, msg.NewTuple, h.tableInfo.GetTable(rel.RelationName)) if err != nil { return fmt.Errorf("failed to decode new values: %w", err) } @@ -148,7 +148,7 @@ func (h *CDCHandler) handleUpdate( return fmt.Errorf("failed to update avro schema: %w", err) } - oldValues, err := h.relationSet.Values(msg.RelationID, msg.OldTuple) + oldValues, err := h.relationSet.Values(msg.RelationID, msg.OldTuple, h.tableInfo.GetTable(rel.RelationName)) if err != nil { // this is not a critical error, old values are optional, just log it // we use level "trace" intentionally to not clog up the logs in production @@ -179,7 +179,7 @@ func (h *CDCHandler) handleDelete( return err } - oldValues, err := h.relationSet.Values(msg.RelationID, msg.OldTuple) + oldValues, err := h.relationSet.Values(msg.RelationID, msg.OldTuple, h.tableInfo.GetTable(rel.RelationName)) if err != nil { return fmt.Errorf("failed to decode old values: %w", err) } diff --git a/source/logrepl/internal/relationset.go b/source/logrepl/internal/relationset.go index f87336d..c872d8d 100644 --- a/source/logrepl/internal/relationset.go +++ b/source/logrepl/internal/relationset.go @@ -18,6 +18,7 @@ import ( "errors" "fmt" + "github.com/conduitio/conduit-connector-postgres/source/common" "github.com/conduitio/conduit-connector-postgres/source/types" "github.com/jackc/pglogrepl" "github.com/jackc/pgx/v5/pgtype" @@ -50,7 +51,7 @@ func (rs *RelationSet) Get(id uint32) (*pglogrepl.RelationMessage, error) { return msg, nil } -func (rs *RelationSet) Values(id uint32, row *pglogrepl.TupleData) (map[string]any, error) { +func (rs *RelationSet) Values(id uint32, row *pglogrepl.TupleData, tableInfo *common.TableInfo) (map[string]any, error) { if row == nil { return nil, errors.New("no tuple data") } @@ -65,7 +66,7 @@ func (rs *RelationSet) Values(id uint32, row *pglogrepl.TupleData) (map[string]a // assert same number of row and rel columns for i, tuple := range row.Columns { col := rel.Columns[i] - v, decodeErr := rs.decodeValue(col, tuple.Data) + v, decodeErr := rs.decodeValue(col, tableInfo.Columns[col.Name], tuple.Data) if decodeErr != nil { return nil, fmt.Errorf("failed to decode value for column %q: %w", col.Name, err) } @@ -84,7 +85,7 @@ func (rs *RelationSet) oidToCodec(id uint32) pgtype.Codec { return dt.Codec } -func (rs *RelationSet) decodeValue(col *pglogrepl.RelationMessageColumn, data []byte) (any, error) { +func (rs *RelationSet) decodeValue(col *pglogrepl.RelationMessageColumn, colInfo *common.ColumnInfo, data []byte) (any, error) { decoder := rs.oidToCodec(col.DataType) // This workaround is due to an issue in pgx v5.7.1. // Namely, that version introduces an XML codec @@ -103,7 +104,7 @@ func (rs *RelationSet) decodeValue(col *pglogrepl.RelationMessageColumn, data [] return nil, fmt.Errorf("failed to decode value of pgtype %v: %w", col.DataType, err) } - v, err := types.Format(col.DataType, val) + v, err := types.Format(col.DataType, val, colInfo.IsNotNull) if err != nil { return nil, fmt.Errorf("failed to format column %q type %T: %w", col.Name, val, err) } diff --git a/source/snapshot/fetch_worker.go b/source/snapshot/fetch_worker.go index 9aea011..b1dbb5d 100644 --- a/source/snapshot/fetch_worker.go +++ b/source/snapshot/fetch_worker.go @@ -390,9 +390,12 @@ func (f *FetchWorker) buildRecordData(fields []pgconn.FieldDescription, values [ payload = make(opencdc.StructuredData) ) + tableInfo := f.getTableInfo() for i, fd := range fields { + isNotNull := tableInfo.Columns[fd.Name].IsNotNull + if fd.Name == f.conf.Key { - k, err := types.Format(fd.DataTypeOID, values[i]) + k, err := types.Format(fd.DataTypeOID, values[i], isNotNull) if err != nil { return key, payload, fmt.Errorf("failed to format key %q: %w", f.conf.Key, err) } @@ -400,7 +403,7 @@ func (f *FetchWorker) buildRecordData(fields []pgconn.FieldDescription, values [ key[f.conf.Key] = k } - v, err := types.Format(fd.DataTypeOID, values[i]) + v, err := types.Format(fd.DataTypeOID, values[i], isNotNull) if err != nil { return key, payload, fmt.Errorf("failed to format payload field %q: %w", fd.Name, err) } @@ -511,7 +514,7 @@ func (f *FetchWorker) extractSchemas(ctx context.Context, fields []pgconn.FieldD sdk.Logger(ctx).Debug(). Msgf("extracting schema for key %v in %v", f.conf.Key, f.conf.Table) - avroKeySch, err := schema.Avro.Extract(f.conf.Table+"_key", f.tableInfoFetcher.GetTable(f.conf.Table), fields, f.conf.Key) + avroKeySch, err := schema.Avro.Extract(f.conf.Table+"_key", f.getTableInfo(), fields, f.conf.Key) if err != nil { return fmt.Errorf("failed to extract key schema for table %v: %w", f.conf.Table, err) } @@ -529,3 +532,7 @@ func (f *FetchWorker) extractSchemas(ctx context.Context, fields []pgconn.FieldD return nil } + +func (f *FetchWorker) getTableInfo() *common.TableInfo { + return f.tableInfoFetcher.GetTable(f.conf.Table) +} diff --git a/source/types/types.go b/source/types/types.go index 5f826fa..47aa031 100644 --- a/source/types/types.go +++ b/source/types/types.go @@ -15,15 +15,44 @@ package types import ( + "reflect" + "github.com/jackc/pgx/v5/pgtype" ) var ( Numeric = NumericFormatter{} - UUID = UUIDFormatter{} + + UUID = UUIDFormatter{} ) -func Format(oid uint32, v any) (any, error) { +func Format(oid uint32, v any, isNotNull bool) (any, error) { + val, err := format(oid, v) + if err != nil { + return nil, err + } + + if val == nil { + return nil, nil + } + + if reflect.TypeOf(val).Kind() != reflect.Ptr && !isNotNull { + return &val, nil + } + + return val, nil +} + +func isPrimitive(v any) bool { + switch v.(type) { + case string, int, int8, int16, int32, int64, uint, uint8, uint16, uint32, uint64, float32, float64, bool: + return true + default: + return false + } +} + +func format(oid uint32, v any) (any, error) { if oid == pgtype.UUIDOID { return UUID.Format(v) } diff --git a/source_integration_test.go b/source_integration_test.go index a783126..c40cb5d 100644 --- a/source_integration_test.go +++ b/source_integration_test.go @@ -64,7 +64,8 @@ func TestSource_Read(t *testing.T) { is.NoErr(err) assertRecordOK(is, tableName, gotRecord) - insertRow(ctx, t, tableName, 2) + insertRowNotNullColumnsOnly(ctx, t, tableName, 3) + insertRowAllColumns(ctx, t, tableName, 4) gotRecord, err = s.Read(ctx) is.NoErr(err) @@ -137,12 +138,13 @@ func prepareSourceIntegrationTestTable(ctx context.Context, t *testing.T) string is.NoErr(err) }) - insertRow(ctx, t, table, 1) + insertRowNotNullColumnsOnly(ctx, t, table, 1) + insertRowAllColumns(ctx, t, table, 2) return table } -func insertRow(ctx context.Context, t *testing.T, table string, rowNumber int) { +func insertRowNotNullColumnsOnly(ctx context.Context, t *testing.T, table string, rowNumber int) { is := is.New(t) conn := test.ConnectSimple(ctx, t, test.RepmgrConnString) @@ -190,3 +192,52 @@ func insertRow(ctx context.Context, t *testing.T, table string, rowNumber int) { _, err := conn.Exec(ctx, query) is.NoErr(err) } + +func insertRowAllColumns(ctx context.Context, t *testing.T, table string, rowNumber int) { + is := is.New(t) + conn := test.ConnectSimple(ctx, t, test.RepmgrConnString) + + query := fmt.Sprintf( + `INSERT INTO %s ( + col_bytea, col_bytea_not_null, + col_varchar, col_varchar_not_null, + col_date, col_date_not_null, + col_float4, col_float4_not_null, + col_float8, col_float8_not_null, + col_int2, col_int2_not_null, + col_int4, col_int4_not_null, + col_int8, col_int8_not_null, + -- col_numeric, col_numeric_not_null, + col_text, col_text_not_null, + col_timestamp, col_timestamp_not_null, + col_timestamptz, col_timestamptz_not_null, + col_uuid, col_uuid_not_null + ) VALUES ( + '%s'::bytea, '%s'::bytea, + 'foo-%v', 'foo-%v', + now(), now(), + %f, %f, + %f, %f, + %d, %d, + %d, %d, + %d, %d, + -- %f, %f, + 'bar-%v', 'bar-%v', + now(), now(), + now(), now(), + gen_random_uuid(), gen_random_uuid() + )`, + table, + fmt.Sprintf("col_bytea_-%v", rowNumber), fmt.Sprintf("col_bytea_-%v", rowNumber), + rowNumber, rowNumber, + float32(rowNumber)/10, float32(rowNumber)/10, + float64(rowNumber)/10, float64(rowNumber)/10, + rowNumber%32768, rowNumber%32768, + rowNumber, rowNumber, + rowNumber, rowNumber, + float64(100+rowNumber)/10, float64(100+rowNumber)/10, + rowNumber, rowNumber, + ) + _, err := conn.Exec(ctx, query) + is.NoErr(err) +} From 321f149cfc103763330bc85f57adf1522fa18fbc Mon Sep 17 00:00:00 2001 From: Haris Osmanagic Date: Thu, 29 May 2025 16:05:22 +0200 Subject: [PATCH 11/60] Merge changes for big.Rat --- .github/dependabot.yml | 8 + .../workflows/dependabot-auto-merge-go.yml | 2 +- .github/workflows/lint.yml | 5 +- .github/workflows/release.yml | 5 +- .golangci.yml | 135 ++-- Makefile | 7 +- README.md | 310 ++++++-- connector.go | 11 +- connector.yaml | 297 ++++++++ destination.go | 112 ++- destination/config.go | 26 +- destination/paramgen.go | 39 - destination_integration_test.go | 195 +++-- go.mod | 137 ++-- go.sum | 306 ++++---- internal/db_info.go | 109 +++ tools.go => internal/utils.go | 17 +- internal/utils_test.go | 71 ++ source.go | 76 +- source/config.go | 40 +- source/config_test.go | 3 +- source/iterator.go | 6 +- source/logrepl/cdc.go | 63 +- source/logrepl/cdc_test.go | 316 +++++--- source/logrepl/combined.go | 30 +- source/logrepl/combined_test.go | 323 +++++--- source/logrepl/internal/publication.go | 8 +- source/logrepl/internal/publication_test.go | 2 +- source/logrepl/internal/relationset_test.go | 13 +- source/paramgen.go | 92 --- source/snapshot/fetch_worker.go | 17 +- .../snapshot/fetch_worker_integration_test.go | 70 +- source/snapshot/iterator.go | 42 +- source/snapshot/iterator_test.go | 69 +- source/types/numeric.go | 37 +- source/types/types_test.go | 7 +- source_integration_test.go | 50 +- spec.go | 33 - test/docker-compose.yml | 2 +- test/helper.go | 51 +- tools/go.mod | 254 +++++++ tools/go.sum | 716 ++++++++++++++++++ 42 files changed, 3054 insertions(+), 1058 deletions(-) create mode 100644 connector.yaml delete mode 100644 destination/paramgen.go create mode 100644 internal/db_info.go rename tools.go => internal/utils.go (68%) create mode 100644 internal/utils_test.go delete mode 100644 source/paramgen.go delete mode 100644 spec.go create mode 100644 tools/go.mod create mode 100644 tools/go.sum diff --git a/.github/dependabot.yml b/.github/dependabot.yml index ed7a994..2589b4b 100644 --- a/.github/dependabot.yml +++ b/.github/dependabot.yml @@ -17,3 +17,11 @@ updates: interval: "daily" commit-message: prefix: "go.mod:" + + # Maintain dependencies for Go tools + - package-ecosystem: "gomod" + directory: "/tools" + schedule: + interval: "weekly" + commit-message: + prefix: "Go tools:" diff --git a/.github/workflows/dependabot-auto-merge-go.yml b/.github/workflows/dependabot-auto-merge-go.yml index 58f2b9b..4bc356f 100644 --- a/.github/workflows/dependabot-auto-merge-go.yml +++ b/.github/workflows/dependabot-auto-merge-go.yml @@ -18,7 +18,7 @@ jobs: steps: - name: Dependabot metadata id: metadata - uses: dependabot/fetch-metadata@v2.2.0 + uses: dependabot/fetch-metadata@v2.4.0 with: github-token: "${{ secrets.GITHUB_TOKEN }}" diff --git a/.github/workflows/lint.yml b/.github/workflows/lint.yml index 93e908d..378d809 100644 --- a/.github/workflows/lint.yml +++ b/.github/workflows/lint.yml @@ -20,11 +20,10 @@ jobs: - name: Golangci-lint version id: golangci-lint-version run: | - GOLANGCI_LINT_VERSION=$( go list -m -f '{{.Version}}' github.com/golangci/golangci-lint ) + GOLANGCI_LINT_VERSION=$( go list -modfile=tools/go.mod -m -f '{{.Version}}' github.com/golangci/golangci-lint/v2 ) echo "v=$GOLANGCI_LINT_VERSION" >> "$GITHUB_OUTPUT" - name: golangci-lint - uses: golangci/golangci-lint-action@v6 + uses: golangci/golangci-lint-action@v8 with: version: ${{ steps.golangci-lint-version.outputs.v }} - args: --timeout=2m diff --git a/.github/workflows/release.yml b/.github/workflows/release.yml index a097c42..7ecf142 100644 --- a/.github/workflows/release.yml +++ b/.github/workflows/release.yml @@ -3,7 +3,7 @@ name: release on: push: tags: - - v* + - '*' permissions: contents: write @@ -18,6 +18,9 @@ jobs: with: fetch-depth: 0 + - name: Check Connector Tag + uses: conduitio/automation/actions/check_connector_tag@main + - name: Set up Go uses: actions/setup-go@v5 with: diff --git a/.golangci.yml b/.golangci.yml index b97f0eb..31a8aa2 100644 --- a/.golangci.yml +++ b/.golangci.yml @@ -1,110 +1,83 @@ -linters-settings: - gofmt: - simplify: false - govet: - check-shadowing: false - nolintlint: - allow-unused: false # report any unused nolint directives - require-explanation: true # require an explanation for nolint directives - require-specific: true # require nolint directives to mention the specific linter being suppressed - gocyclo: - min-complexity: 20 - goconst: - ignore-tests: true - goheader: - template-path: '.golangci.goheader.template' - values: - regexp: - copyright-year: 20[2-9]\d - forbidigo: - forbid: - - p: ^pgxpool\.New.*$ - msg: "Use github.com/conduitio/conduit-connector-postgres/source/cpool.New instead." -issues: - exclude-rules: - - path: test/helper\.go - linters: - - gosec - - path: source/cpool/cpool\.go - linters: - - forbidigo - +version: "2" linters: - # please, do not use `enable-all`: it's deprecated and will be removed soon. - # inverted configuration with `enable-all` and `disable` is not scalable during updates of golangci-lint - disable-all: true + default: none enable: - bodyclose - copyloopvar - # - depguard - dogsled - durationcheck - errcheck - errname - # - errorlint - # - exhaustive - # - exhaustivestruct - forbidigo - # - forcetypeassert - # - funlen - # - gochecknoinits - goconst - gocritic - gocyclo - # - cyclop # not interested in package complexities at the moment - # - godot - - gofmt - # - gofumpt - goheader - - goimports - - revive - # - gomnd - gomoddirectives - gomodguard - goprintffuncname - gosec - - gosimple - govet - # - ifshort - ineffassign - # - importas - # - lll - # - misspell - makezero - # - nakedret - # - nilerr - # - nilnil - # - nlreturn - noctx - nolintlint - # - paralleltest - predeclared + - revive - rowserrcheck - - staticcheck - - stylecheck - sqlclosecheck - # - tagliatelle - # - tenv - # - thelper - # - tparallel - - typecheck + - staticcheck - unconvert - # - unparam - unused - wastedassign - whitespace - # - wrapcheck - # - wsl - - # don't enable: - # - asciicheck - # - dupl - # - gochecknoglobals - # - gocognit - # - godox - # - goerr113 - # - maligned - # - nestif - # - prealloc - # - testpackage - # - wsl + settings: + forbidigo: + forbid: + - pattern: ^pgxpool\.New.*$ + msg: Use github.com/conduitio/conduit-connector-postgres/source/cpool.New instead. + gocyclo: + min-complexity: 20 + goheader: + values: + regexp: + copyright-year: 20[2-9]\d + template-path: .golangci.goheader.template + nolintlint: + require-explanation: true + require-specific: true + allow-unused: false + exclusions: + generated: lax + presets: + - comments + - common-false-positives + - legacy + - std-error-handling + rules: + - linters: + - gosec + path: test/helper\.go + - linters: + - forbidigo + path: source/cpool/cpool\.go + - linters: + - goconst + path: (.+)_test\.go + paths: + - third_party$ + - builtin$ + - examples$ +formatters: + enable: + - gofmt + - goimports + settings: + gofmt: + simplify: false + exclusions: + generated: lax + paths: + - third_party$ + - builtin$ + - examples$ diff --git a/Makefile b/Makefile index 5752635..99fab77 100644 --- a/Makefile +++ b/Makefile @@ -14,11 +14,12 @@ test: .PHONY: lint lint: - golangci-lint run -v + golangci-lint run .PHONY: generate generate: go generate ./... + conn-sdk-cli readmegen -w .PHONY: fmt fmt: @@ -27,6 +28,6 @@ fmt: .PHONY: install-tools install-tools: - @echo Installing tools from tools.go - @go list -e -f '{{ join .Imports "\n" }}' tools.go | xargs -tI % go install % + @echo Installing tools from tools/go.mod + @go list -modfile=tools/go.mod tool | xargs -I % go list -modfile=tools/go.mod -f "%@{{.Module.Version}}" % | xargs -tI % go install % @go mod tidy diff --git a/README.md b/README.md index f5cb71d..fbe2513 100644 --- a/README.md +++ b/README.md @@ -1,99 +1,281 @@ # Conduit Connector PostgreSQL -The PostgreSQL connector is a [Conduit](https://github.com/ConduitIO/conduit) plugin. It provides both, a source -and a destination PostgresSQL connectors. +The PostgreSQL connector is a [Conduit](https://github.com/ConduitIO/conduit) +plugin. It provides both, a source and a destination PostgresSQL connector. -# Source + +## Source -The Postgres Source Connector connects to a database with the provided `url` and starts creating records for each change -detected in the provided tables. +The Postgres Source Connector connects to a database with the provided `url` and +starts creating records for each change detected in the provided tables. -Upon starting, the source takes a snapshot of the provided tables in the database, then switches into CDC mode. In CDC mode, -the plugin reads from a buffer of CDC events. +Upon starting, the source takes a snapshot of the provided tables in the database, +then switches into CDC mode. In CDC mode, the plugin reads from a buffer of CDC events. -## Snapshot Capture +### Snapshot -When the connector first starts, snapshot mode is enabled. The connector acquires a read-only lock on the tables, and -then reads all rows of the tables into Conduit. Once all rows in that initial snapshot are read the connector releases -its lock and switches into CDC mode. +When the connector first starts, snapshot mode is enabled. The connector acquires +a read-only lock on the tables, and then reads all rows of the tables into Conduit. +Once all rows in that initial snapshot are read the connector releases its lock and +switches into CDC mode. -This behavior is enabled by default, but can be turned off by adding `"snapshotMode":"never"` to the Source -configuration. +This behavior is enabled by default, but can be turned off by adding +`"snapshotMode": "never"` to the Source configuration. -## Change Data Capture +### Change Data Capture -This connector implements CDC features for PostgreSQL by creating a logical replication slot and a publication that -listens to changes in the configured tables. Every detected change is converted into a record and returned in the call to -`Read`. If there is no record available at the moment `Read` is called, it blocks until a record is available or the -connector receives a stop signal. +This connector implements Change Data Capture (CDC) features for PostgreSQL by +creating a logical replication slot and a publication that listens to changes in the +configured tables. Every detected change is converted into a record. If there are no +records available, the connector blocks until a record is available or the connector +receives a stop signal. -### Logical Replication Configuration +#### Logical Replication Configuration -When the connector switches to CDC mode, it attempts to run the initial setup commands to create its logical replication -slot and publication. It will connect to an existing slot if one with the configured name exists. +When the connector switches to CDC mode, it attempts to run the initial setup commands +to create its logical replication slot and publication. It will connect to an existing +slot if one with the configured name exists. -The Postgres user specified in the connection URL must have sufficient privileges to run all of these setup commands, or -it will fail. +The Postgres user specified in the connection URL must have sufficient privileges to +run all of these setup commands, or it will fail. -Example configuration for CDC features: +Example pipeline configuration that's using logical replication: -```json -{ - "url": "url", - "tables": "records", - "cdcMode": "logrepl", - "logrepl.publicationName": "meroxademo", - "logrepl.slotName": "meroxademo" -} +```yaml +version: 2.2 +pipelines: + - id: pg-to-log + status: running + connectors: + - id: pg + type: source + plugin: builtin:postgres + settings: + url: "postgres://exampleuser:examplepass@localhost:5433/exampledb?sslmode=disable" + tables: "users" + cdcMode: "logrepl" + logrepl.publicationName: "examplepub" + logrepl.slotName: "exampleslot" + - id: log + type: destination + plugin: builtin:log + settings: + level: info ``` -:warning: When the connector or pipeline is deleted, the connector will automatically attempt to delete the replication slot and publication. This is the default behaviour and can be disabled by setting `logrepl.autoCleanup` to `false`. +:warning: When the connector or pipeline is deleted, the connector will automatically +attempt to delete the replication slot and publication. This is the default behaviour +and can be disabled by setting `logrepl.autoCleanup` to `false`. -## Key Handling +### Key Handling -The connector will automatically look up the primary key column for the specified tables. If that can't be determined, -the connector will return an error. +The connector will automatically look up the primary key column for the specified tables +and use them as the key value. If that can't be determined, the connector will return +an error. -## Configuration Options +## Destination -| name | description | required | default | -| ------------------------- | ------------------------------------------------------------------------------------------------------------------------------------------ | -------- | ------------- | -| `url` | Connection string for the Postgres database. | true | | -| `tables` | List of table names to read from, separated by comma. Example: `"employees,offices,payments"`. Using `*` will read from all public tables. | true | | -| `snapshotMode` | Whether or not the plugin will take a snapshot of the entire table before starting cdc mode (allowed values: `initial` or `never`). | false | `initial` | -| `cdcMode` | Determines the CDC mode (allowed values: `auto`, `logrepl`). | false | `auto` | -| `logrepl.publicationName` | Name of the publication to listen for WAL events. | false | `conduitpub` | -| `logrepl.slotName` | Name of the slot opened for replication events. | false | `conduitslot` | -| `logrepl.autoCleanup` | Whether or not to cleanup the replication slot and pub when connector is deleted | false | `true` | -| ~~`table`~~ | List of table names to read from, separated by comma. **Deprecated: use `tables` instead.** | false | | +The Postgres Destination takes a Conduit record and stores it using a SQL statement. +The Destination is designed to handle different payloads and keys. Because of this, +each record is individually parsed and upserted. -# Destination +### Handling record operations -The Postgres Destination takes a `record.Record` and parses it into a valid SQL query. The Destination is designed to -handle different payloads and keys. Because of this, each record is individually parsed and upserted. +Based on the `Operation` field in the record, the destination will either insert, +update or delete the record in the target table. Snapshot records are always inserted. -## Upsert Behavior +If the target table already contains a record with the same key as a record being +inserted, the record will be updated (upserted). This can overwrite and thus potentially +lose data, so keys should be assigned correctly from the Source. -If the target table already contains a record with the same key, the Destination will upsert with its current received -values. Because Keys must be unique, this can overwrite and thus potentially lose data, so keys should be assigned -correctly from the Source. +If the target table does not contain a record with the same key as a record being +deleted, the record will be ignored. If there is no key, the record will be simply appended. + -## Configuration Options +## Source Configuration Parameters -| name | description | required | default | -| ------- | ------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | -------- | -------------------------------------------- | -| `url` | Connection string for the Postgres database. | true | | -| `table` | Table name. It can contain a Go template that will be executed for each record to determine the table. By default, the table is the value of the `opencdc.collection` metadata field. | false | `{{ index .Metadata "opencdc.collection" }}` | -| `key` | Key represents the column name for the key used to identify and update existing rows. | false | | + +```yaml +version: 2.2 +pipelines: + - id: example + status: running + connectors: + - id: example + plugin: "postgres" + settings: + # Tables is a List of table names to read from, separated by a comma, + # e.g.:"table1,table2". Use "*" if you'd like to listen to all tables. + # Type: string + # Required: yes + tables: "" + # URL is the connection string for the Postgres database. + # Type: string + # Required: yes + url: "" + # CDCMode determines how the connector should listen to changes. + # Type: string + # Required: no + cdcMode: "auto" + # LogreplAutoCleanup determines if the replication slot and + # publication should be removed when the connector is deleted. + # Type: bool + # Required: no + logrepl.autoCleanup: "true" + # LogreplPublicationName determines the publication name in case the + # connector uses logical replication to listen to changes (see + # CDCMode). + # Type: string + # Required: no + logrepl.publicationName: "conduitpub" + # LogreplSlotName determines the replication slot name in case the + # connector uses logical replication to listen to changes (see + # CDCMode). Can only contain lower-case letters, numbers, and the + # underscore character. + # Type: string + # Required: no + logrepl.slotName: "conduitslot" + # WithAvroSchema determines whether the connector should attach an + # avro schema on each record. + # Type: bool + # Required: no + logrepl.withAvroSchema: "true" + # Snapshot fetcher size determines the number of rows to retrieve at a + # time. + # Type: int + # Required: no + snapshot.fetchSize: "50000" + # SnapshotMode is whether the plugin will take a snapshot of the + # entire table before starting cdc mode. + # Type: string + # Required: no + snapshotMode: "initial" + # Maximum delay before an incomplete batch is read from the source. + # Type: duration + # Required: no + sdk.batch.delay: "0" + # Maximum size of batch before it gets read from the source. + # Type: int + # Required: no + sdk.batch.size: "0" + # Specifies whether to use a schema context name. If set to false, no + # schema context name will be used, and schemas will be saved with the + # subject name specified in the connector (not safe because of name + # conflicts). + # Type: bool + # Required: no + sdk.schema.context.enabled: "true" + # Schema context name to be used. Used as a prefix for all schema + # subject names. If empty, defaults to the connector ID. + # Type: string + # Required: no + sdk.schema.context.name: "" + # Whether to extract and encode the record key with a schema. + # Type: bool + # Required: no + sdk.schema.extract.key.enabled: "false" + # The subject of the key schema. If the record metadata contains the + # field "opencdc.collection" it is prepended to the subject name and + # separated with a dot. + # Type: string + # Required: no + sdk.schema.extract.key.subject: "key" + # Whether to extract and encode the record payload with a schema. + # Type: bool + # Required: no + sdk.schema.extract.payload.enabled: "false" + # The subject of the payload schema. If the record metadata contains + # the field "opencdc.collection" it is prepended to the subject name + # and separated with a dot. + # Type: string + # Required: no + sdk.schema.extract.payload.subject: "payload" + # The type of the payload schema. + # Type: string + # Required: no + sdk.schema.extract.type: "avro" +``` + + +## Destination Configuration Parameters + + +```yaml +version: 2.2 +pipelines: + - id: example + status: running + connectors: + - id: example + plugin: "postgres" + settings: + # URL is the connection string for the Postgres database. + # Type: string + # Required: yes + url: "" + # Key represents the column name for the key used to identify and + # update existing rows. + # Type: string + # Required: no + key: "" + # Table is used as the target table into which records are inserted. + # Type: string + # Required: no + table: "{{ index .Metadata "opencdc.collection" }}" + # Maximum delay before an incomplete batch is written to the + # destination. + # Type: duration + # Required: no + sdk.batch.delay: "0" + # Maximum size of batch before it gets written to the destination. + # Type: int + # Required: no + sdk.batch.size: "0" + # Allow bursts of at most X records (0 or less means that bursts are + # not limited). Only takes effect if a rate limit per second is set. + # Note that if `sdk.batch.size` is bigger than `sdk.rate.burst`, the + # effective batch size will be equal to `sdk.rate.burst`. + # Type: int + # Required: no + sdk.rate.burst: "0" + # Maximum number of records written per second (0 means no rate + # limit). + # Type: float + # Required: no + sdk.rate.perSecond: "0" + # The format of the output record. See the Conduit documentation for a + # full list of supported formats + # (https://conduit.io/docs/using/connectors/configuration-parameters/output-format). + # Type: string + # Required: no + sdk.record.format: "opencdc/json" + # Options to configure the chosen output record format. Options are + # normally key=value pairs separated with comma (e.g. + # opt1=val2,opt2=val2), except for the `template` record format, where + # options are a Go template. + # Type: string + # Required: no + sdk.record.format.options: "" + # Whether to extract and decode the record key with a schema. + # Type: bool + # Required: no + sdk.schema.extract.key.enabled: "true" + # Whether to extract and decode the record payload with a schema. + # Type: bool + # Required: no + sdk.schema.extract.payload.enabled: "true" +``` + -# Testing +## Testing -Run `make test` to run all the unit and integration tests, which require Docker to be installed and running. The command -will handle starting and stopping docker containers for you. +Run `make test` to run all the unit and integration tests, which require Docker +to be installed and running. The command will handle starting and stopping +docker containers for you. -# References +## References - https://github.com/bitnami/bitnami-docker-postgresql-repmgr - https://github.com/Masterminds/squirrel diff --git a/connector.go b/connector.go index cf218f9..7cf6e5c 100644 --- a/connector.go +++ b/connector.go @@ -12,14 +12,23 @@ // See the License for the specific language governing permissions and // limitations under the License. +//go:generate conn-sdk-cli specgen + package postgres import ( + _ "embed" + sdk "github.com/conduitio/conduit-connector-sdk" ) +//go:embed connector.yaml +var specs string + +var version = "(devel)" + var Connector = sdk.Connector{ - NewSpecification: Specification, + NewSpecification: sdk.YAMLSpecification(specs, version), NewSource: NewSource, NewDestination: NewDestination, } diff --git a/connector.yaml b/connector.yaml new file mode 100644 index 0000000..b248635 --- /dev/null +++ b/connector.yaml @@ -0,0 +1,297 @@ +version: "1.0" +specification: + name: postgres + summary: Conduit connector for PostgreSQL + description: | + ## Source + + The Postgres Source Connector connects to a database with the provided `url` and + starts creating records for each change detected in the provided tables. + + Upon starting, the source takes a snapshot of the provided tables in the database, + then switches into CDC mode. In CDC mode, the plugin reads from a buffer of CDC events. + + ### Snapshot + + When the connector first starts, snapshot mode is enabled. The connector acquires + a read-only lock on the tables, and then reads all rows of the tables into Conduit. + Once all rows in that initial snapshot are read the connector releases its lock and + switches into CDC mode. + + This behavior is enabled by default, but can be turned off by adding + `"snapshotMode": "never"` to the Source configuration. + + ### Change Data Capture + + This connector implements Change Data Capture (CDC) features for PostgreSQL by + creating a logical replication slot and a publication that listens to changes in the + configured tables. Every detected change is converted into a record. If there are no + records available, the connector blocks until a record is available or the connector + receives a stop signal. + + #### Logical Replication Configuration + + When the connector switches to CDC mode, it attempts to run the initial setup commands + to create its logical replication slot and publication. It will connect to an existing + slot if one with the configured name exists. + + The Postgres user specified in the connection URL must have sufficient privileges to + run all of these setup commands, or it will fail. + + Example pipeline configuration that's using logical replication: + + ```yaml + version: 2.2 + pipelines: + - id: pg-to-log + status: running + connectors: + - id: pg + type: source + plugin: builtin:postgres + settings: + url: "postgres://exampleuser:examplepass@localhost:5433/exampledb?sslmode=disable" + tables: "users" + cdcMode: "logrepl" + logrepl.publicationName: "examplepub" + logrepl.slotName: "exampleslot" + - id: log + type: destination + plugin: builtin:log + settings: + level: info + ``` + + :warning: When the connector or pipeline is deleted, the connector will automatically + attempt to delete the replication slot and publication. This is the default behaviour + and can be disabled by setting `logrepl.autoCleanup` to `false`. + + ### Key Handling + + The connector will automatically look up the primary key column for the specified tables + and use them as the key value. If that can't be determined, the connector will return + an error. + + ## Destination + + The Postgres Destination takes a Conduit record and stores it using a SQL statement. + The Destination is designed to handle different payloads and keys. Because of this, + each record is individually parsed and upserted. + + ### Handling record operations + + Based on the `Operation` field in the record, the destination will either insert, + update or delete the record in the target table. Snapshot records are always inserted. + + If the target table already contains a record with the same key as a record being + inserted, the record will be updated (upserted). This can overwrite and thus potentially + lose data, so keys should be assigned correctly from the Source. + + If the target table does not contain a record with the same key as a record being + deleted, the record will be ignored. + + If there is no key, the record will be simply appended. + version: v0.12.2 + author: Meroxa, Inc. + source: + parameters: + - name: tables + description: |- + Tables is a List of table names to read from, separated by a comma, e.g.:"table1,table2". + Use "*" if you'd like to listen to all tables. + type: string + default: "" + validations: + - type: required + value: "" + - name: url + description: URL is the connection string for the Postgres database. + type: string + default: "" + validations: + - type: required + value: "" + - name: cdcMode + description: CDCMode determines how the connector should listen to changes. + type: string + default: auto + validations: + - type: inclusion + value: auto,logrepl + - name: logrepl.autoCleanup + description: |- + LogreplAutoCleanup determines if the replication slot and publication should be + removed when the connector is deleted. + type: bool + default: "true" + validations: [] + - name: logrepl.publicationName + description: |- + LogreplPublicationName determines the publication name in case the + connector uses logical replication to listen to changes (see CDCMode). + type: string + default: conduitpub + validations: [] + - name: logrepl.slotName + description: |- + LogreplSlotName determines the replication slot name in case the + connector uses logical replication to listen to changes (see CDCMode). + Can only contain lower-case letters, numbers, and the underscore character. + type: string + default: conduitslot + validations: + - type: regex + value: ^[a-z0-9_]+$ + - name: logrepl.withAvroSchema + description: |- + WithAvroSchema determines whether the connector should attach an avro schema on each + record. + type: bool + default: "true" + validations: [] + - name: snapshot.fetchSize + description: Snapshot fetcher size determines the number of rows to retrieve at a time. + type: int + default: "50000" + validations: [] + - name: snapshotMode + description: SnapshotMode is whether the plugin will take a snapshot of the entire table before starting cdc mode. + type: string + default: initial + validations: + - type: inclusion + value: initial,never + - name: sdk.batch.delay + description: Maximum delay before an incomplete batch is read from the source. + type: duration + default: "0" + validations: [] + - name: sdk.batch.size + description: Maximum size of batch before it gets read from the source. + type: int + default: "0" + validations: + - type: greater-than + value: "-1" + - name: sdk.schema.context.enabled + description: |- + Specifies whether to use a schema context name. If set to false, no schema context name will + be used, and schemas will be saved with the subject name specified in the connector + (not safe because of name conflicts). + type: bool + default: "true" + validations: [] + - name: sdk.schema.context.name + description: |- + Schema context name to be used. Used as a prefix for all schema subject names. + If empty, defaults to the connector ID. + type: string + default: "" + validations: [] + - name: sdk.schema.extract.key.enabled + description: Whether to extract and encode the record key with a schema. + type: bool + default: "false" + validations: [] + - name: sdk.schema.extract.key.subject + description: |- + The subject of the key schema. If the record metadata contains the field + "opencdc.collection" it is prepended to the subject name and separated + with a dot. + type: string + default: key + validations: [] + - name: sdk.schema.extract.payload.enabled + description: Whether to extract and encode the record payload with a schema. + type: bool + default: "false" + validations: [] + - name: sdk.schema.extract.payload.subject + description: |- + The subject of the payload schema. If the record metadata contains the + field "opencdc.collection" it is prepended to the subject name and + separated with a dot. + type: string + default: payload + validations: [] + - name: sdk.schema.extract.type + description: The type of the payload schema. + type: string + default: avro + validations: + - type: inclusion + value: avro + destination: + parameters: + - name: url + description: URL is the connection string for the Postgres database. + type: string + default: "" + validations: + - type: required + value: "" + - name: key + description: Key represents the column name for the key used to identify and update existing rows. + type: string + default: "" + validations: [] + - name: table + description: Table is used as the target table into which records are inserted. + type: string + default: '{{ index .Metadata "opencdc.collection" }}' + validations: [] + - name: sdk.batch.delay + description: Maximum delay before an incomplete batch is written to the destination. + type: duration + default: "0" + validations: [] + - name: sdk.batch.size + description: Maximum size of batch before it gets written to the destination. + type: int + default: "0" + validations: + - type: greater-than + value: "-1" + - name: sdk.rate.burst + description: |- + Allow bursts of at most X records (0 or less means that bursts are not + limited). Only takes effect if a rate limit per second is set. Note that + if `sdk.batch.size` is bigger than `sdk.rate.burst`, the effective batch + size will be equal to `sdk.rate.burst`. + type: int + default: "0" + validations: + - type: greater-than + value: "-1" + - name: sdk.rate.perSecond + description: Maximum number of records written per second (0 means no rate limit). + type: float + default: "0" + validations: + - type: greater-than + value: "-1" + - name: sdk.record.format + description: |- + The format of the output record. See the Conduit documentation for a full + list of supported formats (https://conduit.io/docs/using/connectors/configuration-parameters/output-format). + type: string + default: opencdc/json + validations: [] + - name: sdk.record.format.options + description: |- + Options to configure the chosen output record format. Options are normally + key=value pairs separated with comma (e.g. opt1=val2,opt2=val2), except + for the `template` record format, where options are a Go template. + type: string + default: "" + validations: [] + - name: sdk.schema.extract.key.enabled + description: Whether to extract and decode the record key with a schema. + type: bool + default: "true" + validations: [] + - name: sdk.schema.extract.payload.enabled + description: Whether to extract and decode the record payload with a schema. + type: bool + default: "true" + validations: [] diff --git a/destination.go b/destination.go index d2e0a2c..32b940b 100644 --- a/destination.go +++ b/destination.go @@ -18,14 +18,16 @@ import ( "context" "encoding/json" "fmt" + "math/big" "strings" sq "github.com/Masterminds/squirrel" - "github.com/conduitio/conduit-commons/config" "github.com/conduitio/conduit-commons/opencdc" "github.com/conduitio/conduit-connector-postgres/destination" + "github.com/conduitio/conduit-connector-postgres/internal" sdk "github.com/conduitio/conduit-connector-sdk" "github.com/jackc/pgx/v5" + "github.com/shopspring/decimal" ) type Destination struct { @@ -35,45 +37,34 @@ type Destination struct { getTableName destination.TableFn conn *pgx.Conn + dbInfo *internal.DbInfo stmtBuilder sq.StatementBuilderType } +func (d *Destination) Config() sdk.DestinationConfig { + return &d.config +} + func NewDestination() sdk.Destination { d := &Destination{ stmtBuilder: sq.StatementBuilder.PlaceholderFormat(sq.Dollar), } - return sdk.DestinationWithMiddleware(d, sdk.DefaultDestinationMiddleware()...) + return sdk.DestinationWithMiddleware(d) } -func (d *Destination) Parameters() config.Parameters { - return d.config.Parameters() -} - -func (d *Destination) Configure(ctx context.Context, cfg config.Config) error { - err := sdk.Util.ParseConfig(ctx, cfg, &d.config, NewDestination().Parameters()) - if err != nil { - return err - } - // try parsing the url - _, err = pgx.ParseConfig(d.config.URL) +func (d *Destination) Open(ctx context.Context) error { + conn, err := pgx.Connect(ctx, d.config.URL) if err != nil { - return fmt.Errorf("invalid url: %w", err) + return fmt.Errorf("failed to open connection: %w", err) } + d.conn = conn d.getTableName, err = d.config.TableFunction() if err != nil { - return fmt.Errorf("invalid table name or table function: %w", err) + return fmt.Errorf("invalid table name or table name function: %w", err) } - return nil -} - -func (d *Destination) Open(ctx context.Context) error { - conn, err := pgx.Connect(ctx, d.config.URL) - if err != nil { - return fmt.Errorf("failed to open connection: %w", err) - } - d.conn = conn + d.dbInfo = internal.NewDbInfo(conn) return nil } @@ -198,8 +189,8 @@ func (d *Destination) remove(ctx context.Context, r opencdc.Record, b *pgx.Batch Any("key", map[string]interface{}{keyColumnName: key[keyColumnName]}). Msg("deleting record") query, args, err := d.stmtBuilder. - Delete(tableName). - Where(sq.Eq{keyColumnName: key[keyColumnName]}). + Delete(internal.WrapSQLIdent(tableName)). + Where(sq.Eq{internal.WrapSQLIdent(keyColumnName): key[keyColumnName]}). ToSql() if err != nil { return fmt.Errorf("error formatting delete query: %w", err) @@ -228,12 +219,16 @@ func (d *Destination) insert(ctx context.Context, r opencdc.Record, b *pgx.Batch return err } - colArgs, valArgs := d.formatColumnsAndValues(key, payload) + colArgs, valArgs, err := d.formatColumnsAndValues(tableName, key, payload) + if err != nil { + return fmt.Errorf("error formatting columns and values: %w", err) + } + sdk.Logger(ctx).Trace(). Str("table_name", tableName). Msg("inserting record") query, args, err := d.stmtBuilder. - Insert(tableName). + Insert(internal.WrapSQLIdent(tableName)). Columns(colArgs...). Values(valArgs...). ToSql() @@ -291,12 +286,13 @@ func (d *Destination) formatUpsertQuery( keyColumnName string, tableName string, ) (string, []interface{}, error) { - upsertQuery := fmt.Sprintf("ON CONFLICT (%s) DO UPDATE SET", keyColumnName) + upsertQuery := fmt.Sprintf("ON CONFLICT (%s) DO UPDATE SET", internal.WrapSQLIdent(keyColumnName)) for column := range payload { // tuples form a comma separated list, so they need a comma at the end. // `EXCLUDED` references the new record's values. This will overwrite // every column's value except for the key column. - tuple := fmt.Sprintf("%s=EXCLUDED.%s,", column, column) + wrappedCol := internal.WrapSQLIdent(column) + tuple := fmt.Sprintf("%s=EXCLUDED.%s,", wrappedCol, wrappedCol) // TODO: Consider removing this space. upsertQuery += " " // add the tuple to the query string @@ -306,13 +302,16 @@ func (d *Destination) formatUpsertQuery( // remove the last comma from the list of tuples upsertQuery = strings.TrimSuffix(upsertQuery, ",") - // we have to manually append a semi colon to the upsert sql; + // we have to manually append a semicolon to the upsert sql; upsertQuery += ";" - colArgs, valArgs := d.formatColumnsAndValues(key, payload) + colArgs, valArgs, err := d.formatColumnsAndValues(tableName, key, payload) + if err != nil { + return "", nil, fmt.Errorf("error formatting columns and values: %w", err) + } return d.stmtBuilder. - Insert(tableName). + Insert(internal.WrapSQLIdent(tableName)). Columns(colArgs...). Values(valArgs...). SuffixExpr(sq.Expr(upsertQuery)). @@ -321,24 +320,32 @@ func (d *Destination) formatUpsertQuery( // formatColumnsAndValues turns the key and payload into a slice of ordered // columns and values for upserting into Postgres. -func (d *Destination) formatColumnsAndValues(key, payload opencdc.StructuredData) ([]string, []interface{}) { +func (d *Destination) formatColumnsAndValues(table string, key, payload opencdc.StructuredData) ([]string, []interface{}, error) { var colArgs []string var valArgs []interface{} // range over both the key and payload values in order to format the // query for args and values in proper order for key, val := range key { - colArgs = append(colArgs, key) - valArgs = append(valArgs, val) + colArgs = append(colArgs, internal.WrapSQLIdent(key)) + formatted, err := d.formatValue(table, key, val) + if err != nil { + return nil, nil, fmt.Errorf("error formatting value: %w", err) + } + valArgs = append(valArgs, formatted) delete(payload, key) // NB: Delete Key from payload arguments } - for field, value := range payload { - colArgs = append(colArgs, field) - valArgs = append(valArgs, value) + for field, val := range payload { + colArgs = append(colArgs, internal.WrapSQLIdent(field)) + formatted, err := d.formatValue(table, field, val) + if err != nil { + return nil, nil, fmt.Errorf("error formatting value: %w", err) + } + valArgs = append(valArgs, formatted) } - return colArgs, valArgs + return colArgs, valArgs, nil } // getKeyColumnName will return the name of the first item in the key or the @@ -358,3 +365,28 @@ func (d *Destination) getKeyColumnName(key opencdc.StructuredData, defaultKeyNam func (d *Destination) hasKey(e opencdc.Record) bool { return e.Key != nil && len(e.Key.Bytes()) > 0 } + +func (d *Destination) formatValue(table string, column string, val interface{}) (interface{}, error) { + switch v := val.(type) { + case *big.Rat: + return d.formatBigRat(table, column, v) + case big.Rat: + return d.formatBigRat(table, column, &v) + default: + return val, nil + } +} + +func (d *Destination) formatBigRat(table string, column string, v *big.Rat) (string, error) { + scale, err := d.dbInfo.GetNumericColumnScale(table, column) + if err != nil { + return "", fmt.Errorf("failed getting scale of numeric column: %w", err) + } + + if v == nil { + return "", nil + } + + //nolint:gosec // no risk of overflow, because the scale in Pg is always <= 16383 + return decimal.NewFromBigRat(v, int32(scale)).String(), nil +} diff --git a/destination/config.go b/destination/config.go index 2d72152..0569ab2 100644 --- a/destination/config.go +++ b/destination/config.go @@ -12,23 +12,26 @@ // See the License for the specific language governing permissions and // limitations under the License. -//go:generate paramgen Config - package destination import ( "bytes" + "context" "fmt" "strings" "text/template" "github.com/Masterminds/sprig/v3" "github.com/conduitio/conduit-commons/opencdc" + sdk "github.com/conduitio/conduit-connector-sdk" + "github.com/jackc/pgx/v5" ) type TableFn func(opencdc.Record) (string, error) type Config struct { + sdk.DefaultDestinationMiddleware + // URL is the connection string for the Postgres database. URL string `json:"url" validate:"required"` // Table is used as the target table into which records are inserted. @@ -37,10 +40,27 @@ type Config struct { Key string `json:"key"` } +func (c *Config) Validate(ctx context.Context) error { + if _, err := pgx.ParseConfig(c.URL); err != nil { + return fmt.Errorf("invalid url: %w", err) + } + + if _, err := c.TableFunction(); err != nil { + return fmt.Errorf("invalid table name or table function: %w", err) + } + + err := c.DefaultDestinationMiddleware.Validate(ctx) + if err != nil { + return fmt.Errorf("middleware validation failed: %w", err) + } + + return nil +} + // TableFunction returns a function that determines the table for each record individually. // The function might be returning a static table name. // If the table is neither static nor a template, an error is returned. -func (c Config) TableFunction() (f TableFn, err error) { +func (c *Config) TableFunction() (f TableFn, err error) { // Not a template, i.e. it's a static table name if !strings.HasPrefix(c.Table, "{{") && !strings.HasSuffix(c.Table, "}}") { return func(_ opencdc.Record) (string, error) { diff --git a/destination/paramgen.go b/destination/paramgen.go deleted file mode 100644 index 320c223..0000000 --- a/destination/paramgen.go +++ /dev/null @@ -1,39 +0,0 @@ -// Code generated by paramgen. DO NOT EDIT. -// Source: github.com/ConduitIO/conduit-commons/tree/main/paramgen - -package destination - -import ( - "github.com/conduitio/conduit-commons/config" -) - -const ( - ConfigKey = "key" - ConfigTable = "table" - ConfigUrl = "url" -) - -func (Config) Parameters() map[string]config.Parameter { - return map[string]config.Parameter{ - ConfigKey: { - Default: "", - Description: "Key represents the column name for the key used to identify and update existing rows.", - Type: config.ParameterTypeString, - Validations: []config.Validation{}, - }, - ConfigTable: { - Default: "{{ index .Metadata \"opencdc.collection\" }}", - Description: "Table is used as the target table into which records are inserted.", - Type: config.ParameterTypeString, - Validations: []config.Validation{}, - }, - ConfigUrl: { - Default: "", - Description: "URL is the connection string for the Postgres database.", - Type: config.ParameterTypeString, - Validations: []config.Validation{ - config.ValidationRequired{}, - }, - }, - } -} diff --git a/destination_integration_test.go b/destination_integration_test.go index d8e6a12..f97e7fe 100644 --- a/destination_integration_test.go +++ b/destination_integration_test.go @@ -17,10 +17,14 @@ package postgres import ( "context" "fmt" + "math/big" + "strings" "testing" "github.com/conduitio/conduit-commons/opencdc" "github.com/conduitio/conduit-connector-postgres/test" + sdk "github.com/conduitio/conduit-connector-sdk" + "github.com/google/go-cmp/cmp" "github.com/jackc/pgx/v5" "github.com/matryer/is" ) @@ -29,17 +33,23 @@ func TestDestination_Write(t *testing.T) { is := is.New(t) ctx := test.Context(t) conn := test.ConnectSimple(ctx, t, test.RegularConnString) - tableName := test.SetupTestTable(ctx, t, conn) + + // tables with capital letters should be quoted + tableName := strings.ToUpper(test.RandomIdentifier(t)) + test.SetupTestTableWithName(ctx, t, conn, tableName) d := NewDestination() - err := d.Configure( + err := sdk.Util.ParseConfig( ctx, map[string]string{ "url": test.RegularConnString, "table": "{{ index .Metadata \"opencdc.collection\" }}", }, + d.Config(), + Connector.NewSpecification().DestinationParams, ) is.NoErr(err) + err = d.Open(ctx) is.NoErr(err) defer func() { @@ -60,13 +70,16 @@ func TestDestination_Write(t *testing.T) { Key: opencdc.StructuredData{"id": 5000}, Payload: opencdc.Change{ After: opencdc.StructuredData{ - "column1": "foo", - "column2": 123, - "column3": true, + "column1": "foo", + "column2": 123, + "column3": true, + "column4": nil, + "UppercaseColumn1": 222, }, }, }, - }, { + }, + { name: "create", record: opencdc.Record{ Position: opencdc.Position("foo"), @@ -75,13 +88,16 @@ func TestDestination_Write(t *testing.T) { Key: opencdc.StructuredData{"id": 5}, Payload: opencdc.Change{ After: opencdc.StructuredData{ - "column1": "foo", - "column2": 456, - "column3": false, + "column1": "foo", + "column2": 456, + "column3": false, + "column4": nil, + "UppercaseColumn1": 333, }, }, }, - }, { + }, + { name: "insert on update (upsert)", record: opencdc.Record{ Position: opencdc.Position("foo"), @@ -90,13 +106,16 @@ func TestDestination_Write(t *testing.T) { Key: opencdc.StructuredData{"id": 6}, Payload: opencdc.Change{ After: opencdc.StructuredData{ - "column1": "bar", - "column2": 567, - "column3": true, + "column1": "bar", + "column2": 567, + "column3": true, + "column4": nil, + "UppercaseColumn1": 444, }, }, }, - }, { + }, + { name: "update on conflict", record: opencdc.Record{ Position: opencdc.Position("foo"), @@ -105,13 +124,16 @@ func TestDestination_Write(t *testing.T) { Key: opencdc.StructuredData{"id": 1}, Payload: opencdc.Change{ After: opencdc.StructuredData{ - "column1": "foobar", - "column2": 567, - "column3": true, + "column1": "foobar", + "column2": 567, + "column3": true, + "column4": nil, + "UppercaseColumn1": 555, }, }, }, - }, { + }, + { name: "delete", record: opencdc.Record{ Position: opencdc.Position("foo"), @@ -120,6 +142,24 @@ func TestDestination_Write(t *testing.T) { Key: opencdc.StructuredData{"id": 4}, }, }, + { + name: "write a big.Rat", + record: opencdc.Record{ + Position: opencdc.Position("foo"), + Operation: opencdc.OperationSnapshot, + Metadata: map[string]string{opencdc.MetadataCollection: tableName}, + Key: opencdc.StructuredData{"id": 123}, + Payload: opencdc.Change{ + After: opencdc.StructuredData{ + "column1": "abcdef", + "column2": 567, + "column3": true, + "column4": big.NewRat(123, 100), + "UppercaseColumn1": 555, + }, + }, + }, + }, } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { @@ -134,7 +174,16 @@ func TestDestination_Write(t *testing.T) { switch tt.record.Operation { case opencdc.OperationCreate, opencdc.OperationSnapshot, opencdc.OperationUpdate: is.NoErr(err) - is.Equal(tt.record.Payload.After, got) + is.Equal( + "", + cmp.Diff( + tt.record.Payload.After, + got, + cmp.Comparer(func(x, y *big.Rat) bool { + return x.Cmp(y) == 0 + }), + ), + ) // -want, +got case opencdc.OperationDelete: is.Equal(err, pgx.ErrNoRows) } @@ -146,11 +195,20 @@ func TestDestination_Batch(t *testing.T) { is := is.New(t) ctx := test.Context(t) conn := test.ConnectSimple(ctx, t, test.RegularConnString) - tableName := test.SetupTestTable(ctx, t, conn) + + tableName := strings.ToUpper(test.RandomIdentifier(t)) + test.SetupTestTableWithName(ctx, t, conn, tableName) d := NewDestination() - err := d.Configure(ctx, map[string]string{"url": test.RegularConnString, "table": tableName}) + + err := sdk.Util.ParseConfig( + ctx, + map[string]string{"url": test.RegularConnString, "table": tableName}, + d.Config(), + Connector.NewSpecification().DestinationParams, + ) is.NoErr(err) + err = d.Open(ctx) is.NoErr(err) defer func() { @@ -158,40 +216,50 @@ func TestDestination_Batch(t *testing.T) { is.NoErr(err) }() - records := []opencdc.Record{{ - Position: opencdc.Position("foo1"), - Operation: opencdc.OperationCreate, - Key: opencdc.StructuredData{"id": 5}, - Payload: opencdc.Change{ - After: opencdc.StructuredData{ - "column1": "foo1", - "column2": 1, - "column3": false, + records := []opencdc.Record{ + { + Position: opencdc.Position("foo1"), + Operation: opencdc.OperationCreate, + Key: opencdc.StructuredData{"id": 5}, + Payload: opencdc.Change{ + After: opencdc.StructuredData{ + "column1": "foo1", + "column2": 1, + "column3": false, + "column4": nil, + "UppercaseColumn1": 111, + }, }, }, - }, { - Position: opencdc.Position("foo2"), - Operation: opencdc.OperationCreate, - Key: opencdc.StructuredData{"id": 6}, - Payload: opencdc.Change{ - After: opencdc.StructuredData{ - "column1": "foo2", - "column2": 2, - "column3": true, + { + Position: opencdc.Position("foo2"), + Operation: opencdc.OperationCreate, + Key: opencdc.StructuredData{"id": 6}, + Payload: opencdc.Change{ + After: opencdc.StructuredData{ + "column1": "foo2", + "column2": 2, + "column3": true, + "column4": nil, + "UppercaseColumn1": 222, + }, }, }, - }, { - Position: opencdc.Position("foo3"), - Operation: opencdc.OperationCreate, - Key: opencdc.StructuredData{"id": 7}, - Payload: opencdc.Change{ - After: opencdc.StructuredData{ - "column1": "foo3", - "column2": 3, - "column3": false, + { + Position: opencdc.Position("foo3"), + Operation: opencdc.OperationCreate, + Key: opencdc.StructuredData{"id": 7}, + Payload: opencdc.Change{ + After: opencdc.StructuredData{ + "column1": "foo3", + "column2": 3, + "column3": false, + "column4": nil, + "UppercaseColumn1": 333, + }, }, }, - }} + } i, err := d.Write(ctx, records) is.NoErr(err) @@ -207,23 +275,36 @@ func TestDestination_Batch(t *testing.T) { func queryTestTable(ctx context.Context, conn test.Querier, tableName string, id any) (opencdc.StructuredData, error) { row := conn.QueryRow( ctx, - fmt.Sprintf("SELECT column1, column2, column3 FROM %s WHERE id = $1", tableName), + fmt.Sprintf(`SELECT column1, column2, column3, column4, "UppercaseColumn1" FROM %q WHERE id = $1`, tableName), id, ) var ( - col1 string - col2 int - col3 bool + col1 string + col2 int + col3 bool + col4Str *string + uppercaseCol1 int ) - err := row.Scan(&col1, &col2, &col3) + + err := row.Scan(&col1, &col2, &col3, &col4Str, &uppercaseCol1) if err != nil { return nil, err } + // Handle the potential nil case for col4 + var col4 interface{} + if col4Str != nil { + r := new(big.Rat) + r.SetString(*col4Str) + col4 = r + } + return opencdc.StructuredData{ - "column1": col1, - "column2": col2, - "column3": col3, + "column1": col1, + "column2": col2, + "column3": col3, + "column4": col4, + "UppercaseColumn1": uppercaseCol1, }, nil } diff --git a/go.mod b/go.mod index d584c07..70b8177 100644 --- a/go.mod +++ b/go.mod @@ -1,48 +1,45 @@ module github.com/conduitio/conduit-connector-postgres -go 1.23.2 +go 1.24.2 require ( github.com/Masterminds/sprig/v3 v3.3.0 github.com/Masterminds/squirrel v1.5.4 - github.com/conduitio/conduit-commons v0.5.0 - github.com/conduitio/conduit-connector-sdk v0.12.0 - github.com/daixiang0/gci v0.13.5 - github.com/golangci/golangci-lint v1.63.4 - github.com/google/go-cmp v0.6.0 + github.com/conduitio/conduit-commons v0.5.4 + github.com/conduitio/conduit-connector-sdk v0.13.4 + github.com/google/go-cmp v0.7.0 github.com/google/uuid v1.6.0 - github.com/hamba/avro/v2 v2.27.0 + github.com/hamba/avro/v2 v2.28.0 github.com/jackc/pgerrcode v0.0.0-20240316143900-6e2875d9b438 github.com/jackc/pglogrepl v0.0.0-20240307033717-828fbfe908e9 - github.com/jackc/pgx/v5 v5.7.2 + github.com/jackc/pgx/v5 v5.7.5 github.com/matryer/is v1.4.1 - github.com/rs/zerolog v1.33.0 - golang.org/x/tools v0.29.0 + github.com/rs/zerolog v1.34.0 + github.com/shopspring/decimal v1.4.0 gopkg.in/tomb.v2 v2.0.0-20161208151619-d5d1b5820637 - mvdan.cc/gofumpt v0.7.0 ) require ( - 4d63.com/gocheckcompilerdirectives v1.2.1 // indirect - 4d63.com/gochecknoglobals v0.2.1 // indirect + 4d63.com/gocheckcompilerdirectives v1.3.0 // indirect + 4d63.com/gochecknoglobals v0.2.2 // indirect dario.cat/mergo v1.0.1 // indirect - github.com/4meepo/tagalign v1.4.1 // indirect + github.com/4meepo/tagalign v1.4.2 // indirect github.com/Abirdcfly/dupword v0.1.3 // indirect github.com/Antonboom/errname v1.0.0 // indirect github.com/Antonboom/nilnil v1.0.1 // indirect github.com/Antonboom/testifylint v1.5.2 // indirect github.com/BurntSushi/toml v1.4.1-0.20240526193622-a339e1f7089c // indirect - github.com/Crocmagnon/fatcontext v0.5.3 // indirect + github.com/Crocmagnon/fatcontext v0.7.1 // indirect github.com/Djarvur/go-err113 v0.0.0-20210108212216-aea10b59be24 // indirect - github.com/GaijinEntertainment/go-exhaustruct/v3 v3.3.0 // indirect + github.com/GaijinEntertainment/go-exhaustruct/v3 v3.3.1 // indirect github.com/Masterminds/goutils v1.1.1 // indirect - github.com/Masterminds/semver/v3 v3.3.0 // indirect - github.com/OpenPeeDeeP/depguard/v2 v2.2.0 // indirect + github.com/Masterminds/semver/v3 v3.3.1 // indirect + github.com/OpenPeeDeeP/depguard/v2 v2.2.1 // indirect github.com/alecthomas/go-check-sumtype v0.3.1 // indirect github.com/alexkohler/nakedret/v2 v2.0.5 // indirect github.com/alexkohler/prealloc v1.0.0 // indirect github.com/alingse/asasalint v0.0.11 // indirect - github.com/alingse/nilnesserr v0.1.1 // indirect + github.com/alingse/nilnesserr v0.1.2 // indirect github.com/ashanbrown/forbidigo v1.6.0 // indirect github.com/ashanbrown/makezero v1.2.0 // indirect github.com/beorn7/perks v1.0.1 // indirect @@ -53,14 +50,18 @@ require ( github.com/breml/errchkjson v0.4.0 // indirect github.com/butuzov/ireturn v0.3.1 // indirect github.com/butuzov/mirror v1.3.0 // indirect - github.com/catenacyber/perfsprint v0.7.1 // indirect + github.com/catenacyber/perfsprint v0.8.2 // indirect github.com/ccojocar/zxcvbn-go v1.0.2 // indirect github.com/cespare/xxhash/v2 v2.3.0 // indirect github.com/charithe/durationcheck v0.0.10 // indirect github.com/chavacava/garif v0.1.0 // indirect github.com/ckaznocha/intrange v0.3.0 // indirect - github.com/conduitio/conduit-connector-protocol v0.9.0 // indirect + github.com/conduitio/conduit-connector-protocol v0.9.2 // indirect + github.com/conduitio/evolviconf v0.1.0 // indirect + github.com/conduitio/evolviconf/evolviyaml v0.1.0 // indirect + github.com/conduitio/yaml/v3 v3.3.0 // indirect github.com/curioswitch/go-reassign v0.3.0 // indirect + github.com/daixiang0/gci v0.13.6 // indirect github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc // indirect github.com/denis-tingaikin/go-header v0.5.0 // indirect github.com/ettle/strcase v0.2.0 // indirect @@ -69,8 +70,8 @@ require ( github.com/firefart/nonamedreturns v1.0.5 // indirect github.com/fsnotify/fsnotify v1.7.0 // indirect github.com/fzipp/gocyclo v0.6.0 // indirect - github.com/ghostiam/protogetter v0.3.8 // indirect - github.com/go-critic/go-critic v0.11.5 // indirect + github.com/ghostiam/protogetter v0.3.9 // indirect + github.com/go-critic/go-critic v0.12.0 // indirect github.com/go-toolsmith/astcast v1.1.0 // indirect github.com/go-toolsmith/astcopy v1.1.0 // indirect github.com/go-toolsmith/astequal v1.2.0 // indirect @@ -81,24 +82,25 @@ require ( github.com/go-viper/mapstructure/v2 v2.2.1 // indirect github.com/go-xmlfmt/xmlfmt v1.1.3 // indirect github.com/gobwas/glob v0.2.3 // indirect - github.com/goccy/go-json v0.10.3 // indirect + github.com/goccy/go-json v0.10.5 // indirect github.com/gofrs/flock v0.12.1 // indirect github.com/golang/protobuf v1.5.4 // indirect - github.com/golangci/dupl v0.0.0-20180902072040-3e9179ac440a // indirect + github.com/golangci/dupl v0.0.0-20250308024227-f665c8d69b32 // indirect github.com/golangci/go-printf-func-name v0.1.0 // indirect - github.com/golangci/gofmt v0.0.0-20241223200906-057b0627d9b9 // indirect + github.com/golangci/gofmt v0.0.0-20250106114630-d62b90e6713d // indirect + github.com/golangci/golangci-lint v1.64.8 // indirect github.com/golangci/misspell v0.6.0 // indirect github.com/golangci/plugin-module-register v0.1.1 // indirect - github.com/golangci/revgrep v0.5.3 // indirect + github.com/golangci/revgrep v0.8.0 // indirect github.com/golangci/unconvert v0.0.0-20240309020433-c5143eacb3ed // indirect github.com/gordonklaus/ineffassign v0.1.0 // indirect github.com/gostaticanalysis/analysisutil v0.7.1 // indirect - github.com/gostaticanalysis/comment v1.4.2 // indirect - github.com/gostaticanalysis/forcetypeassert v0.1.0 // indirect + github.com/gostaticanalysis/comment v1.5.0 // indirect + github.com/gostaticanalysis/forcetypeassert v0.2.0 // indirect github.com/gostaticanalysis/nilerr v0.1.1 // indirect github.com/hashicorp/go-hclog v1.6.3 // indirect github.com/hashicorp/go-immutable-radix/v2 v2.1.0 // indirect - github.com/hashicorp/go-plugin v1.6.2 // indirect + github.com/hashicorp/go-plugin v1.6.3 // indirect github.com/hashicorp/go-version v1.7.0 // indirect github.com/hashicorp/golang-lru/v2 v2.0.7 // indirect github.com/hashicorp/hcl v1.0.0 // indirect @@ -116,18 +118,17 @@ require ( github.com/jpillora/backoff v1.0.0 // indirect github.com/json-iterator/go v1.1.12 // indirect github.com/julz/importas v0.2.0 // indirect - github.com/karamaru-alpha/copyloopvar v1.1.0 // indirect - github.com/kisielk/errcheck v1.8.0 // indirect - github.com/kkHAIKE/contextcheck v1.1.5 // indirect + github.com/karamaru-alpha/copyloopvar v1.2.1 // indirect + github.com/kisielk/errcheck v1.9.0 // indirect + github.com/kkHAIKE/contextcheck v1.1.6 // indirect github.com/kulti/thelper v0.6.3 // indirect github.com/kunwardeep/paralleltest v1.0.10 // indirect - github.com/kyoh86/exportloopref v0.1.11 // indirect github.com/lann/builder v0.0.0-20180802200727-47ae307949d0 // indirect github.com/lann/ps v0.0.0-20150810152359-62de8c46ede0 // indirect github.com/lasiar/canonicalheader v1.1.2 // indirect - github.com/ldez/exptostd v0.3.1 // indirect - github.com/ldez/gomoddirectives v0.6.0 // indirect - github.com/ldez/grignotin v0.7.0 // indirect + github.com/ldez/exptostd v0.4.2 // indirect + github.com/ldez/gomoddirectives v0.6.1 // indirect + github.com/ldez/grignotin v0.9.0 // indirect github.com/ldez/tagliatelle v0.7.1 // indirect github.com/ldez/usetesting v0.4.2 // indirect github.com/leonklingele/grouper v1.1.2 // indirect @@ -135,11 +136,11 @@ require ( github.com/magiconair/properties v1.8.7 // indirect github.com/maratori/testableexamples v1.0.0 // indirect github.com/maratori/testpackage v1.1.1 // indirect - github.com/matoous/godox v0.0.0-20240105082147-c5b5e0e7c0c0 // indirect - github.com/mattn/go-colorable v0.1.13 // indirect + github.com/matoous/godox v1.1.0 // indirect + github.com/mattn/go-colorable v0.1.14 // indirect github.com/mattn/go-isatty v0.0.20 // indirect github.com/mattn/go-runewidth v0.0.16 // indirect - github.com/mgechev/revive v1.5.1 // indirect + github.com/mgechev/revive v1.7.0 // indirect github.com/mitchellh/copystructure v1.2.0 // indirect github.com/mitchellh/go-homedir v1.1.0 // indirect github.com/mitchellh/mapstructure v1.5.0 // indirect @@ -151,12 +152,12 @@ require ( github.com/nakabonne/nestif v0.3.1 // indirect github.com/nishanths/exhaustive v0.12.0 // indirect github.com/nishanths/predeclared v0.2.2 // indirect - github.com/nunnatsa/ginkgolinter v0.18.4 // indirect + github.com/nunnatsa/ginkgolinter v0.19.1 // indirect github.com/oklog/run v1.1.0 // indirect github.com/olekukonko/tablewriter v0.0.5 // indirect github.com/pelletier/go-toml/v2 v2.2.3 // indirect github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2 // indirect - github.com/polyfloyd/go-errorlint v1.7.0 // indirect + github.com/polyfloyd/go-errorlint v1.7.1 // indirect github.com/prometheus/client_golang v1.20.2 // indirect github.com/prometheus/client_model v0.6.1 // indirect github.com/prometheus/common v0.55.0 // indirect @@ -168,36 +169,37 @@ require ( github.com/quasilyte/stdinfo v0.0.0-20220114132959-f7386bf02567 // indirect github.com/raeperd/recvcheck v0.2.0 // indirect github.com/rivo/uniseg v0.4.7 // indirect - github.com/rogpeppe/go-internal v1.13.1 // indirect + github.com/rogpeppe/go-internal v1.14.1 // indirect github.com/ryancurrah/gomodguard v1.3.5 // indirect github.com/ryanrolds/sqlclosecheck v0.5.1 // indirect github.com/sagikazarmark/locafero v0.6.0 // indirect github.com/sagikazarmark/slog-shim v0.1.0 // indirect + github.com/samber/lo v1.47.0 // indirect + github.com/samber/slog-common v0.18.1 // indirect + github.com/samber/slog-zerolog/v2 v2.7.3 // indirect github.com/sanposhiho/wastedassign/v2 v2.1.0 // indirect github.com/santhosh-tekuri/jsonschema/v6 v6.0.1 // indirect github.com/sashamelentyev/interfacebloat v1.1.0 // indirect github.com/sashamelentyev/usestdlibvars v1.28.0 // indirect - github.com/securego/gosec/v2 v2.21.4 // indirect - github.com/shazow/go-diff v0.0.0-20160112020656-b6b7b6733b8c // indirect - github.com/shopspring/decimal v1.4.0 // indirect + github.com/securego/gosec/v2 v2.22.2 // indirect github.com/sirupsen/logrus v1.9.3 // indirect github.com/sivchari/containedctx v1.0.3 // indirect github.com/sivchari/tenv v1.12.1 // indirect github.com/sonatard/noctx v0.1.0 // indirect github.com/sourcegraph/conc v0.3.0 // indirect github.com/sourcegraph/go-diff v0.7.0 // indirect - github.com/spf13/afero v1.11.0 // indirect + github.com/spf13/afero v1.12.0 // indirect github.com/spf13/cast v1.7.0 // indirect - github.com/spf13/cobra v1.8.1 // indirect - github.com/spf13/pflag v1.0.5 // indirect + github.com/spf13/cobra v1.9.1 // indirect + github.com/spf13/pflag v1.0.6 // indirect github.com/spf13/viper v1.19.0 // indirect github.com/ssgreg/nlreturn/v2 v2.2.1 // indirect github.com/stbenjam/no-sprintf-host-port v0.2.0 // indirect github.com/stretchr/objx v0.5.2 // indirect github.com/stretchr/testify v1.10.0 // indirect github.com/subosito/gotenv v1.6.0 // indirect - github.com/tdakkota/asciicheck v0.3.0 // indirect - github.com/tetafro/godot v1.4.20 // indirect + github.com/tdakkota/asciicheck v0.4.1 // indirect + github.com/tetafro/godot v1.5.0 // indirect github.com/timakin/bodyclose v0.0.0-20241017074812-ed6a65f985e3 // indirect github.com/timonwong/loggercheck v0.10.1 // indirect github.com/tomarrell/wrapcheck/v2 v2.10.0 // indirect @@ -206,34 +208,35 @@ require ( github.com/ultraware/funlen v0.2.0 // indirect github.com/ultraware/whitespace v0.2.0 // indirect github.com/uudashr/gocognit v1.2.0 // indirect - github.com/uudashr/iface v1.3.0 // indirect + github.com/uudashr/iface v1.3.1 // indirect github.com/xen0n/gosmopolitan v1.2.2 // indirect github.com/yagipy/maintidx v1.0.0 // indirect github.com/yeya24/promlinter v0.3.0 // indirect github.com/ykadowak/zerologlint v0.1.5 // indirect gitlab.com/bosi/decorder v0.4.2 // indirect go-simpler.org/musttag v0.13.0 // indirect - go-simpler.org/sloglint v0.7.2 // indirect + go-simpler.org/sloglint v0.9.0 // indirect go.uber.org/automaxprocs v1.6.0 // indirect go.uber.org/goleak v1.3.0 // indirect - go.uber.org/mock v0.5.0 // indirect go.uber.org/multierr v1.11.0 // indirect go.uber.org/zap v1.27.0 // indirect - golang.org/x/crypto v0.32.0 // indirect - golang.org/x/exp v0.0.0-20241009180824-f66d83c29e7c // indirect - golang.org/x/exp/typeparams v0.0.0-20241108190413-2d47ceb2692f // indirect - golang.org/x/mod v0.22.0 // indirect - golang.org/x/net v0.34.0 // indirect - golang.org/x/sync v0.10.0 // indirect - golang.org/x/sys v0.29.0 // indirect - golang.org/x/text v0.21.0 // indirect - golang.org/x/time v0.8.0 // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20240930140551-af27646dc61f // indirect - google.golang.org/grpc v1.68.0 // indirect - google.golang.org/protobuf v1.35.1 // indirect + golang.org/x/crypto v0.38.0 // indirect + golang.org/x/exp v0.0.0-20250506013437-ce4c2cf36ca6 // indirect + golang.org/x/exp/typeparams v0.0.0-20250210185358-939b2ce775ac // indirect + golang.org/x/mod v0.24.0 // indirect + golang.org/x/net v0.40.0 // indirect + golang.org/x/sync v0.14.0 // indirect + golang.org/x/sys v0.33.0 // indirect + golang.org/x/text v0.25.0 // indirect + golang.org/x/time v0.11.0 // indirect + golang.org/x/tools v0.33.0 // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20250219182151-9fdb1cabc7b2 // indirect + google.golang.org/grpc v1.72.1 // indirect + google.golang.org/protobuf v1.36.6 // indirect gopkg.in/ini.v1 v1.67.0 // indirect gopkg.in/yaml.v2 v2.4.0 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect - honnef.co/go/tools v0.5.1 // indirect + honnef.co/go/tools v0.6.1 // indirect + mvdan.cc/gofumpt v0.7.0 // indirect mvdan.cc/unparam v0.0.0-20240528143540-8a5130ca722f // indirect ) diff --git a/go.sum b/go.sum index cb01b7a..bdb09a5 100644 --- a/go.sum +++ b/go.sum @@ -1,11 +1,11 @@ -4d63.com/gocheckcompilerdirectives v1.2.1 h1:AHcMYuw56NPjq/2y615IGg2kYkBdTvOaojYCBcRE7MA= -4d63.com/gocheckcompilerdirectives v1.2.1/go.mod h1:yjDJSxmDTtIHHCqX0ufRYZDL6vQtMG7tJdKVeWwsqvs= -4d63.com/gochecknoglobals v0.2.1 h1:1eiorGsgHOFOuoOiJDy2psSrQbRdIHrlge0IJIkUgDc= -4d63.com/gochecknoglobals v0.2.1/go.mod h1:KRE8wtJB3CXCsb1xy421JfTHIIbmT3U5ruxw2Qu8fSU= +4d63.com/gocheckcompilerdirectives v1.3.0 h1:Ew5y5CtcAAQeTVKUVFrE7EwHMrTO6BggtEj8BZSjZ3A= +4d63.com/gocheckcompilerdirectives v1.3.0/go.mod h1:ofsJ4zx2QAuIP/NO/NAh1ig6R1Fb18/GI7RVMwz7kAY= +4d63.com/gochecknoglobals v0.2.2 h1:H1vdnwnMaZdQW/N+NrkT1SZMTBmcwHe9Vq8lJcYYTtU= +4d63.com/gochecknoglobals v0.2.2/go.mod h1:lLxwTQjL5eIesRbvnzIP3jZtG140FnTdz+AlMa+ogt0= dario.cat/mergo v1.0.1 h1:Ra4+bf83h2ztPIQYNP99R6m+Y7KfnARDfID+a+vLl4s= dario.cat/mergo v1.0.1/go.mod h1:uNxQE+84aUszobStD9th8a29P2fMDhsBdgRYvZOxGmk= -github.com/4meepo/tagalign v1.4.1 h1:GYTu2FaPGOGb/xJalcqHeD4il5BiCywyEYZOA55P6J4= -github.com/4meepo/tagalign v1.4.1/go.mod h1:2H9Yu6sZ67hmuraFgfZkNcg5Py9Ch/Om9l2K/2W1qS4= +github.com/4meepo/tagalign v1.4.2 h1:0hcLHPGMjDyM1gHG58cS73aQF8J4TdVR96TZViorO9E= +github.com/4meepo/tagalign v1.4.2/go.mod h1:+p4aMyFM+ra7nb41CnFG6aSDXqRxU/w1VQqScKqDARI= github.com/Abirdcfly/dupword v0.1.3 h1:9Pa1NuAsZvpFPi9Pqkd93I7LIYRURj+A//dFd5tgBeE= github.com/Abirdcfly/dupword v0.1.3/go.mod h1:8VbB2t7e10KRNdwTVoxdBaxla6avbhGzb8sCTygUMhw= github.com/Antonboom/errname v1.0.0 h1:oJOOWR07vS1kRusl6YRSlat7HFnb3mSfMl6sDMRoTBA= @@ -16,22 +16,22 @@ github.com/Antonboom/testifylint v1.5.2 h1:4s3Xhuv5AvdIgbd8wOOEeo0uZG7PbDKQyKY5l github.com/Antonboom/testifylint v1.5.2/go.mod h1:vxy8VJ0bc6NavlYqjZfmp6EfqXMtBgQ4+mhCojwC1P8= 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/Crocmagnon/fatcontext v0.5.3 h1:zCh/wjc9oyeF+Gmp+V60wetm8ph2tlsxocgg/J0hOps= -github.com/Crocmagnon/fatcontext v0.5.3/go.mod h1:XoCQYY1J+XTfyv74qLXvNw4xFunr3L1wkopIIKG7wGM= +github.com/Crocmagnon/fatcontext v0.7.1 h1:SC/VIbRRZQeQWj/TcQBS6JmrXcfA+BU4OGSVUt54PjM= +github.com/Crocmagnon/fatcontext v0.7.1/go.mod h1:1wMvv3NXEBJucFGfwOJBxSVWcoIO6emV215SMkW9MFU= github.com/Djarvur/go-err113 v0.0.0-20210108212216-aea10b59be24 h1:sHglBQTwgx+rWPdisA5ynNEsoARbiCBOyGcJM4/OzsM= github.com/Djarvur/go-err113 v0.0.0-20210108212216-aea10b59be24/go.mod h1:4UJr5HIiMZrwgkSPdsjy2uOQExX/WEILpIrO9UPGuXs= -github.com/GaijinEntertainment/go-exhaustruct/v3 v3.3.0 h1:/fTUt5vmbkAcMBt4YQiuC23cV0kEsN1MVMNqeOW43cU= -github.com/GaijinEntertainment/go-exhaustruct/v3 v3.3.0/go.mod h1:ONJg5sxcbsdQQ4pOW8TGdTidT2TMAUy/2Xhr8mrYaao= +github.com/GaijinEntertainment/go-exhaustruct/v3 v3.3.1 h1:Sz1JIXEcSfhz7fUi7xHnhpIE0thVASYjvosApmHuD2k= +github.com/GaijinEntertainment/go-exhaustruct/v3 v3.3.1/go.mod h1:n/LSCXNuIYqVfBlVXyHfMQkZDdp1/mmxfSjADd3z1Zg= github.com/Masterminds/goutils v1.1.1 h1:5nUrii3FMTL5diU80unEVvNevw1nH4+ZV4DSLVJLSYI= github.com/Masterminds/goutils v1.1.1/go.mod h1:8cTjp+g8YejhMuvIA5y2vz3BpJxksy863GQaJW2MFNU= -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= +github.com/Masterminds/semver/v3 v3.3.1 h1:QtNSWtVZ3nBfk8mAOu/B6v7FMJ+NHTIgUPi7rj+4nv4= +github.com/Masterminds/semver/v3 v3.3.1/go.mod h1:4V+yj/TJE1HU9XfppCwVMZq3I84lprf4nC11bSS5beM= github.com/Masterminds/sprig/v3 v3.3.0 h1:mQh0Yrg1XPo6vjYXgtf5OtijNAKJRNcTdOOGZe3tPhs= github.com/Masterminds/sprig/v3 v3.3.0/go.mod h1:Zy1iXRYNqNLUolqCpL4uhk6SHUMAOSCzdgBfDb35Lz0= github.com/Masterminds/squirrel v1.5.4 h1:uUcX/aBc8O7Fg9kaISIUsHXdKuqehiXAMQTYX8afzqM= github.com/Masterminds/squirrel v1.5.4/go.mod h1:NNaOrjSoIDfDA40n7sr2tPNZRfjzjA400rg+riTZj10= -github.com/OpenPeeDeeP/depguard/v2 v2.2.0 h1:vDfG60vDtIuf0MEOhmLlLLSzqaRM8EMcgJPdp74zmpA= -github.com/OpenPeeDeeP/depguard/v2 v2.2.0/go.mod h1:CIzddKRvLBC4Au5aYP/i3nyaWQ+ClszLIuVocRiCYFQ= +github.com/OpenPeeDeeP/depguard/v2 v2.2.1 h1:vckeWVESWp6Qog7UZSARNqfu/cZqvki8zsuj3piCMx4= +github.com/OpenPeeDeeP/depguard/v2 v2.2.1/go.mod h1:q4DKzC4UcVaAvcfd41CZh0PWpGgzrVxUYBlgKNGquUo= github.com/alecthomas/assert/v2 v2.11.0 h1:2Q9r3ki8+JYXvGsDyBXwH3LcJ+WK5D0gc5E8vS6K3D0= github.com/alecthomas/assert/v2 v2.11.0/go.mod h1:Bze95FyfUr7x34QZrjL+XP+0qgp/zg8yS+TtBj1WA3k= github.com/alecthomas/go-check-sumtype v0.3.1 h1:u9aUvbGINJxLVXiFvHUlPEaD7VDULsrxJb4Aq31NLkU= @@ -44,8 +44,8 @@ github.com/alexkohler/prealloc v1.0.0 h1:Hbq0/3fJPQhNkN0dR95AVrr6R7tou91y0uHG5pO github.com/alexkohler/prealloc v1.0.0/go.mod h1:VetnK3dIgFBBKmg0YnD9F9x6Icjd+9cvfHR56wJVlKE= github.com/alingse/asasalint v0.0.11 h1:SFwnQXJ49Kx/1GghOFz1XGqHYKp21Kq1nHad/0WQRnw= github.com/alingse/asasalint v0.0.11/go.mod h1:nCaoMhw7a9kSJObvQyVzNTPBDbNpdocqrSP7t/cW5+I= -github.com/alingse/nilnesserr v0.1.1 h1:7cYuJewpy9jFNMEA72Q1+3Nm3zKHzg+Q28D5f2bBFUA= -github.com/alingse/nilnesserr v0.1.1/go.mod h1:1xJPrXonEtX7wyTq8Dytns5P2hNzoWymVUIaKm4HNFg= +github.com/alingse/nilnesserr v0.1.2 h1:Yf8Iwm3z2hUUrP4muWfW83DF4nE3r1xZ26fGWUKCZlo= +github.com/alingse/nilnesserr v0.1.2/go.mod h1:1xJPrXonEtX7wyTq8Dytns5P2hNzoWymVUIaKm4HNFg= github.com/ashanbrown/forbidigo v1.6.0 h1:D3aewfM37Yb3pxHujIPSpTf6oQk9sc9WZi8gerOIVIY= github.com/ashanbrown/forbidigo v1.6.0/go.mod h1:Y8j9jy9ZYAEHXdu723cUlraTqbzjKF1MUyfOKL+AjcU= github.com/ashanbrown/makezero v1.2.0 h1:/2Lp1bypdmK9wDIq7uWBlDF1iMUpIIS4A+pF6C9IEUU= @@ -68,8 +68,8 @@ github.com/butuzov/ireturn v0.3.1 h1:mFgbEI6m+9W8oP/oDdfA34dLisRFCj2G6o/yiI1yZrY github.com/butuzov/ireturn v0.3.1/go.mod h1:ZfRp+E7eJLC0NQmk1Nrm1LOrn/gQlOykv+cVPdiXH5M= github.com/butuzov/mirror v1.3.0 h1:HdWCXzmwlQHdVhwvsfBb2Au0r3HyINry3bDWLYXiKoc= github.com/butuzov/mirror v1.3.0/go.mod h1:AEij0Z8YMALaq4yQj9CPPVYOyJQyiexpQEQgihajRfI= -github.com/catenacyber/perfsprint v0.7.1 h1:PGW5G/Kxn+YrN04cRAZKC+ZuvlVwolYMrIyyTJ/rMmc= -github.com/catenacyber/perfsprint v0.7.1/go.mod h1:/wclWYompEyjUD2FuIIDVKNkqz7IgBIWXIH3V0Zol50= +github.com/catenacyber/perfsprint v0.8.2 h1:+o9zVmCSVa7M4MvabsWvESEhpsMkhfE7k0sHNGL95yw= +github.com/catenacyber/perfsprint v0.8.2/go.mod h1:q//VWC2fWbcdSLEY1R3l8n0zQCDPdE4IjZwyY1HMunM= github.com/ccojocar/zxcvbn-go v1.0.2 h1:na/czXU8RrhXO4EZme6eQJLR4PzcGsahsBOAwU6I3Vg= github.com/ccojocar/zxcvbn-go v1.0.2/go.mod h1:g1qkXtUSvHP8lhHp5GrSmTz6uWALGRMQdw6Qnz/hi60= github.com/cespare/xxhash/v2 v2.3.0 h1:UL815xU9SqsFlibzuggzjXhog7bL6oX9BbNZnL2UFvs= @@ -80,18 +80,24 @@ github.com/chavacava/garif v0.1.0 h1:2JHa3hbYf5D9dsgseMKAmc/MZ109otzgNFk5s87H9Pc github.com/chavacava/garif v0.1.0/go.mod h1:XMyYCkEL58DF0oyW4qDjjnPWONs2HBqYKI+UIPD+Gww= github.com/ckaznocha/intrange v0.3.0 h1:VqnxtK32pxgkhJgYQEeOArVidIPg+ahLP7WBOXZd5ZY= github.com/ckaznocha/intrange v0.3.0/go.mod h1:+I/o2d2A1FBHgGELbGxzIcyd3/9l9DuwjM8FsbSS3Lo= -github.com/conduitio/conduit-commons v0.5.0 h1:28UIuOIo+6WvBZ4EU54KfPhSf44I1/Y65zQ9dC0Ps1E= -github.com/conduitio/conduit-commons v0.5.0/go.mod h1:xyT6XpGvj79gdtsn3qaD2KxadhsAYS+mmBOdln08Wio= -github.com/conduitio/conduit-connector-protocol v0.9.0 h1:7MailxYxAsr376Nz8WStVYSXnlf86bjtzpA/d/66if0= -github.com/conduitio/conduit-connector-protocol v0.9.0/go.mod h1:lF7RUjr9ZMj1rtNubaryHw4mPfjj4DGYDW+wvvRwBkM= -github.com/conduitio/conduit-connector-sdk v0.12.0 h1:WD/ZQhEAJMkvkq0KIyVCGeU8ni2ASMyPpBbAWZQ+lKo= -github.com/conduitio/conduit-connector-sdk v0.12.0/go.mod h1:keZ4eZ4q+7GFEz+Q8G97wvPrrdnBoxh+Bmxl9P9pZW0= +github.com/conduitio/conduit-commons v0.5.4 h1:O1j08gsC7Bf3Y0CbfIcpWz0719Ehx3GoHqVLItsOQbU= +github.com/conduitio/conduit-commons v0.5.4/go.mod h1:GtxenYQA3eWPeH+rCdycHVO/zE9rngAmwRAlqvshL+8= +github.com/conduitio/conduit-connector-protocol v0.9.2 h1:QYZQCYrbKz6Vbtv5KWfa4d7YQSoJq3AEnV1xVWnHk8Y= +github.com/conduitio/conduit-connector-protocol v0.9.2/go.mod h1:bgQ3cs1lGUWmO/TBxtP+mZIy4WL3vPSGEK7BVC5TEUg= +github.com/conduitio/conduit-connector-sdk v0.13.4 h1:XxVdHnfggKJWwhLq6fgEMgZgK1QCcy22fBbKdFVbTTw= +github.com/conduitio/conduit-connector-sdk v0.13.4/go.mod h1:MlGPKFDx/sf+y99fAve6hER3PimhvhzGPljvHEVG0Qk= +github.com/conduitio/evolviconf v0.1.0 h1:rcG+hs6tlrYlX9qomOQJz+K+OnDhbMbioGx3ci55yo0= +github.com/conduitio/evolviconf v0.1.0/go.mod h1:RnbnSqDDYarKgG2p+krP71svG6qLms3+/TnKrPKWk+0= +github.com/conduitio/evolviconf/evolviyaml v0.1.0 h1:nMW7CROIMtHhscm/QLMpMs7uCPp6O2dS4CfU9bhugd4= +github.com/conduitio/evolviconf/evolviyaml v0.1.0/go.mod h1:22+FHPuroT5pPZpg0fuhE8ACIMCl1S+HsAFN1CM3Vho= +github.com/conduitio/yaml/v3 v3.3.0 h1:kbbaOSHcuH39gP4+rgbJGl6DSbLZcJgEaBvkEXJlCsI= +github.com/conduitio/yaml/v3 v3.3.0/go.mod h1:JNgFMOX1t8W4YJuRZOh6GggVtSMsgP9XgTw+7dIenpc= github.com/coreos/go-systemd/v22 v22.5.0/go.mod h1:Y58oyj3AT4RCenI/lSvhwexgC+NSVTIJ3seZv2GcEnc= -github.com/cpuguy83/go-md2man/v2 v2.0.4/go.mod h1:tgQtvFlXSQOSOSIRvRPT7W67SCa46tRHOmNcaadrF8o= +github.com/cpuguy83/go-md2man/v2 v2.0.6/go.mod h1:oOW0eioCTA6cOiMLiUPZOpcVxMig6NIQQ7OS05n1F4g= github.com/curioswitch/go-reassign v0.3.0 h1:dh3kpQHuADL3cobV/sSGETA8DOv457dwl+fbBAhrQPs= github.com/curioswitch/go-reassign v0.3.0/go.mod h1:nApPCCTtqLJN/s8HfItCcKV0jIPwluBOvZP+dsJGA88= -github.com/daixiang0/gci v0.13.5 h1:kThgmH1yBmZSBCh1EJVxQ7JsHpm5Oms0AMed/0LaH4c= -github.com/daixiang0/gci v0.13.5/go.mod h1:12etP2OniiIdP4q+kjUGrC/rUagga7ODbqsom5Eo5Yk= +github.com/daixiang0/gci v0.13.6 h1:RKuEOSkGpSadkGbvZ6hJ4ddItT3cVZ9Vn9Rybk6xjl8= +github.com/daixiang0/gci v0.13.6/go.mod h1:12etP2OniiIdP4q+kjUGrC/rUagga7ODbqsom5Eo5Yk= github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc h1:U9qPSI2PIWSS1VwoXQT9A3Wy9MM3WgvqSxFWenqJduM= @@ -115,12 +121,14 @@ github.com/fsnotify/fsnotify v1.7.0 h1:8JEhPFa5W2WU7YfeZzPNqzMP6Lwt7L2715Ggo0nos github.com/fsnotify/fsnotify v1.7.0/go.mod h1:40Bi/Hjc2AVfZrqy+aj+yEI+/bRxZnMJyTJwOpGvigM= github.com/fzipp/gocyclo v0.6.0 h1:lsblElZG7d3ALtGMx9fmxeTKZaLLpU8mET09yN4BBLo= github.com/fzipp/gocyclo v0.6.0/go.mod h1:rXPyn8fnlpa0R2csP/31uerbiVBugk5whMdlyaLkLoA= -github.com/ghostiam/protogetter v0.3.8 h1:LYcXbYvybUyTIxN2Mj9h6rHrDZBDwZloPoKctWrFyJY= -github.com/ghostiam/protogetter v0.3.8/go.mod h1:WZ0nw9pfzsgxuRsPOFQomgDVSWtDLJRfQJEhsGbmQMA= -github.com/go-critic/go-critic v0.11.5 h1:TkDTOn5v7EEngMxu8KbuFqFR43USaaH8XRJLz1jhVYA= -github.com/go-critic/go-critic v0.11.5/go.mod h1:wu6U7ny9PiaHaZHcvMDmdysMqvDem162Rh3zWTrqk8M= +github.com/ghostiam/protogetter v0.3.9 h1:j+zlLLWzqLay22Cz/aYwTHKQ88GE2DQ6GkWSYFOI4lQ= +github.com/ghostiam/protogetter v0.3.9/go.mod h1:WZ0nw9pfzsgxuRsPOFQomgDVSWtDLJRfQJEhsGbmQMA= +github.com/go-critic/go-critic v0.12.0 h1:iLosHZuye812wnkEz1Xu3aBwn5ocCPfc9yqmFG9pa6w= +github.com/go-critic/go-critic v0.12.0/go.mod h1:DpE0P6OVc6JzVYzmM5gq5jMU31zLr4am5mB/VfFK64w= 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-quicktest/qt v1.101.0 h1:O1K29Txy5P2OK0dGo59b7b0LR6wKfIhttaAhHUyn7eI= github.com/go-quicktest/qt v1.101.0/go.mod h1:14Bz/f7NwaXPtdYEgzsx46kqSxVwTbzVZsDC26tQJow= github.com/go-task/slim-sprig/v3 v3.0.0 h1:sUs3vkvUymDpBKi3qH1YSqBQk9+9D/8M2mN1vB6EwHI= @@ -150,27 +158,27 @@ github.com/go-xmlfmt/xmlfmt v1.1.3 h1:t8Ey3Uy7jDSEisW2K3somuMKIpzktkWptA0iFCnRUW github.com/go-xmlfmt/xmlfmt v1.1.3/go.mod h1:aUCEOzzezBEjDBbFBoSiya/gduyIiWYRP6CnSFIV8AM= github.com/gobwas/glob v0.2.3 h1:A4xDbljILXROh+kObIiy5kIaPYD8e96x1tgBhUI5J+Y= github.com/gobwas/glob v0.2.3/go.mod h1:d3Ez4x06l9bZtSvzIay5+Yzi0fmZzPgnTbPcKjJAkT8= -github.com/goccy/go-json v0.10.3 h1:KZ5WoDbxAIgm2HNbYckL0se1fHD6rz5j4ywS6ebzDqA= -github.com/goccy/go-json v0.10.3/go.mod h1:oq7eo15ShAhp70Anwd5lgX2pLfOS3QCiwU/PULtXL6M= +github.com/goccy/go-json v0.10.5 h1:Fq85nIqj+gXn/S5ahsiTlK3TmC85qgirsdTP/+DeaC4= +github.com/goccy/go-json v0.10.5/go.mod h1:oq7eo15ShAhp70Anwd5lgX2pLfOS3QCiwU/PULtXL6M= github.com/godbus/dbus/v5 v5.0.4/go.mod h1:xhWf0FNVPg57R7Z0UbKHbJfkEywrmjJnf7w5xrFpKfA= github.com/gofrs/flock v0.12.1 h1:MTLVXXHf8ekldpJk3AKicLij9MdwOWkZ+a/jHHZby9E= github.com/gofrs/flock v0.12.1/go.mod h1:9zxTsyu5xtJ9DK+1tFZyibEV7y3uwDxPPfbxeeHCoD0= github.com/golang/protobuf v1.5.4 h1:i7eJL8qZTpSEXOPTxNKhASYpMn+8e5Q6AdndVa1dWek= github.com/golang/protobuf v1.5.4/go.mod h1:lnTiLA8Wa4RWRcIUkrtSVa5nRhsEGBg48fD6rSs7xps= -github.com/golangci/dupl v0.0.0-20180902072040-3e9179ac440a h1:w8hkcTqaFpzKqonE9uMCefW1WDie15eSP/4MssdenaM= -github.com/golangci/dupl v0.0.0-20180902072040-3e9179ac440a/go.mod h1:ryS0uhF+x9jgbj/N71xsEqODy9BN81/GonCZiOzirOk= +github.com/golangci/dupl v0.0.0-20250308024227-f665c8d69b32 h1:WUvBfQL6EW/40l6OmeSBYQJNSif4O11+bmWEz+C7FYw= +github.com/golangci/dupl v0.0.0-20250308024227-f665c8d69b32/go.mod h1:NUw9Zr2Sy7+HxzdjIULge71wI6yEg1lWQr7Evcu8K0E= github.com/golangci/go-printf-func-name v0.1.0 h1:dVokQP+NMTO7jwO4bwsRwLWeudOVUPPyAKJuzv8pEJU= github.com/golangci/go-printf-func-name v0.1.0/go.mod h1:wqhWFH5mUdJQhweRnldEywnR5021wTdZSNgwYceV14s= -github.com/golangci/gofmt v0.0.0-20241223200906-057b0627d9b9 h1:t5wybL6RtO83VwoMOb7U/Peqe3gGKQlPIC66wXmnkvM= -github.com/golangci/gofmt v0.0.0-20241223200906-057b0627d9b9/go.mod h1:Ag3L7sh7E28qAp/5xnpMMTuGYqxLZoSaEHZDkZB1RgU= -github.com/golangci/golangci-lint v1.63.4 h1:bJQFQ3hSfUto597dkL7ipDzOxsGEpiWdLiZ359OWOBI= -github.com/golangci/golangci-lint v1.63.4/go.mod h1:Hx0B7Lg5/NXbaOHem8+KU+ZUIzMI6zNj/7tFwdnn10I= +github.com/golangci/gofmt v0.0.0-20250106114630-d62b90e6713d h1:viFft9sS/dxoYY0aiOTsLKO2aZQAPT4nlQCsimGcSGE= +github.com/golangci/gofmt v0.0.0-20250106114630-d62b90e6713d/go.mod h1:ivJ9QDg0XucIkmwhzCDsqcnxxlDStoTl89jDMIoNxKY= +github.com/golangci/golangci-lint v1.64.8 h1:y5TdeVidMtBGG32zgSC7ZXTFNHrsJkDnpO4ItB3Am+I= +github.com/golangci/golangci-lint v1.64.8/go.mod h1:5cEsUQBSr6zi8XI8OjmcY2Xmliqc4iYL7YoPrL+zLJ4= github.com/golangci/misspell v0.6.0 h1:JCle2HUTNWirNlDIAUO44hUsKhOFqGPoC4LZxlaSXDs= github.com/golangci/misspell v0.6.0/go.mod h1:keMNyY6R9isGaSAu+4Q8NMBwMPkh15Gtc8UCVoDtAWo= github.com/golangci/plugin-module-register v0.1.1 h1:TCmesur25LnyJkpsVrupv1Cdzo+2f7zX0H6Jkw1Ol6c= github.com/golangci/plugin-module-register v0.1.1/go.mod h1:TTpqoB6KkwOJMV8u7+NyXMrkwwESJLOkfl9TxR1DGFc= -github.com/golangci/revgrep v0.5.3 h1:3tL7c1XBMtWHHqVpS5ChmiAAoe4PF/d5+ULzV9sLAzs= -github.com/golangci/revgrep v0.5.3/go.mod h1:U4R/s9dlXZsg8uJmaR1GrloUr14D7qDl8gi2iPXJH8k= +github.com/golangci/revgrep v0.8.0 h1:EZBctwbVd0aMeRnNUsFogoyayvKHyxlV3CdUA46FX2s= +github.com/golangci/revgrep v0.8.0/go.mod h1:U4R/s9dlXZsg8uJmaR1GrloUr14D7qDl8gi2iPXJH8k= github.com/golangci/unconvert v0.0.0-20240309020433-c5143eacb3ed h1:IURFTjxeTfNFP0hTEi1YKjB/ub8zkpaOqFFMApi2EAs= github.com/golangci/unconvert v0.0.0-20240309020433-c5143eacb3ed/go.mod h1:XLXN8bNw4CGRPaqgl3bv/lhz7bsGPh4/xSaMTbo2vkQ= github.com/google/go-cmp v0.5.1/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= @@ -178,11 +186,11 @@ github.com/google/go-cmp v0.5.2/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/ github.com/google/go-cmp v0.5.4/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.6/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.8/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= -github.com/google/go-cmp v0.6.0 h1:ofyhxvXcZhMsU5ulbFiLKl/XBFqE1GSq7atu8tAmTRI= -github.com/google/go-cmp v0.6.0/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= +github.com/google/go-cmp v0.7.0 h1:wk8382ETsv4JYUZwIsn6YpYiWiBsYLSJiTsyBybVuN8= +github.com/google/go-cmp v0.7.0/go.mod h1:pXiqmnSA92OHEEa9HXL2W4E7lf9JzCmGVUdgjX3N/iU= github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= -github.com/google/pprof v0.0.0-20241017200806-017d972448fc h1:NGyrhhFhwvRAZg02jnYVg3GBQy0qGBKmFQJwaPmpmxs= -github.com/google/pprof v0.0.0-20241017200806-017d972448fc/go.mod h1:vavhavw2zAxS5dIdcRluK6cSGGPlZynqzFM8NdvU144= +github.com/google/pprof v0.0.0-20250501235452-c0086092b71a h1:rDA3FfmxwXR+BVKKdz55WwMJ1pD2hJQNW31d+l3mPk4= +github.com/google/pprof v0.0.0-20250501235452-c0086092b71a/go.mod h1:5hDyRhoBCxViHszMt12TnOpEI4VVi+U8Gm9iphldiMA= github.com/google/uuid v1.6.0 h1:NIvaJDMOsjHA8n1jAhLSgzrAzy1Hgr+hNrb57e+94F0= github.com/google/uuid v1.6.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/gordonklaus/ineffassign v0.1.0 h1:y2Gd/9I7MdY1oEIt+n+rowjBNDcLQq3RsH5hwJd0f9s= @@ -190,23 +198,24 @@ github.com/gordonklaus/ineffassign v0.1.0/go.mod h1:Qcp2HIAYhR7mNUVSIxZww3Guk4it github.com/gostaticanalysis/analysisutil v0.7.1 h1:ZMCjoue3DtDWQ5WyU16YbjbQEQ3VuzwxALrpYd+HeKk= github.com/gostaticanalysis/analysisutil v0.7.1/go.mod h1:v21E3hY37WKMGSnbsw2S/ojApNWb6C1//mXO48CXbVc= github.com/gostaticanalysis/comment v1.4.1/go.mod h1:ih6ZxzTHLdadaiSnF5WY3dxUoXfXAlTaRzuaNDlSado= -github.com/gostaticanalysis/comment v1.4.2 h1:hlnx5+S2fY9Zo9ePo4AhgYsYHbM2+eAv8m/s1JiCd6Q= github.com/gostaticanalysis/comment v1.4.2/go.mod h1:KLUTGDv6HOCotCH8h2erHKmpci2ZoR8VPu34YA2uzdM= -github.com/gostaticanalysis/forcetypeassert v0.1.0 h1:6eUflI3DiGusXGK6X7cCcIgVCpZ2CiZ1Q7jl6ZxNV70= -github.com/gostaticanalysis/forcetypeassert v0.1.0/go.mod h1:qZEedyP/sY1lTGV1uJ3VhWZ2mqag3IkWsDHVbplHXak= +github.com/gostaticanalysis/comment v1.5.0 h1:X82FLl+TswsUMpMh17srGRuKaaXprTaytmEpgnKIDu8= +github.com/gostaticanalysis/comment v1.5.0/go.mod h1:V6eb3gpCv9GNVqb6amXzEUX3jXLVK/AdA+IrAMSqvEc= +github.com/gostaticanalysis/forcetypeassert v0.2.0 h1:uSnWrrUEYDr86OCxWa4/Tp2jeYDlogZiZHzGkWFefTk= +github.com/gostaticanalysis/forcetypeassert v0.2.0/go.mod h1:M5iPavzE9pPqWyeiVXSFghQjljW1+l/Uke3PXHS6ILY= github.com/gostaticanalysis/nilerr v0.1.1 h1:ThE+hJP0fEp4zWLkWHWcRyI2Od0p7DlgYG3Uqrmrcpk= github.com/gostaticanalysis/nilerr v0.1.1/go.mod h1:wZYb6YI5YAxxq0i1+VJbY0s2YONW0HU0GPE3+5PWN4A= github.com/gostaticanalysis/testutil v0.3.1-0.20210208050101-bfb5c8eec0e4/go.mod h1:D+FIZ+7OahH3ePw/izIEeH5I06eKs1IKI4Xr64/Am3M= github.com/gostaticanalysis/testutil v0.5.0 h1:Dq4wT1DdTwTGCQQv3rl3IvD5Ld0E6HiY+3Zh0sUGqw8= github.com/gostaticanalysis/testutil v0.5.0/go.mod h1:OLQSbuM6zw2EvCcXTz1lVq5unyoNft372msDY0nY5Hs= -github.com/hamba/avro/v2 v2.27.0 h1:IAM4lQ0VzUIKBuo4qlAiLKfqALSrFC+zi1iseTtbBKU= -github.com/hamba/avro/v2 v2.27.0/go.mod h1:jN209lopfllfrz7IGoZErlDz+AyUJ3vrBePQFZwYf5I= +github.com/hamba/avro/v2 v2.28.0 h1:E8J5D27biyAulWKNiEBhV85QPc9xRMCUCGJewS0KYCE= +github.com/hamba/avro/v2 v2.28.0/go.mod h1:9TVrlt1cG1kkTUtm9u2eO5Qb7rZXlYzoKqPt8TSH+TA= github.com/hashicorp/go-hclog v1.6.3 h1:Qr2kF+eVWjTiYmU7Y31tYlP1h0q/X3Nl3tPGdaB11/k= github.com/hashicorp/go-hclog v1.6.3/go.mod h1:W4Qnvbt70Wk/zYJryRzDRU/4r0kIg0PVHBcfoyhpF5M= github.com/hashicorp/go-immutable-radix/v2 v2.1.0 h1:CUW5RYIcysz+D3B+l1mDeXrQ7fUvGGCwJfdASSzbrfo= github.com/hashicorp/go-immutable-radix/v2 v2.1.0/go.mod h1:hgdqLXA4f6NIjRVisM1TJ9aOJVNRqKZj+xDGF6m7PBw= -github.com/hashicorp/go-plugin v1.6.2 h1:zdGAEd0V1lCaU0u+MxWQhtSDQmahpkwOun8U8EiRVog= -github.com/hashicorp/go-plugin v1.6.2/go.mod h1:CkgLQ5CZqNmdL9U9JzM532t8ZiYQ35+pj3b1FD37R0Q= +github.com/hashicorp/go-plugin v1.6.3 h1:xgHB+ZUSYeuJi96WtxEjzi23uh7YQpznjGh0U0UUrwg= +github.com/hashicorp/go-plugin v1.6.3/go.mod h1:MRobyh+Wc/nYy1V4KAXUiYfzxoYhs7V1mlH1Z7iY2h0= github.com/hashicorp/go-uuid v1.0.3 h1:2gKiV6YVmrJ1i2CKKa9obLvRieoRGviZFL26PcT/Co8= github.com/hashicorp/go-uuid v1.0.3/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= github.com/hashicorp/go-version v1.2.1/go.mod h1:fltr4n8CU8Ke44wwGCBoEymUuxUHl09ZGVZPK5anwXA= @@ -234,14 +243,14 @@ github.com/jackc/pgpassfile v1.0.0 h1:/6Hmqy13Ss2zCq62VdNG8tM1wchn8zjSGOBJ6icpsI github.com/jackc/pgpassfile v1.0.0/go.mod h1:CEx0iS5ambNFdcRtxPj5JhEz+xB6uRky5eyVu/W2HEg= github.com/jackc/pgservicefile v0.0.0-20240606120523-5a60cdf6a761 h1:iCEnooe7UlwOQYpKFhBabPMi4aNAfoODPEFNiAnClxo= github.com/jackc/pgservicefile v0.0.0-20240606120523-5a60cdf6a761/go.mod h1:5TJZWKEWniPve33vlWYSoGYefn3gLQRzjfDlhSJ9ZKM= -github.com/jackc/pgx/v5 v5.7.2 h1:mLoDLV6sonKlvjIEsV56SkWNCnuNv531l94GaIzO+XI= -github.com/jackc/pgx/v5 v5.7.2/go.mod h1:ncY89UGWxg82EykZUwSpUKEfccBGGYq1xjrOpsbsfGQ= +github.com/jackc/pgx/v5 v5.7.5 h1:JHGfMnQY+IEtGM63d+NGMjoRpysB2JBwDr5fsngwmJs= +github.com/jackc/pgx/v5 v5.7.5/go.mod h1:aruU7o91Tc2q2cFp5h4uP3f6ztExVpyVv88Xl/8Vl8M= github.com/jackc/puddle/v2 v2.2.2 h1:PR8nw+E/1w0GLuRFSmiioY6UooMp6KJv0/61nB7icHo= github.com/jackc/puddle/v2 v2.2.2/go.mod h1:vriiEXHvEE654aYKXXjOvZM39qJ0q+azkZFrfEOc3H4= github.com/jgautheron/goconst v1.7.1 h1:VpdAG7Ca7yvvJk5n8dMwQhfEZJh95kl/Hl9S1OI5Jkk= github.com/jgautheron/goconst v1.7.1/go.mod h1:aAosetZ5zaeC/2EfMeRswtxUFBpe2Hr7HzkgX4fanO4= -github.com/jhump/protoreflect v1.16.0 h1:54fZg+49widqXYQ0b+usAFHbMkBGR4PpXrsHc8+TBDg= -github.com/jhump/protoreflect v1.16.0/go.mod h1:oYPd7nPvcBw/5wlDfm/AVmU9zH9BgqGCI469pGxfj/8= +github.com/jhump/protoreflect v1.15.1 h1:HUMERORf3I3ZdX05WaQ6MIpd/NJ434hTp5YiKgfCL6c= +github.com/jhump/protoreflect v1.15.1/go.mod h1:jD/2GMKKE6OqX8qTjhADU1e6DShO+gavG9e0Q693nKo= github.com/jingyugao/rowserrcheck v1.1.1 h1:zibz55j/MJtLsjP1OF4bSdgXxwL1b+Vn7Tjzq7gFzUs= github.com/jingyugao/rowserrcheck v1.1.1/go.mod h1:4yvlZSDb3IyDTUZJUmpZfm2Hwok+Dtp+nu2qOq+er9c= github.com/jjti/go-spancheck v0.6.4 h1:Tl7gQpYf4/TMU7AT84MN83/6PutY21Nb9fuQjFTpRRc= @@ -252,14 +261,14 @@ github.com/json-iterator/go v1.1.12 h1:PV8peI4a0ysnczrg+LtxykD8LfKY9ML6u2jnxaEnr github.com/json-iterator/go v1.1.12/go.mod h1:e30LSqwooZae/UwlEbR2852Gd8hjQvJoHmT4TnhNGBo= github.com/julz/importas v0.2.0 h1:y+MJN/UdL63QbFJHws9BVC5RpA2iq0kpjrFajTGivjQ= github.com/julz/importas v0.2.0/go.mod h1:pThlt589EnCYtMnmhmRYY/qn9lCf/frPOK+WMx3xiJY= -github.com/karamaru-alpha/copyloopvar v1.1.0 h1:x7gNyKcC2vRBO1H2Mks5u1VxQtYvFiym7fCjIP8RPos= -github.com/karamaru-alpha/copyloopvar v1.1.0/go.mod h1:u7CIfztblY0jZLOQZgH3oYsJzpC2A7S6u/lfgSXHy0k= -github.com/kisielk/errcheck v1.8.0 h1:ZX/URYa7ilESY19ik/vBmCn6zdGQLxACwjAcWbHlYlg= -github.com/kisielk/errcheck v1.8.0/go.mod h1:1kLL+jV4e+CFfueBmI1dSK2ADDyQnlrnrY/FqKluHJQ= -github.com/kkHAIKE/contextcheck v1.1.5 h1:CdnJh63tcDe53vG+RebdpdXJTc9atMgGqdx8LXxiilg= -github.com/kkHAIKE/contextcheck v1.1.5/go.mod h1:O930cpht4xb1YQpK+1+AgoM3mFsvxr7uyFptcnWTYUA= -github.com/klauspost/compress v1.17.11 h1:In6xLpyWOi1+C7tXUUWv2ot1QvBjxevKAaI6IXrJmUc= -github.com/klauspost/compress v1.17.11/go.mod h1:pMDklpSncoRMuLFrf1W9Ss9KT+0rH90U12bZKk7uwG0= +github.com/karamaru-alpha/copyloopvar v1.2.1 h1:wmZaZYIjnJ0b5UoKDjUHrikcV0zuPyyxI4SVplLd2CI= +github.com/karamaru-alpha/copyloopvar v1.2.1/go.mod h1:nFmMlFNlClC2BPvNaHMdkirmTJxVCY0lhxBtlfOypMM= +github.com/kisielk/errcheck v1.9.0 h1:9xt1zI9EBfcYBvdU1nVrzMzzUPUtPKs9bVSIM3TAb3M= +github.com/kisielk/errcheck v1.9.0/go.mod h1:kQxWMMVZgIkDq7U8xtG/n2juOjbLgZtedi0D+/VL/i8= +github.com/kkHAIKE/contextcheck v1.1.6 h1:7HIyRcnyzxL9Lz06NGhiKvenXq7Zw6Q0UQu/ttjfJCE= +github.com/kkHAIKE/contextcheck v1.1.6/go.mod h1:3dDbMRNBFaq8HFXWC1JyvDSPm43CmE6IuHam8Wr0rkg= +github.com/klauspost/compress v1.18.0 h1:c/Cqfb0r+Yi+JtIEq73FWXVkRonBlf0CRNYc8Zttxdo= +github.com/klauspost/compress v1.18.0/go.mod h1:2Pp+KzxcywXVXMr50+X0Q/Lsb43OQHYWRCY2AiWywWQ= github.com/kr/pretty v0.3.1 h1:flRD4NNwYAUpkphVc1HcthR4KEIFJ65n8Mw5qdRn3LE= github.com/kr/pretty v0.3.1/go.mod h1:hoEshYVHaxMs3cyo3Yncou5ZscifuDolrwPKZanG3xk= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= @@ -268,20 +277,18 @@ github.com/kulti/thelper v0.6.3 h1:ElhKf+AlItIu+xGnI990no4cE2+XaSu1ULymV2Yulxs= github.com/kulti/thelper v0.6.3/go.mod h1:DsqKShOvP40epevkFrvIwkCMNYxMeTNjdWL4dqWHZ6I= github.com/kunwardeep/paralleltest v1.0.10 h1:wrodoaKYzS2mdNVnc4/w31YaXFtsc21PCTdvWJ/lDDs= github.com/kunwardeep/paralleltest v1.0.10/go.mod h1:2C7s65hONVqY7Q5Efj5aLzRCNLjw2h4eMc9EcypGjcY= -github.com/kyoh86/exportloopref v0.1.11 h1:1Z0bcmTypkL3Q4k+IDHMWTcnCliEZcaPiIe0/ymEyhQ= -github.com/kyoh86/exportloopref v0.1.11/go.mod h1:qkV4UF1zGl6EkF1ox8L5t9SwyeBAZ3qLMd6up458uqA= github.com/lann/builder v0.0.0-20180802200727-47ae307949d0 h1:SOEGU9fKiNWd/HOJuq6+3iTQz8KNCLtVX6idSoTLdUw= github.com/lann/builder v0.0.0-20180802200727-47ae307949d0/go.mod h1:dXGbAdH5GtBTC4WfIxhKZfyBF/HBFgRZSWwZ9g/He9o= github.com/lann/ps v0.0.0-20150810152359-62de8c46ede0 h1:P6pPBnrTSX3DEVR4fDembhRWSsG5rVo6hYhAB/ADZrk= github.com/lann/ps v0.0.0-20150810152359-62de8c46ede0/go.mod h1:vmVJ0l/dxyfGW6FmdpVm2joNMFikkuWg0EoCKLGUMNw= github.com/lasiar/canonicalheader v1.1.2 h1:vZ5uqwvDbyJCnMhmFYimgMZnJMjwljN5VGY0VKbMXb4= github.com/lasiar/canonicalheader v1.1.2/go.mod h1:qJCeLFS0G/QlLQ506T+Fk/fWMa2VmBUiEI2cuMK4djI= -github.com/ldez/exptostd v0.3.1 h1:90yWWoAKMFHeovTK8uzBms9Ppp8Du/xQ20DRO26Ymrw= -github.com/ldez/exptostd v0.3.1/go.mod h1:iZBRYaUmcW5jwCR3KROEZ1KivQQp6PHXbDPk9hqJKCQ= -github.com/ldez/gomoddirectives v0.6.0 h1:Jyf1ZdTeiIB4dd+2n4qw+g4aI9IJ6JyfOZ8BityWvnA= -github.com/ldez/gomoddirectives v0.6.0/go.mod h1:TuwOGYoPAoENDWQpe8DMqEm5nIfjrxZXmxX/CExWyZ4= -github.com/ldez/grignotin v0.7.0 h1:vh0dI32WhHaq6LLPZ38g7WxXuZ1+RzyrJ7iPG9JMa8c= -github.com/ldez/grignotin v0.7.0/go.mod h1:uaVTr0SoZ1KBii33c47O1M8Jp3OP3YDwhZCmzT9GHEk= +github.com/ldez/exptostd v0.4.2 h1:l5pOzHBz8mFOlbcifTxzfyYbgEmoUqjxLFHZkjlbHXs= +github.com/ldez/exptostd v0.4.2/go.mod h1:iZBRYaUmcW5jwCR3KROEZ1KivQQp6PHXbDPk9hqJKCQ= +github.com/ldez/gomoddirectives v0.6.1 h1:Z+PxGAY+217f/bSGjNZr/b2KTXcyYLgiWI6geMBN2Qc= +github.com/ldez/gomoddirectives v0.6.1/go.mod h1:cVBiu3AHR9V31em9u2kwfMKD43ayN5/XDgr+cdaFaKs= +github.com/ldez/grignotin v0.9.0 h1:MgOEmjZIVNn6p5wPaGp/0OKWyvq42KnzAt/DAb8O4Ow= +github.com/ldez/grignotin v0.9.0/go.mod h1:uaVTr0SoZ1KBii33c47O1M8Jp3OP3YDwhZCmzT9GHEk= github.com/ldez/tagliatelle v0.7.1 h1:bTgKjjc2sQcsgPiT902+aadvMjCeMHrY7ly2XKFORIk= github.com/ldez/tagliatelle v0.7.1/go.mod h1:3zjxUpsNB2aEZScWiZTHrAXOl1x25t3cRmzfK1mlo2I= github.com/ldez/usetesting v0.4.2 h1:J2WwbrFGk3wx4cZwSMiCQQ00kjGR0+tuuyW0Lqm4lwA= @@ -296,15 +303,16 @@ github.com/maratori/testableexamples v1.0.0 h1:dU5alXRrD8WKSjOUnmJZuzdxWOEQ57+7s github.com/maratori/testableexamples v1.0.0/go.mod h1:4rhjL1n20TUTT4vdh3RDqSizKLyXp7K2u6HgraZCGzE= github.com/maratori/testpackage v1.1.1 h1:S58XVV5AD7HADMmD0fNnziNHqKvSdDuEKdPD1rNTU04= github.com/maratori/testpackage v1.1.1/go.mod h1:s4gRK/ym6AMrqpOa/kEbQTV4Q4jb7WeLZzVhVVVOQMc= -github.com/matoous/godox v0.0.0-20240105082147-c5b5e0e7c0c0 h1:Ny7cm4KSWceJLYyI1sm+aFIVDWSGXLcOJ0O0UaS5wdU= -github.com/matoous/godox v0.0.0-20240105082147-c5b5e0e7c0c0/go.mod h1:jgE/3fUXiTurkdHOLT5WEkThTSuE7yxHv5iWPa80afs= +github.com/matoous/godox v1.1.0 h1:W5mqwbyWrwZv6OQ5Z1a/DHGMOvXYCBP3+Ht7KMoJhq4= +github.com/matoous/godox v1.1.0/go.mod h1:jgE/3fUXiTurkdHOLT5WEkThTSuE7yxHv5iWPa80afs= github.com/matryer/is v1.4.0/go.mod h1:8I/i5uYgLzgsgEloJE1U6xx5HkBQpAZvepWuujKwMRU= github.com/matryer/is v1.4.1 h1:55ehd8zaGABKLXQUe2awZ99BD/PTc2ls+KV/dXphgEQ= github.com/matryer/is v1.4.1/go.mod h1:8I/i5uYgLzgsgEloJE1U6xx5HkBQpAZvepWuujKwMRU= github.com/mattn/go-colorable v0.1.9/go.mod h1:u6P/XSegPjTcexA+o6vUJrdnUu04hMope9wVRipJSqc= github.com/mattn/go-colorable v0.1.12/go.mod h1:u5H1YNBxpqRaxsYJYSkiCWKzEfiAb1Gb520KVy5xxl4= -github.com/mattn/go-colorable v0.1.13 h1:fFA4WZxdEF4tXPZVKMLwD8oUnCTTo08duU7wxecdEvA= github.com/mattn/go-colorable v0.1.13/go.mod h1:7S9/ev0klgBDR4GtXTXX8a3vIGJpMovkB8vQcUbaXHg= +github.com/mattn/go-colorable v0.1.14 h1:9A9LHSqF/7dyVVX6g0U9cwm9pG3kP9gSzcuIPHPsaIE= +github.com/mattn/go-colorable v0.1.14/go.mod h1:6LmQG8QLFO4G5z1gPvYEzlUgJ2wF+stgPZH1UqBm1s8= github.com/mattn/go-isatty v0.0.12/go.mod h1:cbi8OIDigv2wuxKPP5vlRcQ1OAZbq2CE4Kysco4FUpU= github.com/mattn/go-isatty v0.0.14/go.mod h1:7GGIvUiUoEMVVmxf/4nioHXj79iQHKdU27kJ6hsGG94= github.com/mattn/go-isatty v0.0.16/go.mod h1:kYGgaQfpe5nmfYZH+SKPsOc2e4SrIfOl2e/yFXSvRLM= @@ -314,8 +322,8 @@ github.com/mattn/go-isatty v0.0.20/go.mod h1:W+V8PltTTMOvKvAeJH7IuucS94S2C6jfK/D github.com/mattn/go-runewidth v0.0.9/go.mod h1:H031xJmbD/WCDINGzjvQ9THkh0rPKHF+m2gUSrubnMI= github.com/mattn/go-runewidth v0.0.16 h1:E5ScNMtiwvlvB5paMFdw9p4kSQzbXFikJ5SQO6TULQc= github.com/mattn/go-runewidth v0.0.16/go.mod h1:Jdepj2loyihRzMpdS35Xk/zdY8IAYHsh153qUoGf23w= -github.com/mgechev/revive v1.5.1 h1:hE+QPeq0/wIzJwOphdVyUJ82njdd8Khp4fUIHGZHW3M= -github.com/mgechev/revive v1.5.1/go.mod h1:lC9AhkJIBs5zwx8wkudyHrU+IJkrEKmpCmGMnIJPk4o= +github.com/mgechev/revive v1.7.0 h1:JyeQ4yO5K8aZhIKf5rec56u0376h8AlKNQEmjfkjKlY= +github.com/mgechev/revive v1.7.0/go.mod h1:qZnwcNhoguE58dfi96IJeSTPeZQejNeoMQLUZGi4SW4= github.com/mitchellh/copystructure v1.2.0 h1:vpKXTN4ewci03Vljg/q9QvCGUDttBOGBIa15WveJJGw= github.com/mitchellh/copystructure v1.2.0/go.mod h1:qLl+cE2AmVv+CoeAwDPye/v+N2HKCj9FbZEVFJRxO9s= github.com/mitchellh/go-homedir v1.1.0 h1:lukF9ziXFxDFPkA1vsr5zpc1XuPDn/wFntq5mG+4E0Y= @@ -339,16 +347,16 @@ github.com/nishanths/exhaustive v0.12.0 h1:vIY9sALmw6T/yxiASewa4TQcFsVYZQQRUQJhK github.com/nishanths/exhaustive v0.12.0/go.mod h1:mEZ95wPIZW+x8kC4TgC+9YCUgiST7ecevsVDTgc2obs= github.com/nishanths/predeclared v0.2.2 h1:V2EPdZPliZymNAn79T8RkNApBjMmVKh5XRpLm/w98Vk= github.com/nishanths/predeclared v0.2.2/go.mod h1:RROzoN6TnGQupbC+lqggsOlcgysk3LMK/HI84Mp280c= -github.com/nunnatsa/ginkgolinter v0.18.4 h1:zmX4KUR+6fk/vhUFt8DOP6KwznekhkmVSzzVJve2vyM= -github.com/nunnatsa/ginkgolinter v0.18.4/go.mod h1:AMEane4QQ6JwFz5GgjI5xLUM9S/CylO+UyM97fN2iBI= +github.com/nunnatsa/ginkgolinter v0.19.1 h1:mjwbOlDQxZi9Cal+KfbEJTCz327OLNfwNvoZ70NJ+c4= +github.com/nunnatsa/ginkgolinter v0.19.1/go.mod h1:jkQ3naZDmxaZMXPWaS9rblH+i+GWXQCaS/JFIWcOH2s= github.com/oklog/run v1.1.0 h1:GEenZ1cK0+q0+wsJew9qUg/DyD8k3JzYsZAi5gYi2mA= github.com/oklog/run v1.1.0/go.mod h1:sVPdnTZT1zYwAJeCMu2Th4T21pA3FPOQRfWjQlk7DVU= github.com/olekukonko/tablewriter v0.0.5 h1:P2Ga83D34wi1o9J6Wh1mRuqd4mF/x/lgBS7N7AbDhec= github.com/olekukonko/tablewriter v0.0.5/go.mod h1:hPp6KlRPjbx+hW8ykQs1w3UBbZlj6HuIJcUGPhkA7kY= -github.com/onsi/ginkgo/v2 v2.20.2 h1:7NVCeyIWROIAheY21RLS+3j2bb52W0W82tkberYytp4= -github.com/onsi/ginkgo/v2 v2.20.2/go.mod h1:K9gyxPIlb+aIvnZ8bd9Ak+YP18w3APlR+5coaZoE2ag= -github.com/onsi/gomega v1.34.2 h1:pNCwDkzrsv7MS9kpaQvVb1aVLahQXyJ/Tv5oAZMI3i8= -github.com/onsi/gomega v1.34.2/go.mod h1:v1xfxRgk0KIsG+QOdm7p8UosrOzPYRo60fd3B/1Dukc= +github.com/onsi/ginkgo/v2 v2.22.2 h1:/3X8Panh8/WwhU/3Ssa6rCKqPLuAkVY2I0RoyDLySlU= +github.com/onsi/ginkgo/v2 v2.22.2/go.mod h1:oeMosUL+8LtarXBHu/c0bx2D/K9zyQ6uX3cTyztHwsk= +github.com/onsi/gomega v1.36.2 h1:koNYke6TVk6ZmnyHrCXba/T/MoLBXFjeC1PtvYgw0A8= +github.com/onsi/gomega v1.36.2/go.mod h1:DdwyADRjrc825LhMEkD76cHR5+pUnjhUN8GlHlRPHzY= github.com/otiai10/copy v1.2.0/go.mod h1:rrF5dJ5F0t/EWSYODDu4j9/vEeYHMkc8jt0zJChqQWw= github.com/otiai10/copy v1.14.0 h1:dCI/t1iTdYGtkvCuBG2BgR6KZa83PTclw4U5n2wAllU= github.com/otiai10/copy v1.14.0/go.mod h1:ECfuL02W+/FkTWZWgQqXPWZgW9oeKCSQ5qVfSc4qc4w= @@ -362,8 +370,8 @@ github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINE github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2 h1:Jamvg5psRIccs7FGNTlIRMkT8wgtp5eCXdBlqhYGL6U= github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= -github.com/polyfloyd/go-errorlint v1.7.0 h1:Zp6lzCK4hpBDj8y8a237YK4EPrMXQWvOe3nGoH4pFrU= -github.com/polyfloyd/go-errorlint v1.7.0/go.mod h1:dGWKu85mGHnegQ2SWpEybFityCg3j7ZbwsVUxAOk9gY= +github.com/polyfloyd/go-errorlint v1.7.1 h1:RyLVXIbosq1gBdk/pChWA8zWYLsq9UEw7a1L5TVMCnA= +github.com/polyfloyd/go-errorlint v1.7.1/go.mod h1:aXjNb1x2TNhoLsk26iv1yl7a+zTnXPhwEMtEXukiLR8= github.com/prashantv/gostub v1.1.0 h1:BTyx3RfQjRHnUWaGF9oQos79AlQ5k8WNktv7VGvVH4g= github.com/prashantv/gostub v1.1.0/go.mod h1:A5zLQHz7ieHGG7is6LLXLz7I8+3LZzsrV0P1IAHhP5U= github.com/prometheus/client_golang v1.20.2 h1:5ctymQzZlyOON1666svgwn3s6IKWgfbjsejTMiXIyjg= @@ -389,11 +397,11 @@ github.com/raeperd/recvcheck v0.2.0/go.mod h1:n04eYkwIR0JbgD73wT8wL4JjPC3wm0nFtz github.com/rivo/uniseg v0.2.0/go.mod h1:J6wj4VEh+S6ZtnVlnTBMWIodfgj8LQOQFoIToxlJtxc= github.com/rivo/uniseg v0.4.7 h1:WUdvkW8uEhrYfLC4ZzdpI2ztxP1I582+49Oc5Mq64VQ= github.com/rivo/uniseg v0.4.7/go.mod h1:FN3SvrM+Zdj16jyLfmOkMNblXMcoc8DfTHruCPUcx88= -github.com/rogpeppe/go-internal v1.13.1 h1:KvO1DLK/DRN07sQ1LQKScxyZJuNnedQ5/wKSR38lUII= -github.com/rogpeppe/go-internal v1.13.1/go.mod h1:uMEvuHeurkdAXX61udpOXGD/AzZDWNMNyH2VO9fmH0o= -github.com/rs/xid v1.5.0/go.mod h1:trrq9SKmegXys3aeAKXMUTdJsYXVwGY3RLcfgqegfbg= -github.com/rs/zerolog v1.33.0 h1:1cU2KZkvPxNyfgEmhHAz/1A9Bz+llsdYzklWFzgp0r8= -github.com/rs/zerolog v1.33.0/go.mod h1:/7mN4D5sKwJLZQ2b/znpjC3/GQWY/xaDXUM0kKWRHss= +github.com/rogpeppe/go-internal v1.14.1 h1:UQB4HGPB6osV0SQTLymcB4TgvyWu6ZyliaW0tI/otEQ= +github.com/rogpeppe/go-internal v1.14.1/go.mod h1:MaRKkUm5W0goXpeCfT7UZI6fk/L7L7so1lCWt35ZSgc= +github.com/rs/xid v1.6.0/go.mod h1:7XoLgs4eV+QndskICGsho+ADou8ySMSjJKDIan90Nz0= +github.com/rs/zerolog v1.34.0 h1:k43nTLIwcTVQAncfCw4KZ2VY6ukYoZaBPNOE8txlOeY= +github.com/rs/zerolog v1.34.0/go.mod h1:bJsvje4Z08ROH4Nhs5iH600c3IkWhwp44iRc54W6wYQ= github.com/russross/blackfriday/v2 v2.1.0/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= github.com/ryancurrah/gomodguard v1.3.5 h1:cShyguSwUEeC0jS7ylOiG/idnd1TpJ1LfHGpV3oJmPU= github.com/ryancurrah/gomodguard v1.3.5/go.mod h1:MXlEPQRxgfPQa62O8wzK3Ozbkv9Rkqr+wKjSxTdsNJE= @@ -403,6 +411,12 @@ github.com/sagikazarmark/locafero v0.6.0 h1:ON7AQg37yzcRPU69mt7gwhFEBwxI6P9T4Qu3 github.com/sagikazarmark/locafero v0.6.0/go.mod h1:77OmuIc6VTraTXKXIs/uvUxKGUXjE1GbemJYHqdNjX0= github.com/sagikazarmark/slog-shim v0.1.0 h1:diDBnUNK9N/354PgrxMywXnAwEr1QZcOr6gto+ugjYE= github.com/sagikazarmark/slog-shim v0.1.0/go.mod h1:SrcSrq8aKtyuqEI1uvTDTK1arOWRIczQRv+GVI1AkeQ= +github.com/samber/lo v1.47.0 h1:z7RynLwP5nbyRscyvcD043DWYoOcYRv3mV8lBeqOCLc= +github.com/samber/lo v1.47.0/go.mod h1:RmDH9Ct32Qy3gduHQuKJ3gW1fMHAnE/fAzQuf6He5cU= +github.com/samber/slog-common v0.18.1 h1:c0EipD/nVY9HG5shgm/XAs67mgpWDMF+MmtptdJNCkQ= +github.com/samber/slog-common v0.18.1/go.mod h1:QNZiNGKakvrfbJ2YglQXLCZauzkI9xZBjOhWFKS3IKk= +github.com/samber/slog-zerolog/v2 v2.7.3 h1:/MkPDl/tJhijN2GvB1MWwBn2FU8RiL3rQ8gpXkQm2EY= +github.com/samber/slog-zerolog/v2 v2.7.3/go.mod h1:oWU7WHof4Xp8VguiNO02r1a4VzkgoOyOZhY5CuRke60= github.com/sanposhiho/wastedassign/v2 v2.1.0 h1:crurBF7fJKIORrV85u9UUpePDYGWnwvv3+A96WvwXT0= github.com/sanposhiho/wastedassign/v2 v2.1.0/go.mod h1:+oSmSC+9bQ+VUAxA66nBb0Z7N8CK7mscKTDYC6aIek4= github.com/santhosh-tekuri/jsonschema/v6 v6.0.1 h1:PKK9DyHxif4LZo+uQSgXNqs0jj5+xZwwfKHgph2lxBw= @@ -411,10 +425,8 @@ github.com/sashamelentyev/interfacebloat v1.1.0 h1:xdRdJp0irL086OyW1H/RTZTr1h/tM github.com/sashamelentyev/interfacebloat v1.1.0/go.mod h1:+Y9yU5YdTkrNvoX0xHc84dxiN1iBi9+G8zZIhPVoNjQ= github.com/sashamelentyev/usestdlibvars v1.28.0 h1:jZnudE2zKCtYlGzLVreNp5pmCdOxXUzwsMDBkR21cyQ= github.com/sashamelentyev/usestdlibvars v1.28.0/go.mod h1:9nl0jgOfHKWNFS43Ojw0i7aRoS4j6EBye3YBhmAIRF8= -github.com/securego/gosec/v2 v2.21.4 h1:Le8MSj0PDmOnHJgUATjD96PaXRvCpKC+DGJvwyy0Mlk= -github.com/securego/gosec/v2 v2.21.4/go.mod h1:Jtb/MwRQfRxCXyCm1rfM1BEiiiTfUOdyzzAhlr6lUTA= -github.com/shazow/go-diff v0.0.0-20160112020656-b6b7b6733b8c h1:W65qqJCIOVP4jpqPQ0YvHYKwcMEMVWIzWC5iNQQfBTU= -github.com/shazow/go-diff v0.0.0-20160112020656-b6b7b6733b8c/go.mod h1:/PevMnwAxekIXwN8qQyfc5gl2NlkB3CQlkizAbOkeBs= +github.com/securego/gosec/v2 v2.22.2 h1:IXbuI7cJninj0nRpZSLCUlotsj8jGusohfONMrHoF6g= +github.com/securego/gosec/v2 v2.22.2/go.mod h1:UEBGA+dSKb+VqM6TdehR7lnQtIIMorYJ4/9CW1KVQBE= github.com/shopspring/decimal v1.4.0 h1:bxl37RwXBklmTi0C79JfXCEBD1cqqHt0bbgBAGFp81k= github.com/shopspring/decimal v1.4.0/go.mod h1:gawqmDU56v4yIKSwfBSFip1HdCCXN8/+DMd9qYNcwME= github.com/shurcooL/go v0.0.0-20180423040247-9e1955d9fb6e/go.mod h1:TDJrrUr11Vxrven61rcy3hJMUqaf/CLWYhHNPmT14Lk= @@ -431,14 +443,15 @@ github.com/sourcegraph/conc v0.3.0 h1:OQTbbt6P72L20UqAkXXuLOj79LfEanQ+YQFNpLA9yS github.com/sourcegraph/conc v0.3.0/go.mod h1:Sdozi7LEKbFPqYX2/J+iBAM6HpqSLTASQIKqDmF7Mt0= github.com/sourcegraph/go-diff v0.7.0 h1:9uLlrd5T46OXs5qpp8L/MTltk0zikUGi0sNNyCpA8G0= github.com/sourcegraph/go-diff v0.7.0/go.mod h1:iBszgVvyxdc8SFZ7gm69go2KDdt3ag071iBaWPF6cjs= -github.com/spf13/afero v1.11.0 h1:WJQKhtpdm3v2IzqG8VMqrr6Rf3UYpEF239Jy9wNepM8= -github.com/spf13/afero v1.11.0/go.mod h1:GH9Y3pIexgf1MTIWtNGyogA5MwRIDXGUr+hbWNoBjkY= +github.com/spf13/afero v1.12.0 h1:UcOPyRBYczmFn6yvphxkn9ZEOY65cpwGKb5mL36mrqs= +github.com/spf13/afero v1.12.0/go.mod h1:ZTlWwG4/ahT8W7T0WQ5uYmjI9duaLQGy3Q2OAl4sk/4= github.com/spf13/cast v1.7.0 h1:ntdiHjuueXFgm5nzDRdOS4yfT43P5Fnud6DH50rz/7w= github.com/spf13/cast v1.7.0/go.mod h1:ancEpBxwJDODSW/UG4rDrAqiKolqNNh2DX3mk86cAdo= -github.com/spf13/cobra v1.8.1 h1:e5/vxKd/rZsfSJMUX1agtjeTDf+qv1/JdBF8gg5k9ZM= -github.com/spf13/cobra v1.8.1/go.mod h1:wHxEcudfqmLYa8iTfL+OuZPbBZkmvliBWKIezN3kD9Y= -github.com/spf13/pflag v1.0.5 h1:iy+VFUOCP1a+8yFto/drg2CJ5u0yRoB7fZw3DKv/JXA= +github.com/spf13/cobra v1.9.1 h1:CXSaggrXdbHK9CF+8ywj8Amf7PBRmPCOJugH954Nnlo= +github.com/spf13/cobra v1.9.1/go.mod h1:nDyEzZ8ogv936Cinf6g1RU9MRY64Ir93oCnqb9wxYW0= github.com/spf13/pflag v1.0.5/go.mod h1:McXfInJRrz4CZXVZOBLb0bTZqETkiAhM9Iw0y3An2Bg= +github.com/spf13/pflag v1.0.6 h1:jFzHGLGAlb3ruxLB8MhbI6A8+AQX/2eW4qeyNZXNp2o= +github.com/spf13/pflag v1.0.6/go.mod h1:McXfInJRrz4CZXVZOBLb0bTZqETkiAhM9Iw0y3An2Bg= github.com/spf13/viper v1.19.0 h1:RWq5SEjt8o25SROyN3z2OrDB9l7RPd3lwTWU8EcEdcI= github.com/spf13/viper v1.19.0/go.mod h1:GQUN9bilAbhU/jgc1bKs99f/suXKeUMct8Adx5+Ntkg= github.com/ssgreg/nlreturn/v2 v2.2.1 h1:X4XDI7jstt3ySqGU86YGAURbxw3oTDPK9sPEi6YEwQ0= @@ -463,14 +476,14 @@ github.com/stretchr/testify v1.10.0 h1:Xv5erBjTwe/5IxqUQTdXv5kgmIvbHo3QQyRwhJsOf github.com/stretchr/testify v1.10.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY= github.com/subosito/gotenv v1.6.0 h1:9NlTDc1FTs4qu0DDq7AEtTPNw6SVm7uBMsUCUjABIf8= github.com/subosito/gotenv v1.6.0/go.mod h1:Dk4QP5c2W3ibzajGcXpNraDfq2IrhjMIvMSWPKKo0FU= -github.com/tdakkota/asciicheck v0.3.0 h1:LqDGgZdholxZMaJgpM6b0U9CFIjDCbFdUF00bDnBKOQ= -github.com/tdakkota/asciicheck v0.3.0/go.mod h1:KoJKXuX/Z/lt6XzLo8WMBfQGzak0SrAKZlvRr4tg8Ac= +github.com/tdakkota/asciicheck v0.4.1 h1:bm0tbcmi0jezRA2b5kg4ozmMuGAFotKI3RZfrhfovg8= +github.com/tdakkota/asciicheck v0.4.1/go.mod h1:0k7M3rCfRXb0Z6bwgvkEIMleKH3kXNz9UqJ9Xuqopr8= github.com/tenntenn/modver v1.0.1 h1:2klLppGhDgzJrScMpkj9Ujy3rXPUspSjAcev9tSEBgA= github.com/tenntenn/modver v1.0.1/go.mod h1:bePIyQPb7UeioSRkw3Q0XeMhYZSMx9B8ePqg6SAMGH0= github.com/tenntenn/text/transform v0.0.0-20200319021203-7eef512accb3 h1:f+jULpRQGxTSkNYKJ51yaw6ChIqO+Je8UqsTKN/cDag= github.com/tenntenn/text/transform v0.0.0-20200319021203-7eef512accb3/go.mod h1:ON8b8w4BN/kE1EOhwT0o+d62W65a6aPw1nouo9LMgyY= -github.com/tetafro/godot v1.4.20 h1:z/p8Ek55UdNvzt4TFn2zx2KscpW4rWqcnUrdmvWJj7E= -github.com/tetafro/godot v1.4.20/go.mod h1:2oVxTBSftRTh4+MVfUaUXR6bn2GDXCaMcOG4Dk3rfio= +github.com/tetafro/godot v1.5.0 h1:aNwfVI4I3+gdxjMgYPus9eHmoBeJIbnajOyqZYStzuw= +github.com/tetafro/godot v1.5.0/go.mod h1:2oVxTBSftRTh4+MVfUaUXR6bn2GDXCaMcOG4Dk3rfio= github.com/timakin/bodyclose v0.0.0-20241017074812-ed6a65f985e3 h1:y4mJRFlM6fUyPhoXuFg/Yu02fg/nIPFMOY8tOqppoFg= github.com/timakin/bodyclose v0.0.0-20241017074812-ed6a65f985e3/go.mod h1:mkjARE7Yr8qU23YcGMSALbIxTQ9r9QBVahQOBRfU460= github.com/timonwong/loggercheck v0.10.1 h1:uVZYClxQFpw55eh+PIoqM7uAOHMrhVcDoWDery9R8Lg= @@ -487,8 +500,8 @@ github.com/ultraware/whitespace v0.2.0 h1:TYowo2m9Nfj1baEQBjuHzvMRbp19i+RCcRYrSW github.com/ultraware/whitespace v0.2.0/go.mod h1:XcP1RLD81eV4BW8UhQlpaR+SDc2givTvyI8a586WjW8= github.com/uudashr/gocognit v1.2.0 h1:3BU9aMr1xbhPlvJLSydKwdLN3tEUUrzPSSM8S4hDYRA= github.com/uudashr/gocognit v1.2.0/go.mod h1:k/DdKPI6XBZO1q7HgoV2juESI2/Ofj9AcHPZhBBdrTU= -github.com/uudashr/iface v1.3.0 h1:zwPch0fs9tdh9BmL5kcgSpvnObV+yHjO4JjVBl8IA10= -github.com/uudashr/iface v1.3.0/go.mod h1:4QvspiRd3JLPAEXBQ9AiZpLbJlrWWgRChOKDJEuQTdg= +github.com/uudashr/iface v1.3.1 h1:bA51vmVx1UIhiIsQFSNq6GZ6VPTk3WNMZgRiCe9R29U= +github.com/uudashr/iface v1.3.1/go.mod h1:4QvspiRd3JLPAEXBQ9AiZpLbJlrWWgRChOKDJEuQTdg= github.com/xen0n/gosmopolitan v1.2.2 h1:/p2KTnMzwRexIW8GlKawsTWOxn7UHA+jCMF/V8HHtvU= github.com/xen0n/gosmopolitan v1.2.2/go.mod h1:7XX7Mj61uLYrj0qmeN0zi7XDon9JRAEhYQqAPLVNTeg= github.com/yagipy/maintidx v1.0.0 h1:h5NvIsCz+nRDapQ0exNv4aJ0yXSI0420omVANTv3GJM= @@ -509,14 +522,26 @@ go-simpler.org/assert v0.9.0 h1:PfpmcSvL7yAnWyChSjOz6Sp6m9j5lyK8Ok9pEL31YkQ= go-simpler.org/assert v0.9.0/go.mod h1:74Eqh5eI6vCK6Y5l3PI8ZYFXG4Sa+tkr70OIPJAUr28= go-simpler.org/musttag v0.13.0 h1:Q/YAW0AHvaoaIbsPj3bvEI5/QFP7w696IMUpnKXQfCE= go-simpler.org/musttag v0.13.0/go.mod h1:FTzIGeK6OkKlUDVpj0iQUXZLUO1Js9+mvykDQy9C5yM= -go-simpler.org/sloglint v0.7.2 h1:Wc9Em/Zeuu7JYpl+oKoYOsQSy2X560aVueCW/m6IijY= -go-simpler.org/sloglint v0.7.2/go.mod h1:US+9C80ppl7VsThQclkM7BkCHQAzuz8kHLsW3ppuluo= +go-simpler.org/sloglint v0.9.0 h1:/40NQtjRx9txvsB/RN022KsUJU+zaaSb/9q9BSefSrE= +go-simpler.org/sloglint v0.9.0/go.mod h1:G/OrAF6uxj48sHahCzrbarVMptL2kjWTaUeC8+fOGww= +go.opentelemetry.io/auto/sdk v1.1.0 h1:cH53jehLUN6UFLY71z+NDOiNJqDdPRaXzTel0sJySYA= +go.opentelemetry.io/auto/sdk v1.1.0/go.mod h1:3wSPjt5PWp2RhlCcmmOial7AvC4DQqZb7a7wCow3W8A= +go.opentelemetry.io/otel v1.35.0 h1:xKWKPxrxB6OtMCbmMY021CqC45J+3Onta9MqjhnusiQ= +go.opentelemetry.io/otel v1.35.0/go.mod h1:UEqy8Zp11hpkUrL73gSlELM0DupHoiq72dR+Zqel/+Y= +go.opentelemetry.io/otel/metric v1.35.0 h1:0znxYu2SNyuMSQT4Y9WDWej0VpcsxkuklLa4/siN90M= +go.opentelemetry.io/otel/metric v1.35.0/go.mod h1:nKVFgxBZ2fReX6IlyW28MgZojkoAkJGaE8CpgeAU3oE= +go.opentelemetry.io/otel/sdk v1.34.0 h1:95zS4k/2GOy069d321O8jWgYsW3MzVV+KuSPKp7Wr1A= +go.opentelemetry.io/otel/sdk v1.34.0/go.mod h1:0e/pNiaMAqaykJGKbi+tSjWfNNHMTxoC9qANsCzbyxU= +go.opentelemetry.io/otel/sdk/metric v1.34.0 h1:5CeK9ujjbFVL5c1PhLuStg1wxA7vQv7ce1EK0Gyvahk= +go.opentelemetry.io/otel/sdk/metric v1.34.0/go.mod h1:jQ/r8Ze28zRKoNRdkjCZxfs6YvBTG1+YIqyFVFYec5w= +go.opentelemetry.io/otel/trace v1.35.0 h1:dPpEfJu1sDIqruz7BHFG3c7528f6ddfSWfFDVt/xgMs= +go.opentelemetry.io/otel/trace v1.35.0/go.mod h1:WUk7DtFp1Aw2MkvqGdwiXYDZZNvA/1J8o6xRXLrIkyc= go.uber.org/automaxprocs v1.6.0 h1:O3y2/QNTOdbF+e/dpXNNW7Rx2hZ4sTIPyybbxyNqTUs= go.uber.org/automaxprocs v1.6.0/go.mod h1:ifeIMSnPZuznNm6jmdzmU3/bfk01Fe2fotchwEFJ8r8= go.uber.org/goleak v1.3.0 h1:2K3zAYmnTNqV73imy9J1T3WC+gmCePx2hEGkimedGto= go.uber.org/goleak v1.3.0/go.mod h1:CoHD4mav9JJNrW/WLlf7HGZPjdw8EucARQHekz1X6bE= -go.uber.org/mock v0.5.0 h1:KAMbZvZPyBPWgD14IrIQ38QCyjwpvVVV6K/bHl1IwQU= -go.uber.org/mock v0.5.0/go.mod h1:ge71pBPLYDk7QIi1LupWxdAykm7KIEFchiOqd6z7qMM= +go.uber.org/mock v0.5.2 h1:LbtPTcP8A5k9WPXj54PPPbjcI4Y6lhyOZXn+VS7wNko= +go.uber.org/mock v0.5.2/go.mod h1:wLlUxC2vVTPTaE3UD51E0BGOAElKrILxhVSDYQLld5o= 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/zap v1.27.0 h1:aJMhYGrd5QSmlpLMr2MftRKl7t8J8PTZPA732ud/XR8= @@ -527,14 +552,14 @@ golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPh golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= golang.org/x/crypto v0.13.0/go.mod h1:y6Z2r+Rw4iayiXXAIxJIDAJ1zMW4yaTpebo8fPOliYc= golang.org/x/crypto v0.14.0/go.mod h1:MVFd36DqK4CsrnJYDkBA3VC4m2GkXAM0PvzMCn4JQf4= -golang.org/x/crypto v0.32.0 h1:euUpcYgM8WcP71gNpTqQCn6rC2t6ULUPiOzfWaXVVfc= -golang.org/x/crypto v0.32.0/go.mod h1:ZnnJkOaASj8g0AjIduWNlq2NRxL0PlBrbKVyZ6V/Ugc= -golang.org/x/exp v0.0.0-20241009180824-f66d83c29e7c h1:7dEasQXItcW1xKJ2+gg5VOiBnqWrJc+rq0DPKyvvdbY= -golang.org/x/exp v0.0.0-20241009180824-f66d83c29e7c/go.mod h1:NQtJDoLvd6faHhE7m4T/1IY708gDefGGjR/iUW8yQQ8= +golang.org/x/crypto v0.38.0 h1:jt+WWG8IZlBnVbomuhg2Mdq0+BBQaHbtqHEFEigjUV8= +golang.org/x/crypto v0.38.0/go.mod h1:MvrbAqul58NNYPKnOra203SB9vpuZW0e+RRZV+Ggqjw= +golang.org/x/exp v0.0.0-20250506013437-ce4c2cf36ca6 h1:y5zboxd6LQAqYIhHnB48p0ByQ/GnQx2BE33L8BOHQkI= +golang.org/x/exp v0.0.0-20250506013437-ce4c2cf36ca6/go.mod h1:U6Lno4MTRCDY+Ba7aCcauB9T60gsv5s4ralQzP72ZoQ= golang.org/x/exp/typeparams v0.0.0-20220428152302-39d4317da171/go.mod h1:AbB0pIl9nAr9wVwH+Z2ZpaocVmF5I4GyWCDIsVjR0bk= golang.org/x/exp/typeparams v0.0.0-20230203172020-98cc5a0785f9/go.mod h1:AbB0pIl9nAr9wVwH+Z2ZpaocVmF5I4GyWCDIsVjR0bk= -golang.org/x/exp/typeparams v0.0.0-20241108190413-2d47ceb2692f h1:WTyX8eCCyfdqiPYkRGm0MqElSfYFH3yR1+rl/mct9sA= -golang.org/x/exp/typeparams v0.0.0-20241108190413-2d47ceb2692f/go.mod h1:AbB0pIl9nAr9wVwH+Z2ZpaocVmF5I4GyWCDIsVjR0bk= +golang.org/x/exp/typeparams v0.0.0-20250210185358-939b2ce775ac h1:TSSpLIG4v+p0rPv1pNOQtl1I8knsO4S9trOxNMOLVP4= +golang.org/x/exp/typeparams v0.0.0-20250210185358-939b2ce775ac/go.mod h1:AbB0pIl9nAr9wVwH+Z2ZpaocVmF5I4GyWCDIsVjR0bk= golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.4.1/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= @@ -546,9 +571,8 @@ golang.org/x/mod v0.8.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= golang.org/x/mod v0.9.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= golang.org/x/mod v0.12.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= golang.org/x/mod v0.13.0/go.mod h1:hTbmBsO62+eylJbnUtE2MGJUyE7QWk4xUqPFrRgJ+7c= -golang.org/x/mod v0.22.0 h1:D4nJWe9zXqHOmWqj4VMOJhvzj7bEZg4wEYa759z1pH4= -golang.org/x/mod v0.22.0/go.mod h1:6SkKJ3Xj0I0BrPOZoBy3bdMptDDU9oJrpohJ3eWZ1fY= -golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/mod v0.24.0 h1:ZfthKaKaT4NrhGVZHO1/WDTwGES4De8KtWO0SIbNJMU= +golang.org/x/mod v0.24.0/go.mod h1:IXM97Txy2VM4PJ3gI61r1YEk/gAj6zAHN3AdZt6S9Ww= golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= @@ -559,14 +583,13 @@ golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96b golang.org/x/net v0.0.0-20211015210444-4f30a5c0130f/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= golang.org/x/net v0.2.0/go.mod h1:KqCZLdyyvdV855qA2rE3GC2aiw5xGR5TEjj8smXukLY= -golang.org/x/net v0.5.0/go.mod h1:DivGGAXEgPSlEBzxGzZI+ZLohi+xUj054jfeKui00ws= golang.org/x/net v0.6.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/net v0.8.0/go.mod h1:QVkue5JL9kW//ek3r6jTKnTFis1tRmNAW2P1shuFdJc= golang.org/x/net v0.10.0/go.mod h1:0qNGK6F8kojg2nk9dLZ2mShWaEBan6FAoqfSigmmuDg= golang.org/x/net v0.15.0/go.mod h1:idbUs1IY1+zTqbi8yxTbhexhEEk5ur9LInksu6HrEpk= golang.org/x/net v0.16.0/go.mod h1:NxSsAGuq816PNPmqtQdLE42eU2Fs7NoRIZrHJAlaCOE= -golang.org/x/net v0.34.0 h1:Mb7Mrk043xzHgnRM88suvJFwzVrRfHEHJEl5/71CKw0= -golang.org/x/net v0.34.0/go.mod h1:di0qlW3YNM5oh6GqDGQr92MyTozJPmybPK4Ev/Gm31k= +golang.org/x/net v0.40.0 h1:79Xs7wF06Gbdcg4kdCCIQArK11Z1hr5POQ6+fIYHNuY= +golang.org/x/net v0.40.0/go.mod h1:y0hY0exeL2Pku80/zKK7tpntoX23cqL3Oa6njdgRtds= golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20200625203802-6e8e738ad208/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -576,8 +599,8 @@ golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.1.0/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.3.0/go.mod h1:FU7BRWz2tNW+3quACPkgCx/L+uEAv1htQ0V83Z9Rj+Y= golang.org/x/sync v0.4.0/go.mod h1:FU7BRWz2tNW+3quACPkgCx/L+uEAv1htQ0V83Z9Rj+Y= -golang.org/x/sync v0.10.0 h1:3NQrjDixjgGwUOCaF8w2+VYHv0Ve/vGYSbdkTa98gmQ= -golang.org/x/sync v0.10.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= +golang.org/x/sync v0.14.0 h1:woo0S4Yywslg6hp4eUFjTVOyKt0RookbpAHG4c1HmhQ= +golang.org/x/sync v0.14.0/go.mod h1:1dzgHSNfp02xaA81J2MS99Qcpr2w7fw1gpm99rleRqA= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200116001909-b77594299b42/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -600,18 +623,16 @@ golang.org/x/sys v0.0.0-20220715151400-c0bba94af5f8/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220811171246-fbc7d0a398ab/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.2.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.4.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.6.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.8.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.12.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.13.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.29.0 h1:TPYlXGxvx1MGTn2GiZDhnjPA9wZzZeGKHHmKhHYvgaU= -golang.org/x/sys v0.29.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.33.0 h1:q3i8TbbEz+JRD9ywIRlyRAQbM0qF7hu24q3teo2hbuw= +golang.org/x/sys v0.33.0/go.mod h1:BJP2sWEmIv4KK5OTEluFJCKSidICx8ciO85XgH3Ak8k= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= golang.org/x/term v0.2.0/go.mod h1:TVmDHMZPmdnySmBfhjOoOdhjzdE1h4u1VwSiw2l1Nuc= -golang.org/x/term v0.4.0/go.mod h1:9P2UbLfCdcvo3p/nzKvsmas4TnlujnuoV9hGgYzW1lQ= golang.org/x/term v0.5.0/go.mod h1:jMB1sMXY+tzblOD4FWmEbocvup2/aLOaQEp7JmGp78k= golang.org/x/term v0.6.0/go.mod h1:m6U89DPEgQRMq3DNkDClhWw02AUbt2daBVO4cn4Hv9U= golang.org/x/term v0.8.0/go.mod h1:xPskH00ivmX89bAKVGSKKtLOWNx2+17Eiy94tnKShWo= @@ -623,17 +644,15 @@ golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= golang.org/x/text v0.4.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= -golang.org/x/text v0.6.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= golang.org/x/text v0.7.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= golang.org/x/text v0.8.0/go.mod h1:e1OnstbJyHTd6l/uOt8jFFHp6TRDWZR/bV3emEE/zU8= golang.org/x/text v0.9.0/go.mod h1:e1OnstbJyHTd6l/uOt8jFFHp6TRDWZR/bV3emEE/zU8= golang.org/x/text v0.13.0/go.mod h1:TvPlkZtksWOMsz7fbANvkp4WM8x/WCo/om8BMLbz+aE= -golang.org/x/text v0.21.0 h1:zyQAAkrwaneQ066sspRyJaG9VNi/YJ1NfzcGB3hZ/qo= -golang.org/x/text v0.21.0/go.mod h1:4IBbMaMmOPCJ8SecivzSH54+73PCFmPWxNTLm+vZkEQ= -golang.org/x/time v0.8.0 h1:9i3RxcPv3PZnitoVGMPDKZSq1xW1gK1Xy3ArNOGZfEg= -golang.org/x/time v0.8.0/go.mod h1:3BpzKBy/shNhVucY/MWOyx10tF3SFh9QdLuxbVysPQM= +golang.org/x/text v0.25.0 h1:qVyWApTSYLk/drJRO5mDlNYskwQznZmkpV2c8q9zls4= +golang.org/x/text v0.25.0/go.mod h1:WEdwpYrmk1qmdHvhkSTNPm3app7v4rsT8F2UD6+VHIA= +golang.org/x/time v0.11.0 h1:/bpjEDfN9tkoN/ryeYHnv5hcMlc8ncjMcM4XBk5NWV0= +golang.org/x/time v0.11.0/go.mod h1:CDIdPxbZBQxdj6cxyCIdrNogrJKMJ7pr37NYpMcMDSg= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= -golang.org/x/tools v0.0.0-20190321232350-e250d351ecad/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20200324003944-a576cf524670/go.mod h1:Sl4aGygMT6LrqrWclx+PTx3U+LnKx/seiNR+3G19Ar8= golang.org/x/tools v0.0.0-20200329025819-fd4102a86c65/go.mod h1:Sl4aGygMT6LrqrWclx+PTx3U+LnKx/seiNR+3G19Ar8= @@ -647,23 +666,22 @@ golang.org/x/tools v0.1.5/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= golang.org/x/tools v0.1.10/go.mod h1:Uh6Zz+xoGYZom868N8YTex3t7RhtHDBrE8Gzo9bV56E= golang.org/x/tools v0.1.12/go.mod h1:hNGJHUnrk76NpqgfD5Aqm5Crs+Hm0VOH/i9J2+nxYbc= golang.org/x/tools v0.3.0/go.mod h1:/rWhSS2+zyEVwoJf8YAX6L2f0ntZ7Kn/mGgAWcipA5k= -golang.org/x/tools v0.5.0/go.mod h1:N+Kgy78s5I24c24dU8OfWNEotWjutIs8SnJvn5IDq+k= golang.org/x/tools v0.6.0/go.mod h1:Xwgl3UAJ/d3gWutnCtw505GrjyAbvKui8lOU390QaIU= golang.org/x/tools v0.7.0/go.mod h1:4pg6aUX35JBAogB10C9AtvVL+qowtN4pT3CGSQex14s= golang.org/x/tools v0.13.0/go.mod h1:HvlwmtVNQAhOuCjW7xxvovg8wbNq7LwfXh/k7wXUl58= golang.org/x/tools v0.14.0/go.mod h1:uYBEerGOWcJyEORxN+Ek8+TT266gXkNlHdJBwexUsBg= -golang.org/x/tools v0.29.0 h1:Xx0h3TtM9rzQpQuR4dKLrdglAmCEN5Oi+P74JdhdzXE= -golang.org/x/tools v0.29.0/go.mod h1:KMQVMRsVxU6nHCFXrBPhDB8XncLNLM0lIy/F14RP588= +golang.org/x/tools v0.33.0 h1:4qz2S3zmRxbGIhDIAgjxvFutSvH5EfnsYrRBj0UI0bc= +golang.org/x/tools v0.33.0/go.mod h1:CIJMaWEY88juyUfo7UbgPqbC8rU2OqfAV1h2Qp0oMYI= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240930140551-af27646dc61f h1:cUMEy+8oS78BWIH9OWazBkzbr090Od9tWBNtZHkOhf0= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240930140551-af27646dc61f/go.mod h1:UqMtugtsSgubUsoxbuAoiCXvqvErP7Gf0so0mK9tHxU= -google.golang.org/grpc v1.68.0 h1:aHQeeJbo8zAkAa3pRzrVjZlbz6uSfeOXlJNQM0RAbz0= -google.golang.org/grpc v1.68.0/go.mod h1:fmSPC5AsjSBCK54MyHRx48kpOti1/jRfOlwEWywNjWA= -google.golang.org/protobuf v1.35.1 h1:m3LfL6/Ca+fqnjnlqQXNpFPABW1UD7mjh8KO2mKFytA= -google.golang.org/protobuf v1.35.1/go.mod h1:9fA7Ob0pmnwhb644+1+CVWFRbNajQ6iRojtC/QF5bRE= +google.golang.org/genproto/googleapis/rpc v0.0.0-20250219182151-9fdb1cabc7b2 h1:DMTIbak9GhdaSxEjvVzAeNZvyc03I61duqNbnm3SU0M= +google.golang.org/genproto/googleapis/rpc v0.0.0-20250219182151-9fdb1cabc7b2/go.mod h1:LuRYeWDFV6WOn90g357N17oMCaxpgCnbi/44qJvDn2I= +google.golang.org/grpc v1.72.1 h1:HR03wO6eyZ7lknl75XlxABNVLLFc2PAb6mHlYh756mA= +google.golang.org/grpc v1.72.1/go.mod h1:wH5Aktxcg25y1I3w7H69nHfXdOG3UiadoBtjh3izSDM= +google.golang.org/protobuf v1.36.6 h1:z1NpPI8ku2WgiWnf+t9wTPsn6eP1L7ksHUlkfLvd9xY= +google.golang.org/protobuf v1.36.6/go.mod h1:jduwjTPXsFjZGTmRluh+L6NjiWu7pchiJ2/5YcXBHnY= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q= @@ -677,8 +695,8 @@ gopkg.in/yaml.v2 v2.4.0/go.mod h1:RDklbk79AGWmwhnvt/jBztapEOGDOx6ZbXqjP6csGnQ= gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA= gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= -honnef.co/go/tools v0.5.1 h1:4bH5o3b5ZULQ4UrBmP+63W9r7qIkqJClEA9ko5YKx+I= -honnef.co/go/tools v0.5.1/go.mod h1:e9irvo83WDG9/irijV44wr3tbhcFeRnfpVlRqVwpzMs= +honnef.co/go/tools v0.6.1 h1:R094WgE8K4JirYjBaOpz/AvTyUu/3wbmAoskKN/pxTI= +honnef.co/go/tools v0.6.1/go.mod h1:3puzxxljPCe8RGJX7BIy1plGbxEOZni5mR2aXe3/uk4= mvdan.cc/gofumpt v0.7.0 h1:bg91ttqXmi9y2xawvkuMXyvAA/1ZGJqYAEGjXuP0JXU= mvdan.cc/gofumpt v0.7.0/go.mod h1:txVFJy/Sc/mvaycET54pV8SW8gWxTlUuGHVEcncmNUo= mvdan.cc/unparam v0.0.0-20240528143540-8a5130ca722f h1:lMpcwN6GxNbWtbpI1+xzFLSW8XzX0u72NttUGVFjO3U= diff --git a/internal/db_info.go b/internal/db_info.go new file mode 100644 index 0000000..48ac9f4 --- /dev/null +++ b/internal/db_info.go @@ -0,0 +1,109 @@ +// Copyright © 2025 Meroxa, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package internal + +import ( + "context" + "errors" + "fmt" + "time" + + "github.com/jackc/pgx/v5" +) + +type DbInfo struct { + conn *pgx.Conn + cache map[string]*tableCache +} + +type tableCache struct { + columns map[string]int + nextRefresh time.Time +} + +// cacheExpiration defines how long table information remains valid +const cacheExpiration = 5 * time.Minute + +func NewDbInfo(conn *pgx.Conn) *DbInfo { + return &DbInfo{ + conn: conn, + cache: map[string]*tableCache{}, + } +} + +func (d *DbInfo) GetNumericColumnScale(table string, column string) (int, error) { + // Check if table exists in cache and is not expired + tableInfo, ok := d.cache[table] + if ok && time.Now().Before(tableInfo.nextRefresh) { + scale, ok := tableInfo.columns[column] + if ok { + return scale, nil + } + } else { + // Table info has expired, refresh the cache + d.cache[table] = &tableCache{ + columns: map[string]int{}, + nextRefresh: time.Now().Add(cacheExpiration), + } + } + + // Fetch scale from database + scale, err := d.numericScaleFromDb(table, column) + if err != nil { + return 0, err + } + + d.cache[table].columns[column] = scale + + return scale, nil +} + +func (d *DbInfo) numericScaleFromDb(table string, column string) (int, error) { + // Query to get the column type and numeric scale + query := ` + SELECT + data_type, + numeric_scale + FROM + information_schema.columns + WHERE + table_name = $1 + AND column_name = $2 + ` + + var dataType string + var numericScale *int + + // todo use proper ctx + err := d.conn.QueryRow(context.Background(), query, table, column).Scan(&dataType, &numericScale) + if err != nil { + if errors.Is(err, pgx.ErrNoRows) { + return 0, fmt.Errorf("column %s not found in table %s", column, table) + } + return 0, fmt.Errorf("error querying column info: %w", err) + } + + // Check if the column is of the numeric/decimal type + if dataType != "numeric" && dataType != "decimal" { + return 0, fmt.Errorf("column %s in table %s is not a numeric type (actual type: %s)", column, table, dataType) + } + + // Handle case where numeric_scale is NULL + if numericScale == nil { + return 0, nil // The default scale is 0 when not specified + } + + return *numericScale, nil +} diff --git a/tools.go b/internal/utils.go similarity index 68% rename from tools.go rename to internal/utils.go index eef7ac3..5fae481 100644 --- a/tools.go +++ b/internal/utils.go @@ -1,4 +1,4 @@ -// Copyright © 2024 Meroxa, Inc. +// Copyright © 2025 Meroxa, Inc. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -12,14 +12,13 @@ // See the License for the specific language governing permissions and // limitations under the License. -//go:build tools - -package postgres +package internal import ( - _ "github.com/conduitio/conduit-commons/paramgen" - _ "github.com/daixiang0/gci" - _ "github.com/golangci/golangci-lint/cmd/golangci-lint" - _ "golang.org/x/tools/cmd/stringer" - _ "mvdan.cc/gofumpt" + "strconv" ) + +// WrapSQLIdent is used to wrap PostgreSQL identifier under quotes. +// It allows to use uppercase letters and special characters (like `-`) in the +// names of identifiers +var WrapSQLIdent = strconv.Quote diff --git a/internal/utils_test.go b/internal/utils_test.go new file mode 100644 index 0000000..c289561 --- /dev/null +++ b/internal/utils_test.go @@ -0,0 +1,71 @@ +// Copyright © 2025 Meroxa, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package internal + +import ( + "context" + "fmt" + "strings" + "testing" + + "github.com/conduitio/conduit-connector-postgres/test" + "github.com/matryer/is" +) + +func TestSQLIdentWrapping(t *testing.T) { + is := is.New(t) + ctx := test.Context(t) + conn := test.ConnectSimple(ctx, t, test.RegularConnString) + + cases := []struct { + ident string + testName string + expectError bool + }{ + {"just_a_name", "common case", false}, + {"слон", "unicode chars", false}, + {"test table", "spaces", false}, + {"TEST_table", "uppercase letters", false}, + {`'test_table'`, "single quotes", false}, + {"tes`t_table", "apostrophe", false}, + {`te"st_table`, "double quotes", true}, + } + + for _, c := range cases { + t.Run(c.testName, func(t *testing.T) { + w := WrapSQLIdent(c.ident) + + t.Cleanup(func() { + if c.expectError { + return + } + + query := fmt.Sprintf("DROP TABLE %s", w) + _, err := conn.Exec(context.Background(), query) + is.NoErr(err) + }) + + query := fmt.Sprintf("CREATE TABLE %s (%s int)", w, w) + _, err := conn.Exec(context.Background(), query) + + if c.expectError { + is.True(err != nil) + is.True(strings.Contains(err.Error(), `(SQLSTATE 42601)`)) // syntax error + } else { + is.NoErr(err) + } + }) + } +} diff --git a/source.go b/source.go index aa82f69..625e8e4 100644 --- a/source.go +++ b/source.go @@ -24,6 +24,7 @@ import ( "github.com/conduitio/conduit-commons/csync" "github.com/conduitio/conduit-commons/lang" "github.com/conduitio/conduit-commons/opencdc" + "github.com/conduitio/conduit-connector-postgres/internal" "github.com/conduitio/conduit-connector-postgres/source" "github.com/conduitio/conduit-connector-postgres/source/cpool" "github.com/conduitio/conduit-connector-postgres/source/logrepl" @@ -42,36 +43,27 @@ type Source struct { tableKeys map[string]string } +func (s *Source) Config() sdk.SourceConfig { + return &s.config +} + func NewSource() sdk.Source { return sdk.SourceWithMiddleware( &Source{ tableKeys: make(map[string]string), - }, - sdk.DefaultSourceMiddleware( - // disable schema extraction by default, postgres will build its own schema - sdk.SourceWithSchemaExtractionConfig{ - PayloadEnabled: lang.Ptr(false), - KeyEnabled: lang.Ptr(false), + config: source.Config{ + DefaultSourceMiddleware: sdk.DefaultSourceMiddleware{ + // disable schema extraction by default, postgres will build its own schema + SourceWithSchemaExtraction: sdk.SourceWithSchemaExtraction{ + PayloadEnabled: lang.Ptr(false), + KeyEnabled: lang.Ptr(false), + }, + }, }, - )..., + }, ) } -func (s *Source) Parameters() config.Parameters { - return s.config.Parameters() -} - -func (s *Source) Configure(ctx context.Context, cfg config.Config) error { - err := sdk.Util.ParseConfig(ctx, cfg, &s.config, NewSource().Parameters()) - if err != nil { - return err - } - - s.config = s.config.Init() - - return s.config.Validate() -} - func (s *Source) Open(ctx context.Context, pos opencdc.Position) error { pool, err := cpool.New(ctx, s.config.URL) if err != nil { @@ -127,8 +119,8 @@ func (s *Source) Open(ctx context.Context, pos opencdc.Position) error { return nil } -func (s *Source) Read(ctx context.Context) (opencdc.Record, error) { - return s.iterator.Next(ctx) +func (s *Source) ReadN(ctx context.Context, n int) ([]opencdc.Record, error) { + return s.iterator.NextN(ctx, n) } func (s *Source) Ack(ctx context.Context, pos opencdc.Position) error { @@ -157,42 +149,28 @@ func (s *Source) Teardown(ctx context.Context) error { } func (s *Source) LifecycleOnDeleted(ctx context.Context, cfg config.Config) error { - if err := s.Configure(ctx, cfg); err != nil { - return fmt.Errorf("fail to handle lifecycle delete event: %w", err) - } - - // N.B. This should not stay in here for long, enrich the default. - // Events are not passed enriched config with defaults. - params := s.config.Parameters() - - if _, ok := cfg["logrepl.autoCleanup"]; !ok { // not set - s.config.LogreplAutoCleanup = params["logrepl.autoCleanup"].Default == "true" - } - - if _, ok := cfg["logrepl.slotName"]; !ok { - s.config.LogreplSlotName = params["logrepl.slotName"].Default - } - - if _, ok := cfg["logrepl.publicationName"]; !ok { - s.config.LogreplPublicationName = params["logrepl.publicationName"].Default + var oldConfig source.Config + err := sdk.Util.ParseConfig(ctx, cfg, &oldConfig, Connector.NewSpecification().SourceParams) + if err != nil { + return fmt.Errorf("lifecycle delete event: failed to parse configuration: %w", err) } - switch s.config.CDCMode { + switch oldConfig.CDCMode { case source.CDCModeAuto: fallthrough // TODO: Adjust as `auto` changes. case source.CDCModeLogrepl: - if !s.config.LogreplAutoCleanup { + if !oldConfig.LogreplAutoCleanup { sdk.Logger(ctx).Warn().Msg("Skipping logrepl auto cleanup") return nil } return logrepl.Cleanup(ctx, logrepl.CleanupConfig{ - URL: s.config.URL, - SlotName: s.config.LogreplSlotName, - PublicationName: s.config.LogreplPublicationName, + URL: oldConfig.URL, + SlotName: oldConfig.LogreplSlotName, + PublicationName: oldConfig.LogreplPublicationName, }) default: - sdk.Logger(ctx).Warn().Msgf("cannot handle CDC mode %q", s.config.CDCMode) + sdk.Logger(ctx).Warn().Msgf("cannot handle CDC mode %q", oldConfig.CDCMode) return nil } } @@ -231,7 +209,7 @@ func (s *Source) getPrimaryKey(ctx context.Context, tableName string) (string, e JOIN pg_attribute a ON a.attrelid = i.indrelid AND a.attnum = ANY(i.indkey) WHERE i.indrelid = $1::regclass AND i.indisprimary` - rows, err := s.pool.Query(ctx, query, tableName) + rows, err := s.pool.Query(ctx, query, internal.WrapSQLIdent(tableName)) if err != nil { return "", fmt.Errorf("failed to query table keys: %w", err) } diff --git a/source/config.go b/source/config.go index b2e7165..4d71cf9 100644 --- a/source/config.go +++ b/source/config.go @@ -12,15 +12,14 @@ // See the License for the specific language governing permissions and // limitations under the License. -//go:generate paramgen Config - package source import ( + "context" "errors" "fmt" - "github.com/conduitio/conduit-commons/config" + sdk "github.com/conduitio/conduit-connector-sdk" "github.com/jackc/pgx/v5" ) @@ -47,14 +46,14 @@ const ( ) type Config struct { + sdk.DefaultSourceMiddleware + // URL is the connection string for the Postgres database. URL string `json:"url" validate:"required"` // Tables is a List of table names to read from, separated by a comma, e.g.:"table1,table2". // Use "*" if you'd like to listen to all tables. - Tables []string `json:"tables"` - // Deprecated: use `tables` instead. - Table []string `json:"table"` + Tables []string `json:"tables" validate:"required"` // SnapshotMode is whether the plugin will take a snapshot of the entire table before starting cdc mode. SnapshotMode SnapshotMode `json:"snapshotMode" validate:"inclusion=initial|never" default:"initial"` @@ -70,7 +69,8 @@ type Config struct { LogreplPublicationName string `json:"logrepl.publicationName" default:"conduitpub"` // LogreplSlotName determines the replication slot name in case the // connector uses logical replication to listen to changes (see CDCMode). - LogreplSlotName string `json:"logrepl.slotName" default:"conduitslot"` + // Can only contain lower-case letters, numbers, and the underscore character. + LogreplSlotName string `json:"logrepl.slotName" validate:"regex=^[a-z0-9_]+$" default:"conduitslot"` // LogreplAutoCleanup determines if the replication slot and publication should be // removed when the connector is deleted. @@ -78,34 +78,20 @@ type Config struct { // WithAvroSchema determines whether the connector should attach an avro schema on each // record. - WithAvroSchema bool `json:"logrepl.withAvroSchema" default:"false"` + WithAvroSchema bool `json:"logrepl.withAvroSchema" default:"true"` } // Validate validates the provided config values. -func (c Config) Validate() error { +func (c *Config) Validate(ctx context.Context) error { var errs []error - - // try parsing the url - _, err := pgx.ParseConfig(c.URL) - if err != nil { + if _, err := pgx.ParseConfig(c.URL); err != nil { errs = append(errs, fmt.Errorf("invalid url: %w", err)) } - if len(c.Tables) > 0 && len(c.Table) > 0 { - errs = append(errs, fmt.Errorf(`error validating "tables": cannot provide both "table" and "tables", use "tables" only`)) + err := c.DefaultSourceMiddleware.Validate(ctx) + if err != nil { + errs = append(errs, err) } - if len(c.Tables) == 0 { - errs = append(errs, fmt.Errorf(`error validating "tables": %w`, config.ErrRequiredParameterMissing)) - } return errors.Join(errs...) } - -// Init sets the desired value on Tables while Table is being deprecated. -func (c Config) Init() Config { - if len(c.Table) > 0 && len(c.Tables) == 0 { - c.Tables = c.Table - c.Table = nil - } - return c -} diff --git a/source/config_test.go b/source/config_test.go index 7e9ae3b..7d7e313 100644 --- a/source/config_test.go +++ b/source/config_test.go @@ -15,6 +15,7 @@ package source import ( + "context" "testing" "github.com/matryer/is" @@ -47,7 +48,7 @@ func TestConfig_Validate(t *testing.T) { for _, tc := range testCases { t.Run(tc.name, func(t *testing.T) { is := is.New(t) - err := tc.cfg.Validate() + err := tc.cfg.Validate(context.Background()) if tc.wantErr { is.True(err != nil) return diff --git a/source/iterator.go b/source/iterator.go index 2511807..12a31bd 100644 --- a/source/iterator.go +++ b/source/iterator.go @@ -23,9 +23,9 @@ import ( // Iterator is an object that can iterate over a queue of records. type Iterator interface { - // Next takes and returns the next record from the queue. Next is allowed to - // block until either a record is available or the context gets canceled. - Next(context.Context) (opencdc.Record, error) + // NextN takes and returns up to n records from the queue. NextN is allowed to + // block until either at least one record is available or the context gets canceled. + NextN(context.Context, int) ([]opencdc.Record, error) // Ack signals that a record at a specific position was successfully // processed. Ack(context.Context, opencdc.Position) error diff --git a/source/logrepl/cdc.go b/source/logrepl/cdc.go index d39871f..4f05b02 100644 --- a/source/logrepl/cdc.go +++ b/source/logrepl/cdc.go @@ -39,7 +39,7 @@ type CDCConfig struct { } // CDCIterator asynchronously listens for events from the logical replication -// slot and returns them to the caller through Next. +// slot and returns them to the caller through NextN. type CDCIterator struct { config CDCConfig records chan opencdc.Record @@ -120,35 +120,62 @@ func (i *CDCIterator) StartSubscriber(ctx context.Context) error { return nil } -// Next returns the next record retrieved from the subscription. This call will -// block until either a record is returned from the subscription, the -// subscription stops because of an error or the context gets canceled. -// Returns error when the subscription has been started. -func (i *CDCIterator) Next(ctx context.Context) (opencdc.Record, error) { +// NextN takes and returns up to n records from the queue. NextN is allowed to +// block until either at least one record is available or the context gets canceled. +func (i *CDCIterator) NextN(ctx context.Context, n int) ([]opencdc.Record, error) { if !i.subscriberReady() { - return opencdc.Record{}, errors.New("logical replication has not been started") + return nil, errors.New("logical replication has not been started") } - for { + if n <= 0 { + return nil, fmt.Errorf("n must be greater than 0, got %d", n) + } + + var recs []opencdc.Record + + // Block until at least one record is received or context is canceled + select { + case <-ctx.Done(): + return nil, ctx.Err() + case <-i.sub.Done(): + if err := i.sub.Err(); err != nil { + return nil, fmt.Errorf("logical replication error: %w", err) + } + if err := ctx.Err(); err != nil { + // subscription is done because the context is canceled, we went + // into the wrong case by chance + return nil, err + } + // subscription stopped without an error and the context is still + // open, this is a strange case, shouldn't actually happen + return nil, fmt.Errorf("subscription stopped, no more data to fetch (this smells like a bug)") + case rec := <-i.records: + recs = append(recs, rec) + } + + for len(recs) < n { select { + case rec := <-i.records: + recs = append(recs, rec) case <-ctx.Done(): - return opencdc.Record{}, ctx.Err() + return nil, ctx.Err() case <-i.sub.Done(): if err := i.sub.Err(); err != nil { - return opencdc.Record{}, fmt.Errorf("logical replication error: %w", err) + return recs, fmt.Errorf("logical replication error: %w", err) } if err := ctx.Err(); err != nil { - // subscription is done because the context is canceled, we went - // into the wrong case by chance - return opencdc.Record{}, err + // Return what we have with context error + return recs, err } - // subscription stopped without an error and the context is still - // open, this is a strange case, shouldn't actually happen - return opencdc.Record{}, fmt.Errorf("subscription stopped, no more data to fetch (this smells like a bug)") - case r := <-i.records: - return r, nil + // Return what we have with subscription stopped error + return recs, fmt.Errorf("subscription stopped, no more data to fetch (this smells like a bug)") + default: + // No more records currently available + return recs, nil } } + + return recs, nil } // Ack forwards the acknowledgment to the subscription. diff --git a/source/logrepl/cdc_test.go b/source/logrepl/cdc_test.go index 7e30c67..ff57442 100644 --- a/source/logrepl/cdc_test.go +++ b/source/logrepl/cdc_test.go @@ -18,6 +18,7 @@ import ( "context" "errors" "fmt" + "math/big" "strings" "testing" "time" @@ -119,7 +120,7 @@ func TestCDCIterator_New(t *testing.T) { } } -func TestCDCIterator_Next(t *testing.T) { +func TestCDCIterator_Operation_NextN(t *testing.T) { ctx := test.Context(t) is := is.New(t) @@ -159,15 +160,16 @@ func TestCDCIterator_Next(t *testing.T) { Payload: opencdc.Change{ Before: nil, After: opencdc.StructuredData{ - "id": int64(6), - "column1": "bizz", - "column2": int32(456), - "column3": false, - "column4": 12.3, - "column5": int64(14), - "column6": []byte(`{"foo2": "bar2"}`), - "column7": []byte(`{"foo2": "baz2"}`), - "key": nil, + "id": int64(6), + "column1": "bizz", + "column2": int32(456), + "column3": false, + "column4": big.NewRat(123, 10), + "column5": big.NewRat(14, 1), + "column6": []byte(`{"foo2": "bar2"}`), + "column7": []byte(`{"foo2": "baz2"}`), + "key": nil, + "UppercaseColumn1": nil, }, }, }, @@ -193,15 +195,16 @@ func TestCDCIterator_Next(t *testing.T) { Key: opencdc.StructuredData{"id": int64(1)}, Payload: opencdc.Change{ After: opencdc.StructuredData{ - "id": int64(1), - "column1": "test cdc updates", - "column2": int32(123), - "column3": false, - "column4": 12.2, - "column5": int64(4), - "column6": []byte(`{"foo": "bar"}`), - "column7": []byte(`{"foo": "baz"}`), - "key": []uint8("1"), + "id": int64(1), + "column1": "test cdc updates", + "column2": int32(123), + "column3": false, + "column4": big.NewRat(122, 10), + "column5": big.NewRat(4, 1), + "column6": []byte(`{"foo": "bar"}`), + "column7": []byte(`{"foo": "baz"}`), + "key": []uint8("1"), + "UppercaseColumn1": int32(1), }, }, }, @@ -229,26 +232,28 @@ func TestCDCIterator_Next(t *testing.T) { Key: opencdc.StructuredData{"id": int64(1)}, Payload: opencdc.Change{ Before: opencdc.StructuredData{ - "id": int64(1), - "column1": "test cdc updates", - "column2": int32(123), - "column3": false, - "column4": 12.2, - "column5": int64(4), - "column6": []byte(`{"foo": "bar"}`), - "column7": []byte(`{"foo": "baz"}`), - "key": []uint8("1"), + "id": int64(1), + "column1": "test cdc updates", + "column2": int32(123), + "column3": false, + "column4": big.NewRat(122, 10), + "column5": big.NewRat(4, 1), + "column6": []byte(`{"foo": "bar"}`), + "column7": []byte(`{"foo": "baz"}`), + "key": []uint8("1"), + "UppercaseColumn1": int32(1), }, After: opencdc.StructuredData{ - "id": int64(1), - "column1": "test cdc full updates", - "column2": int32(123), - "column3": false, - "column4": 12.2, - "column5": int64(4), - "column6": []byte(`{"foo": "bar"}`), - "column7": []byte(`{"foo": "baz"}`), - "key": []uint8("1"), + "id": int64(1), + "column1": "test cdc full updates", + "column2": int32(123), + "column3": false, + "column4": big.NewRat(122, 10), + "column5": big.NewRat(4, 1), + "column6": []byte(`{"foo": "bar"}`), + "column7": []byte(`{"foo": "baz"}`), + "key": []uint8("1"), + "UppercaseColumn1": int32(1), }, }, }, @@ -276,15 +281,16 @@ func TestCDCIterator_Next(t *testing.T) { Key: opencdc.StructuredData{"id": int64(4)}, Payload: opencdc.Change{ Before: opencdc.StructuredData{ - "id": int64(4), - "column1": nil, - "column2": nil, - "column3": nil, - "column4": nil, - "column5": nil, - "column6": nil, - "column7": nil, - "key": nil, + "id": int64(4), + "column1": nil, + "column2": nil, + "column3": nil, + "column4": nil, + "column5": nil, + "column6": nil, + "column7": nil, + "key": nil, + "UppercaseColumn1": nil, }, }, }, @@ -312,15 +318,16 @@ func TestCDCIterator_Next(t *testing.T) { Key: opencdc.StructuredData{"id": int64(3)}, Payload: opencdc.Change{ Before: opencdc.StructuredData{ - "id": int64(3), - "key": []uint8("3"), - "column1": "baz", - "column2": int32(789), - "column3": false, - "column4": nil, - "column5": int64(9), - "column6": []byte(`{"foo": "bar"}`), - "column7": []byte(`{"foo": "baz"}`), + "id": int64(3), + "key": []uint8("3"), + "column1": "baz", + "column2": int32(789), + "column3": false, + "column4": nil, + "column5": big.NewRat(9, 1), + "column6": []byte(`{"foo": "bar"}`), + "column7": []byte(`{"foo": "baz"}`), + "UppercaseColumn1": int32(3), }, }, }, @@ -337,9 +344,11 @@ func TestCDCIterator_Next(t *testing.T) { // fetch the change nextCtx, cancel := context.WithTimeout(ctx, time.Second*10) defer cancel() - got, err := i.Next(nextCtx) + records, err := i.NextN(nextCtx, 1) is.NoErr(err) + got := records[0] + readAt, err := got.Metadata.GetReadAt() is.NoErr(err) is.True(readAt.After(now)) // ReadAt should be after now @@ -347,46 +356,19 @@ func TestCDCIterator_Next(t *testing.T) { tt.want.Metadata[opencdc.MetadataReadAt] = got.Metadata[opencdc.MetadataReadAt] tt.want.Position = got.Position - is.Equal("", cmp.Diff(tt.want, got, cmpopts.IgnoreUnexported(opencdc.Record{}))) + is.Equal("", cmp.Diff( + tt.want, + got, + cmpopts.IgnoreUnexported(opencdc.Record{}), + cmp.Comparer(func(x, y *big.Rat) bool { + return x.Cmp(y) == 0 + }), + )) is.NoErr(i.Ack(ctx, got.Position)) }) } } -func TestCDCIterator_Next_Fail(t *testing.T) { - ctx := test.Context(t) - - pool := test.ConnectPool(ctx, t, test.RepmgrConnString) - table := test.SetupTestTable(ctx, t, pool) - - t.Run("fail when sub is done", func(t *testing.T) { - is := is.New(t) - - i := testCDCIterator(ctx, t, pool, table, true) - <-i.sub.Ready() - - is.NoErr(i.Teardown(ctx)) - - _, err := i.Next(ctx) - expectErr := "logical replication error:" - - match := strings.Contains(err.Error(), expectErr) - if !match { - t.Logf("%s != %s", err.Error(), expectErr) - } - is.True(match) - }) - - t.Run("fail when subscriber is not started", func(t *testing.T) { - is := is.New(t) - - i := testCDCIterator(ctx, t, pool, table, false) - - _, nexterr := i.Next(ctx) - is.Equal(nexterr.Error(), "logical replication has not been started") - }) -} - func TestCDCIterator_EnsureLSN(t *testing.T) { ctx := test.Context(t) is := is.New(t) @@ -401,8 +383,11 @@ func TestCDCIterator_EnsureLSN(t *testing.T) { VALUES (6, 'bizz', 456, false, 12.3, 14)`, table)) is.NoErr(err) - r, err := i.Next(ctx) + rr, err := i.NextN(ctx, 1) is.NoErr(err) + is.True(len(rr) > 0) + + r := rr[0] p, err := position.ParseSDKPosition(r.Position) is.NoErr(err) @@ -479,6 +464,138 @@ func TestCDCIterator_Ack(t *testing.T) { }) } } +func TestCDCIterator_NextN(t *testing.T) { + ctx := test.Context(t) + pool := test.ConnectPool(ctx, t, test.RepmgrConnString) + table := test.SetupTestTable(ctx, t, pool) + + t.Run("retrieve exact N records", func(t *testing.T) { + is := is.New(t) + i := testCDCIterator(ctx, t, pool, table, true) + <-i.sub.Ready() + + for j := 1; j <= 3; j++ { + _, err := pool.Exec(ctx, fmt.Sprintf(`INSERT INTO %s (id, column1, column2, column3, column4, column5) + VALUES (%d, 'test-%d', %d, false, 12.3, 14)`, table, j+10, j, j*100)) + is.NoErr(err) + } + + var allRecords []opencdc.Record + attemptCtx, cancel := context.WithTimeout(ctx, 5*time.Second) + defer cancel() + + // Collect records until we have all 3 + for len(allRecords) < 3 { + records, err := i.NextN(attemptCtx, 3-len(allRecords)) + is.NoErr(err) + // Only proceed if we got at least one record + is.True(len(records) > 0) + allRecords = append(allRecords, records...) + } + + is.Equal(len(allRecords), 3) + + for j, r := range allRecords { + is.Equal(r.Operation, opencdc.OperationCreate) + is.Equal(r.Key.(opencdc.StructuredData)["id"], int64(j+11)) + change := r.Payload + data := change.After.(opencdc.StructuredData) + is.Equal(data["column1"], fmt.Sprintf("test-%d", j+1)) + //nolint:gosec // no risk to overflow + is.Equal(data["column2"], (int32(j)+1)*100) + } + }) + + t.Run("retrieve fewer records than requested", func(t *testing.T) { + is := is.New(t) + i := testCDCIterator(ctx, t, pool, table, true) + <-i.sub.Ready() + + for j := 1; j <= 2; j++ { + _, err := pool.Exec(ctx, fmt.Sprintf(`INSERT INTO %s (id, column1, column2, column3, column4, column5) + VALUES (%d, 'test-%d', %d, false, 12.3, 14)`, table, j+20, j, j*100)) + is.NoErr(err) + } + + // Will keep calling NextN until all records are received + records := make([]opencdc.Record, 0, 2) + for len(records) < 2 { + recordsTmp, err := i.NextN(ctx, 5) + is.NoErr(err) + records = append(records, recordsTmp...) + } + + // nothing else to fetch + ctxWithTimeout, cancel := context.WithTimeout(ctx, 500*time.Millisecond) + defer cancel() + _, err := i.NextN(ctxWithTimeout, 5) + is.True(errors.Is(err, context.DeadlineExceeded)) + + for j, r := range records { + is.Equal(r.Operation, opencdc.OperationCreate) + is.Equal(r.Key.(opencdc.StructuredData)["id"], int64(j+21)) + change := r.Payload + data := change.After.(opencdc.StructuredData) + is.Equal(data["column1"], fmt.Sprintf("test-%d", j+1)) + //nolint:gosec // no risk to overflow + is.Equal(data["column2"], (int32(j)+1)*100) + } + }) + + t.Run("context cancellation", func(t *testing.T) { + is := is.New(t) + i := testCDCIterator(ctx, t, pool, table, true) + <-i.sub.Ready() + + ctxTimeout, cancel := context.WithTimeout(ctx, 100*time.Millisecond) + defer cancel() + + _, err := i.NextN(ctxTimeout, 5) + is.True(errors.Is(err, context.DeadlineExceeded)) + }) + + t.Run("subscriber not started", func(t *testing.T) { + is := is.New(t) + i := testCDCIterator(ctx, t, pool, table, false) + + _, err := i.NextN(ctx, 5) + is.Equal(err.Error(), "logical replication has not been started") + }) + + t.Run("invalid N values", func(t *testing.T) { + is := is.New(t) + i := testCDCIterator(ctx, t, pool, table, true) + <-i.sub.Ready() + + _, err := i.NextN(ctx, 0) + is.True(strings.Contains(err.Error(), "n must be greater than 0")) + + _, err = i.NextN(ctx, -1) + is.True(strings.Contains(err.Error(), "n must be greater than 0")) + }) + + t.Run("subscription termination", func(t *testing.T) { + is := is.New(t) + i := testCDCIterator(ctx, t, pool, table, true) + <-i.sub.Ready() + + _, err := pool.Exec(ctx, fmt.Sprintf(`INSERT INTO %s (id, column1, column2, column3, column4, column5) + VALUES (30, 'test-1', 100, false, 12.3, 14)`, table)) + is.NoErr(err) + + go func() { + time.Sleep(100 * time.Millisecond) + is.NoErr(i.Teardown(ctx)) + }() + + records, err := i.NextN(ctx, 5) + if err != nil { + is.True(strings.Contains(err.Error(), "logical replication error")) + } else { + is.True(len(records) > 0) + } + }) +} func testCDCIterator(ctx context.Context, t *testing.T, pool *pgxpool.Pool, table string, start bool) *CDCIterator { is := is.New(t) @@ -554,8 +671,11 @@ func TestCDCIterator_Schema(t *testing.T) { ) is.NoErr(err) - r, err := i.Next(ctx) + rr, err := i.NextN(ctx, 1) is.NoErr(err) + is.True(len(rr) > 0) + + r := rr[0] assertPayloadSchemaOK(ctx, is, test.TestTableAvroSchemaV1, table, r) assertKeySchemaOK(ctx, is, table, r) @@ -574,8 +694,11 @@ func TestCDCIterator_Schema(t *testing.T) { ) is.NoErr(err) - r, err := i.Next(ctx) + rr, err := i.NextN(ctx, 1) is.NoErr(err) + is.True(len(rr) > 0) + + r := rr[0] assertPayloadSchemaOK(ctx, is, test.TestTableAvroSchemaV2, table, r) assertKeySchemaOK(ctx, is, table, r) @@ -594,8 +717,11 @@ func TestCDCIterator_Schema(t *testing.T) { ) is.NoErr(err) - r, err := i.Next(ctx) + rr, err := i.NextN(ctx, 1) is.NoErr(err) + is.True(len(rr) > 0) + + r := rr[0] assertPayloadSchemaOK(ctx, is, test.TestTableAvroSchemaV3, table, r) assertKeySchemaOK(ctx, is, table, r) diff --git a/source/logrepl/combined.go b/source/logrepl/combined.go index 71af64f..7ed853d 100644 --- a/source/logrepl/combined.go +++ b/source/logrepl/combined.go @@ -27,7 +27,7 @@ import ( ) type iterator interface { - Next(context.Context) (opencdc.Record, error) + NextN(context.Context, int) ([]opencdc.Record, error) Ack(context.Context, opencdc.Position) error Teardown(context.Context) error } @@ -111,27 +111,29 @@ func NewCombinedIterator(ctx context.Context, pool *pgxpool.Pool, conf Config) ( return c, nil } -// Next provides the next available record from the snapshot or CDC stream. -// If the end of the snapshot is reached, next will switch to the CDC iterator and retrive -// the next available record. Failure to switch the iterator will return an error. -func (c *CombinedIterator) Next(ctx context.Context) (opencdc.Record, error) { - r, err := c.activeIterator.Next(ctx) +// NextN retrieves up to n records from the active iterator. +// If the end of the snapshot is reached during this call, it will switch to the CDC iterator +// and continue retrieving records from there. +func (c *CombinedIterator) NextN(ctx context.Context, n int) ([]opencdc.Record, error) { + if n <= 0 { + return nil, fmt.Errorf("n must be greater than 0, got %d", n) + } + + records, err := c.activeIterator.NextN(ctx, n) if err != nil { - // Snapshot iterator is done, handover to CDC iterator if !errors.Is(err, snapshot.ErrIteratorDone) { - return opencdc.Record{}, fmt.Errorf("failed to fetch next record: %w", err) + return nil, fmt.Errorf("failed to fetch records in batch: %w", err) } + // Snapshot iterator is done, handover to CDC iterator if err := c.useCDCIterator(ctx); err != nil { - return opencdc.Record{}, err + return nil, err } - sdk.Logger(ctx).Debug().Msg("Snapshot completed, switching to CDC mode") - // retry with new iterator - return c.activeIterator.Next(ctx) + sdk.Logger(ctx).Debug().Msg("Snapshot completed, switching to CDC mode") + return c.NextN(ctx, n) } - - return r, nil + return records, nil } func (c *CombinedIterator) Ack(ctx context.Context, p opencdc.Position) error { diff --git a/source/logrepl/combined_test.go b/source/logrepl/combined_test.go index 7f54c70..d48976d 100644 --- a/source/logrepl/combined_test.go +++ b/source/logrepl/combined_test.go @@ -18,6 +18,7 @@ import ( "context" "errors" "fmt" + "math/big" "testing" "time" @@ -136,9 +137,9 @@ func TestCombinedIterator_New(t *testing.T) { }) } -func TestCombinedIterator_Next(t *testing.T) { +func TestCombinedIterator_NextN(t *testing.T) { ctx := test.Context(t) - ctx, cancel := context.WithTimeout(ctx, time.Second*30) + ctx, cancel := context.WithTimeout(ctx, time.Second*120) defer cancel() is := is.New(t) @@ -155,6 +156,7 @@ func TestCombinedIterator_Next(t *testing.T) { }) is.NoErr(err) + // Add a record to the table for CDC mode testing _, err = pool.Exec(ctx, fmt.Sprintf( `INSERT INTO %s (id, column1, column2, column3, column4, column5, column6, column7) VALUES (6, 'bizz', 1010, false, 872.2, 101, '{"foo12": "bar12"}', '{"foo13": "bar13"}')`, @@ -163,75 +165,116 @@ func TestCombinedIterator_Next(t *testing.T) { is.NoErr(err) var lastPos opencdc.Position - expectedRecords := testRecords() - // compare snapshot - for id := 1; id < 5; id++ { - t.Run(fmt.Sprint("next_snapshot", id), func(t *testing.T) { - is := is.New(t) - r, err := i.Next(ctx) - is.NoErr(err) + t.Run("invalid_n_value", func(t *testing.T) { + is := is.New(t) + _, err := i.NextN(ctx, 0) + is.True(err != nil) + is.True(err.Error() == "n must be greater than 0, got 0") - jsonPos := fmt.Sprintf(`{"type":1,"snapshots":{"%s":{"last_read":%d,"snapshot_end":4}}}`, table, id) - is.Equal(string(r.Position), jsonPos) + _, err = i.NextN(ctx, -1) + is.True(err != nil) + is.True(err.Error() == "n must be greater than 0, got -1") + }) - is.Equal("", cmp.Diff( - expectedRecords[id], - r.Payload.After.(opencdc.StructuredData), - )) + t.Run("nextN_snapshot_batch", func(t *testing.T) { + is := is.New(t) - is.NoErr(i.Ack(ctx, r.Position)) - }) - } + // Request 3 records in batch (snapshot mode) + records, err := i.NextN(ctx, 3) + is.NoErr(err) + is.True(len(records) > 0) - // interrupt repl connection - var terminated bool - is.NoErr(pool.QueryRow(ctx, fmt.Sprintf( - `SELECT pg_terminate_backend(pid) FROM pg_stat_activity WHERE - query ILIKE '%%CREATE_REPLICATION_SLOT %s%%' and pid <> pg_backend_pid() - `, - table, - )).Scan(&terminated)) - is.True(terminated) + for _, r := range records { + pos, err := position.ParseSDKPosition(r.Position) + is.NoErr(err) + is.Equal(pos.Type, position.TypeSnapshot) - t.Run("next_cdc_5", func(t *testing.T) { - is := is.New(t) + // check it's a valid record with an id + data := r.Payload.After.(opencdc.StructuredData) + _, hasID := data["id"] + is.True(hasID) - r, err := i.Next(ctx) - is.NoErr(err) + is.NoErr(i.Ack(ctx, r.Position)) + } + }) - pos, err := position.ParseSDKPosition(r.Position) - is.NoErr(err) - is.Equal(pos.Type, position.TypeCDC) + t.Run("nextN_snapshot_to_cdc_transition", func(t *testing.T) { + is := is.New(t) - lsn, err := pos.LSN() - is.NoErr(err) - is.True(lsn != 0) + transitionComplete := false + retryCount := 0 + maxRetries := 10 - is.Equal("", cmp.Diff( - expectedRecords[5], - r.Payload.After.(opencdc.StructuredData), - )) + for retryCount < maxRetries && !transitionComplete { + // Request more records - we might get remaining snapshot records + records, err := i.NextN(ctx, 2) + is.NoErr(err) - is.NoErr(i.Ack(ctx, r.Position)) - lastPos = r.Position - }) + if len(records) == 0 { + retryCount++ + continue + } + + for _, r := range records { + pos, err := position.ParseSDKPosition(r.Position) + is.NoErr(err) + + if pos.Type == position.TypeCDC { + lsn, err := pos.LSN() + is.NoErr(err) + is.True(lsn != 0) + + // Store position for next test + lastPos = r.Position + transitionComplete = true + } + + is.NoErr(i.Ack(ctx, r.Position)) + } + + retryCount++ + } + + is.True(transitionComplete) + if !transitionComplete { + t.Fatalf("Failed to transition from snapshot to CDC mode") + } + // interrupt repl connection - handle case when connection might already be closed + var terminated bool + err = pool.QueryRow(ctx, fmt.Sprintf( + `SELECT pg_terminate_backend(pid) FROM pg_stat_activity WHERE + query ILIKE '%%CREATE_REPLICATION_SLOT %s%%' and pid <> pg_backend_pid() + `, + table, + )).Scan(&terminated) - is.NoErr(i.Teardown(ctx)) + is.NoErr(i.Teardown(ctx)) + }) - t.Run("next_connector_resume_cdc_6", func(t *testing.T) { + t.Run("nextN_connector_resume_cdc", func(t *testing.T) { is := is.New(t) + + pos, err := position.ParseSDKPosition(lastPos) + is.NoErr(err) + is.Equal(pos.Type, position.TypeCDC) + i, err := NewCombinedIterator(ctx, pool, Config{ Position: lastPos, Tables: []string{table}, TableKeys: map[string]string{table: "id"}, PublicationName: table, SlotName: table, - WithSnapshot: true, + WithSnapshot: false, }) is.NoErr(err) + // Verify we're in CDC mode + cdcMode := i.activeIterator == i.cdcIterator + is.True(cdcMode) + + // Insert two more records for testing CDC batch _, err = pool.Exec(ctx, fmt.Sprintf( `INSERT INTO %s (id, column1, column2, column3, column4, column5, column6, column7) VALUES (7, 'buzz', 10101, true, 121.9, 51, '{"foo7": "bar7"}', '{"foo8": "bar8"}')`, @@ -239,23 +282,73 @@ func TestCombinedIterator_Next(t *testing.T) { )) is.NoErr(err) - r, err := i.Next(ctx) + _, err = pool.Exec(ctx, fmt.Sprintf( + `INSERT INTO %s (id, column1, column2, column3, column4, column5, column6, column7) + VALUES (8, 'fizz', 20202, false, 232.8, 62, '{"foo9": "bar9"}', '{"foo10": "bar10"}')`, + table, + )) is.NoErr(err) - pos, err := position.ParseSDKPosition(r.Position) - is.NoErr(err) - is.Equal(pos.Type, position.TypeCDC) + // Request 2 records in CDC mode + records := make([]opencdc.Record, 0, 2) + var retries int + maxRetries := 10 + for retries < maxRetries { + records, err = i.NextN(ctx, 2) + is.NoErr(err) + + if len(records) > 0 { + t.Logf("Got %d records after %d retries", len(records), retries) + break + } + + t.Logf("No CDC records returned, retry %d/%d", retries+1, maxRetries) + retries++ + } + + is.True(len(records) > 0) + + if len(records) > 0 { + pos, err := position.ParseSDKPosition(records[0].Position) + is.NoErr(err) + is.Equal(pos.Type, position.TypeCDC) + + lsn, err := pos.LSN() + is.NoErr(err) + is.True(lsn != 0) - lsn, err := pos.LSN() + is.Equal("", cmp.Diff( + expectedRecords[6], + records[0].Payload.After.(opencdc.StructuredData), + cmp.Comparer(func(x, y *big.Rat) bool { + return x.Cmp(y) == 0 + }), + )) + + is.NoErr(i.Ack(ctx, records[0].Position)) + } + is.NoErr(i.Teardown(ctx)) + }) + t.Run("nextN_context_cancellation", func(t *testing.T) { + is := is.New(t) + + i, err := NewCombinedIterator(ctx, pool, Config{ + Position: opencdc.Position{}, + Tables: []string{table}, + TableKeys: map[string]string{table: "id"}, + PublicationName: table, + SlotName: table, + WithSnapshot: true, + }) is.NoErr(err) - is.True(lsn != 0) - is.Equal("", cmp.Diff( - expectedRecords[6], - r.Payload.After.(opencdc.StructuredData), - )) + cancelCtx, cancelFn := context.WithCancel(ctx) + cancelFn() + + // Request should fail with context canceled + _, err = i.NextN(cancelCtx, 2) + is.True(errors.Is(err, context.Canceled)) - is.NoErr(i.Ack(ctx, r.Position)) is.NoErr(i.Teardown(ctx)) }) @@ -270,70 +363,76 @@ func testRecords() []opencdc.StructuredData { return []opencdc.StructuredData{ {}, { - "id": int64(1), - "key": []uint8("1"), - "column1": "foo", - "column2": int32(123), - "column3": false, - "column4": 12.2, - "column5": int64(4), - "column6": []byte(`{"foo": "bar"}`), - "column7": []byte(`{"foo": "baz"}`), + "id": int64(1), + "key": []uint8("1"), + "column1": "foo", + "column2": int32(123), + "column3": false, + "column4": big.NewRat(122, 10), + "column5": big.NewRat(4, 1), + "column6": []byte(`{"foo": "bar"}`), + "column7": []byte(`{"foo": "baz"}`), + "UppercaseColumn1": int32(1), }, { - "id": int64(2), - "key": []uint8("2"), - "column1": "bar", - "column2": int32(456), - "column3": true, - "column4": 13.42, - "column5": int64(8), - "column6": []byte(`{"foo": "bar"}`), - "column7": []byte(`{"foo": "baz"}`), + "id": int64(2), + "key": []uint8("2"), + "column1": "bar", + "column2": int32(456), + "column3": true, + "column4": big.NewRat(1342, 100), // 13.42 + "column5": big.NewRat(8, 1), + "column6": []byte(`{"foo": "bar"}`), + "column7": []byte(`{"foo": "baz"}`), + "UppercaseColumn1": int32(2), }, { - "id": int64(3), - "key": []uint8("3"), - "column1": "baz", - "column2": int32(789), - "column3": false, - "column4": nil, - "column5": int64(9), - "column6": []byte(`{"foo": "bar"}`), - "column7": []byte(`{"foo": "baz"}`), + "id": int64(3), + "key": []uint8("3"), + "column1": "baz", + "column2": int32(789), + "column3": false, + "column4": nil, + "column5": big.NewRat(9, 1), + "column6": []byte(`{"foo": "bar"}`), + "column7": []byte(`{"foo": "baz"}`), + "UppercaseColumn1": int32(3), }, { - "id": int64(4), - "key": []uint8("4"), - "column1": nil, - "column2": nil, - "column3": nil, - "column4": 91.1, - "column5": nil, - "column6": nil, - "column7": nil, + "id": int64(4), + "key": []uint8("4"), + "column1": nil, + "column2": nil, + "column3": nil, + "column4": big.NewRat(911, 10), // 91.1 + "column5": nil, + "column6": nil, + "column7": nil, + "UppercaseColumn1": nil, }, { - "id": int64(6), - "key": nil, - "column1": "bizz", - "column2": int32(1010), - "column3": false, - "column4": 872.2, - "column5": int64(101), - "column6": []byte(`{"foo12": "bar12"}`), - "column7": []byte(`{"foo13": "bar13"}`), + "id": int64(6), + "key": nil, + "column1": "bizz", + "column2": int32(1010), + "column3": false, + "column4": big.NewRat(8722, 10), // 872.2 + "column5": big.NewRat(101, 1), + "column6": []byte(`{"foo12": "bar12"}`), + "column7": []byte(`{"foo13": "bar13"}`), + "UppercaseColumn1": nil, }, { - "id": int64(7), - "key": nil, - "column1": "buzz", - "column2": int32(10101), - "column3": true, - "column4": 121.9, - "column5": int64(51), - "column6": []byte(`{"foo7": "bar7"}`), - "column7": []byte(`{"foo8": "bar8"}`), + "id": int64(7), + "key": nil, + "column1": "buzz", + "column2": int32(10101), + "column3": true, + "column4": big.NewRat(1219, 10), // 121.9 + "column5": big.NewRat(51, 1), + "column6": []byte(`{"foo7": "bar7"}`), + "column7": []byte(`{"foo8": "bar8"}`), + "UppercaseColumn1": nil, }, } } diff --git a/source/logrepl/internal/publication.go b/source/logrepl/internal/publication.go index 7e45666..c19cc13 100644 --- a/source/logrepl/internal/publication.go +++ b/source/logrepl/internal/publication.go @@ -19,6 +19,7 @@ import ( "fmt" "strings" + "github.com/conduitio/conduit-connector-postgres/internal" "github.com/jackc/pgx/v5/pgxpool" ) @@ -36,7 +37,12 @@ func CreatePublication(ctx context.Context, conn *pgxpool.Pool, name string, opt return fmt.Errorf("publication %q requires at least one table", name) } - forTableString := fmt.Sprintf("FOR TABLE %s", strings.Join(opts.Tables, ", ")) + wrappedTablesNames := make([]string, 0, len(opts.Tables)) + for _, t := range opts.Tables { + wrappedTablesNames = append(wrappedTablesNames, internal.WrapSQLIdent(t)) + } + + forTableString := fmt.Sprintf("FOR TABLE %s", strings.Join(wrappedTablesNames, ", ")) var publicationParams string if len(opts.PublicationParams) > 0 { diff --git a/source/logrepl/internal/publication_test.go b/source/logrepl/internal/publication_test.go index 081689e..f04a707 100644 --- a/source/logrepl/internal/publication_test.go +++ b/source/logrepl/internal/publication_test.go @@ -27,7 +27,7 @@ func TestCreatePublication(t *testing.T) { ctx := test.Context(t) pool := test.ConnectPool(ctx, t, test.RegularConnString) - pubNames := []string{"testpub", "123", "test-hyphen", "test=equal"} + pubNames := []string{"testpub", "123", "test-hyphen", "test:semicolon", "test.dot", "test=equal"} pubParams := [][]string{ nil, {"publish = 'insert'"}, diff --git a/source/logrepl/internal/relationset_test.go b/source/logrepl/internal/relationset_test.go index 614a6e1..c120bfe 100644 --- a/source/logrepl/internal/relationset_test.go +++ b/source/logrepl/internal/relationset_test.go @@ -107,6 +107,7 @@ func setupTableAllTypes(ctx context.Context, t *testing.T, conn test.Querier) st table := test.RandomIdentifier(t) query := ` CREATE TABLE %s ( + id bigserial PRIMARY KEY, col_bit bit(8), col_varbit varbit(8), col_boolean boolean, @@ -260,6 +261,7 @@ func insertRowAllTypes(ctx context.Context, t *testing.T, conn test.Querier, tab func isValuesAllTypes(is *is.I, got map[string]any) { want := map[string]any{ + "id": int64(1), "col_bit": pgtype.Bits{ Bytes: []byte{0b01}, Len: 8, @@ -312,7 +314,7 @@ func isValuesAllTypes(is *is.I, got map[string]any) { "col_macaddr": net.HardwareAddr{0x08, 0x00, 0x2b, 0x01, 0x02, 0x26}, "col_macaddr8": net.HardwareAddr{0x08, 0x00, 0x2b, 0x01, 0x02, 0x03, 0x04, 0x27}, "col_money": "$28.00", - "col_numeric": float64(292929.29), + "col_numeric": big.NewRat(29292929, 100), "col_path": pgtype.Path{ P: []pgtype.Vec2{{X: 30, Y: 31}, {X: 32, Y: 33}, {X: 34, Y: 35}}, Closed: false, @@ -351,11 +353,15 @@ func isValuesAllTypes(is *is.I, got map[string]any) { cmp.Comparer(func(x, y netip.Prefix) bool { return x.String() == y.String() }), + cmp.Comparer(func(x, y *big.Rat) bool { + return x.Cmp(y) == 0 + }), )) } func isValuesAllTypesStandalone(is *is.I, got map[string]any) { want := map[string]any{ + "id": int64(1), "col_bit": pgtype.Bits{ Bytes: []byte{0b01}, Len: 8, @@ -408,7 +414,7 @@ func isValuesAllTypesStandalone(is *is.I, got map[string]any) { "col_macaddr": net.HardwareAddr{0x08, 0x00, 0x2b, 0x01, 0x02, 0x26}, "col_macaddr8": net.HardwareAddr{0x08, 0x00, 0x2b, 0x01, 0x02, 0x03, 0x04, 0x27}, "col_money": "$28.00", - "col_numeric": float64(292929.29), + "col_numeric": big.NewRat(29292929, 100), "col_path": pgtype.Path{ P: []pgtype.Vec2{{X: 30, Y: 31}, {X: 32, Y: 33}, {X: 34, Y: 35}}, Closed: false, @@ -447,5 +453,8 @@ func isValuesAllTypesStandalone(is *is.I, got map[string]any) { cmp.Comparer(func(x, y netip.Prefix) bool { return x.String() == y.String() }), + cmp.Comparer(func(x, y *big.Rat) bool { + return x.Cmp(y) == 0 + }), )) } diff --git a/source/paramgen.go b/source/paramgen.go deleted file mode 100644 index 032e846..0000000 --- a/source/paramgen.go +++ /dev/null @@ -1,92 +0,0 @@ -// Code generated by paramgen. DO NOT EDIT. -// Source: github.com/ConduitIO/conduit-commons/tree/main/paramgen - -package source - -import ( - "github.com/conduitio/conduit-commons/config" -) - -const ( - ConfigCdcMode = "cdcMode" - ConfigLogreplAutoCleanup = "logrepl.autoCleanup" - ConfigLogreplPublicationName = "logrepl.publicationName" - ConfigLogreplSlotName = "logrepl.slotName" - ConfigLogreplWithAvroSchema = "logrepl.withAvroSchema" - ConfigSnapshotFetchSize = "snapshot.fetchSize" - ConfigSnapshotMode = "snapshotMode" - ConfigTable = "table" - ConfigTables = "tables" - ConfigUrl = "url" -) - -func (Config) Parameters() map[string]config.Parameter { - return map[string]config.Parameter{ - ConfigCdcMode: { - Default: "auto", - Description: "CDCMode determines how the connector should listen to changes.", - Type: config.ParameterTypeString, - Validations: []config.Validation{ - config.ValidationInclusion{List: []string{"auto", "logrepl"}}, - }, - }, - ConfigLogreplAutoCleanup: { - Default: "true", - Description: "LogreplAutoCleanup determines if the replication slot and publication should be\nremoved when the connector is deleted.", - Type: config.ParameterTypeBool, - Validations: []config.Validation{}, - }, - ConfigLogreplPublicationName: { - Default: "conduitpub", - Description: "LogreplPublicationName determines the publication name in case the\nconnector uses logical replication to listen to changes (see CDCMode).", - Type: config.ParameterTypeString, - Validations: []config.Validation{}, - }, - ConfigLogreplSlotName: { - Default: "conduitslot", - Description: "LogreplSlotName determines the replication slot name in case the\nconnector uses logical replication to listen to changes (see CDCMode).", - Type: config.ParameterTypeString, - Validations: []config.Validation{}, - }, - ConfigLogreplWithAvroSchema: { - Default: "false", - Description: "WithAvroSchema determines whether the connector should attach an avro schema on each\nrecord.", - Type: config.ParameterTypeBool, - Validations: []config.Validation{}, - }, - ConfigSnapshotFetchSize: { - Default: "50000", - Description: "Snapshot fetcher size determines the number of rows to retrieve at a time.", - Type: config.ParameterTypeInt, - Validations: []config.Validation{}, - }, - ConfigSnapshotMode: { - Default: "initial", - Description: "SnapshotMode is whether the plugin will take a snapshot of the entire table before starting cdc mode.", - Type: config.ParameterTypeString, - Validations: []config.Validation{ - config.ValidationInclusion{List: []string{"initial", "never"}}, - }, - }, - ConfigTable: { - Default: "", - Description: "Deprecated: use `tables` instead.", - Type: config.ParameterTypeString, - Validations: []config.Validation{}, - }, - ConfigTables: { - Default: "", - Description: "Tables is a List of table names to read from, separated by a comma, e.g.:\"table1,table2\".\nUse \"*\" if you'd like to listen to all tables.", - Type: config.ParameterTypeString, - Validations: []config.Validation{}, - }, - ConfigUrl: { - Default: "", - Description: "URL is the connection string for the Postgres database.", - Type: config.ParameterTypeString, - Validations: []config.Validation{ - config.ValidationRequired{}, - }, - }, - } -} diff --git a/source/snapshot/fetch_worker.go b/source/snapshot/fetch_worker.go index af00d8a..64b7231 100644 --- a/source/snapshot/fetch_worker.go +++ b/source/snapshot/fetch_worker.go @@ -24,6 +24,7 @@ import ( "github.com/conduitio/conduit-commons/opencdc" cschema "github.com/conduitio/conduit-commons/schema" + "github.com/conduitio/conduit-connector-postgres/internal" "github.com/conduitio/conduit-connector-postgres/source/common" "github.com/conduitio/conduit-connector-postgres/source/position" "github.com/conduitio/conduit-connector-postgres/source/schema" @@ -248,10 +249,10 @@ func (f *FetchWorker) Run(ctx context.Context) error { func (f *FetchWorker) createCursor(ctx context.Context, tx pgx.Tx) (func(), error) { // This query will scan the table for rows based on the conditions. selectQuery := fmt.Sprintf( - "SELECT * FROM %s WHERE %s > %d AND %s <= %d ORDER BY %s", - f.conf.Table, - f.conf.Key, f.lastRead, // range start - f.conf.Key, f.snapshotEnd, // range end, + "SELECT * FROM %s WHERE %s > %d AND %s <= %d ORDER BY %q", + internal.WrapSQLIdent(f.conf.Table), + internal.WrapSQLIdent(f.conf.Key), f.lastRead, // range start + internal.WrapSQLIdent(f.conf.Key), f.snapshotEnd, // range end, f.conf.Key, // order by ) @@ -276,7 +277,9 @@ func (f *FetchWorker) updateSnapshotEnd(ctx context.Context, tx pgx.Tx) error { return nil } - query := fmt.Sprintf("SELECT COALESCE(max(%s), 0) FROM %s", f.conf.Key, f.conf.Table) + query := fmt.Sprintf("SELECT COALESCE(max(%s), 0) FROM %s", + internal.WrapSQLIdent(f.conf.Key), internal.WrapSQLIdent(f.conf.Table), + ) if err := tx.QueryRow(ctx, query).Scan(&f.snapshotEnd); err != nil { return fmt.Errorf("failed to get snapshot end with query %q: %w", query, err) } @@ -395,7 +398,7 @@ func (f *FetchWorker) buildRecordData(fields []pgconn.FieldDescription, values [ tableInfo := f.getTableInfo() for i, fd := range fields { isNotNull := tableInfo.Columns[fd.Name].IsNotNull - + if fd.Name == f.conf.Key { k, err := types.Format(fd.DataTypeOID, values[i], isNotNull) if err != nil { @@ -459,7 +462,7 @@ func (*FetchWorker) validateKey(ctx context.Context, table, key string, tx pgx.T `SELECT EXISTS(SELECT a.attname FROM pg_index i JOIN pg_attribute a ON a.attrelid = i.indrelid AND a.attnum = ANY(i.indkey) WHERE i.indrelid = $1::regclass AND a.attname = $2 AND i.indisprimary)`, - table, key, + internal.WrapSQLIdent(table), key, ).Scan(&isPK); err != nil { return fmt.Errorf("unable to determine key %q constraints: %w", key, err) } diff --git a/source/snapshot/fetch_worker_integration_test.go b/source/snapshot/fetch_worker_integration_test.go index d49fb09..9b991a9 100644 --- a/source/snapshot/fetch_worker_integration_test.go +++ b/source/snapshot/fetch_worker_integration_test.go @@ -18,6 +18,7 @@ import ( "context" "errors" "fmt" + "math/big" "strings" "testing" "time" @@ -121,9 +122,12 @@ func Test_FetcherValidate(t *testing.T) { var ( ctx = test.Context(t) pool = test.ConnectPool(ctx, t, test.RegularConnString) - table = test.SetupTestTable(ctx, t, pool) + table = strings.ToUpper(test.RandomIdentifier(t)) ) + // uppercase table name is required to test primary key fetching + test.SetupTestTableWithName(ctx, t, pool, table) + t.Run("success", func(t *testing.T) { is := is.New(t) f := NewFetchWorker( @@ -271,7 +275,7 @@ func Test_FetcherRun_Initial(t *testing.T) { } is.NoErr(tt.Err()) - is.True(len(gotFetchData) == 4) + is.Equal(len(gotFetchData), 4) var ( value6 = []byte(`{"foo": "bar"}`) @@ -279,17 +283,23 @@ func Test_FetcherRun_Initial(t *testing.T) { ) expectedMatch := []opencdc.StructuredData{ - {"id": int64(1), "key": []uint8{49}, "column1": "foo", "column2": int32(123), "column3": false, "column4": 12.2, "column5": int64(4), "column6": value6, "column7": value7}, - {"id": int64(2), "key": []uint8{50}, "column1": "bar", "column2": int32(456), "column3": true, "column4": 13.42, "column5": int64(8), "column6": value6, "column7": value7}, - {"id": int64(3), "key": []uint8{51}, "column1": "baz", "column2": int32(789), "column3": false, "column4": nil, "column5": int64(9), "column6": value6, "column7": value7}, - {"id": int64(4), "key": []uint8{52}, "column1": nil, "column2": nil, "column3": nil, "column4": 91.1, "column5": nil, "column6": nil, "column7": nil}, + {"id": int64(1), "key": []uint8{49}, "column1": "foo", "column2": int32(123), "column3": false, "column4": big.NewRat(122, 10), "column5": big.NewRat(4, 1), "column6": value6, "column7": value7, "UppercaseColumn1": int32(1)}, + {"id": int64(2), "key": []uint8{50}, "column1": "bar", "column2": int32(456), "column3": true, "column4": big.NewRat(1342, 100), "column5": big.NewRat(8, 1), "column6": value6, "column7": value7, "UppercaseColumn1": int32(2)}, + {"id": int64(3), "key": []uint8{51}, "column1": "baz", "column2": int32(789), "column3": false, "column4": nil, "column5": big.NewRat(9, 1), "column6": value6, "column7": value7, "UppercaseColumn1": int32(3)}, + {"id": int64(4), "key": []uint8{52}, "column1": nil, "column2": nil, "column3": nil, "column4": big.NewRat(911, 10), "column5": nil, "column6": nil, "column7": nil, "UppercaseColumn1": nil}, } for i, got := range gotFetchData { t.Run(fmt.Sprintf("payload_%d", i+1), func(t *testing.T) { is := is.New(t) is.Equal(got.Key, opencdc.StructuredData{"id": int64(i + 1)}) - is.Equal("", cmp.Diff(expectedMatch[i], got.Payload)) + is.Equal("", cmp.Diff( + expectedMatch[i], + got.Payload, + cmp.Comparer(func(x, y *big.Rat) bool { + return x.Cmp(y) == 0 + }), + )) is.Equal(got.Position, position.SnapshotPosition{ LastRead: int64(i + 1), @@ -344,17 +354,27 @@ func Test_FetcherRun_Resume(t *testing.T) { // validate generated record is.Equal(dd[0].Key, opencdc.StructuredData{"id": int64(3)}) - is.Equal("", cmp.Diff(dd[0].Payload, opencdc.StructuredData{ - "id": int64(3), - "key": []uint8{51}, - "column1": "baz", - "column2": int32(789), - "column3": false, - "column4": nil, - "column5": int64(9), - "column6": []byte(`{"foo": "bar"}`), - "column7": []byte(`{"foo": "baz"}`), - })) + is.Equal( + "", + cmp.Diff( + dd[0].Payload, + opencdc.StructuredData{ + "id": int64(3), + "key": []uint8{51}, + "column1": "baz", + "column2": int32(789), + "column3": false, + "column4": nil, + "column5": big.NewRat(9, 1), + "column6": []byte(`{"foo": "bar"}`), + "column7": []byte(`{"foo": "baz"}`), + "UppercaseColumn1": int32(3), + }, + cmp.Comparer(func(x, y *big.Rat) bool { + return x.Cmp(y) == 0 + }), + ), + ) is.Equal(dd[0].Position, position.SnapshotPosition{ LastRead: 3, @@ -475,9 +495,11 @@ func Test_FetchWorker_updateSnapshotEnd(t *testing.T) { is = is.New(t) ctx = test.Context(t) pool = test.ConnectPool(ctx, t, test.RegularConnString) - table = test.SetupTestTable(ctx, t, pool) + table = strings.ToUpper(test.RandomIdentifier(t)) ) + test.SetupTestTableWithName(ctx, t, pool, table) + tx, err := pool.Begin(ctx) is.NoErr(err) t.Cleanup(func() { is.NoErr(tx.Rollback(ctx)) }) @@ -496,6 +518,14 @@ func Test_FetchWorker_updateSnapshotEnd(t *testing.T) { }}, expected: 4, }, + { + desc: "success with capitalized key", + w: &FetchWorker{conf: FetchConfig{ + Table: table, + Key: "UppercaseColumn1", + }}, + expected: 3, + }, { desc: "skip update when set", w: &FetchWorker{snapshotEnd: 10}, @@ -561,7 +591,7 @@ func Test_FetchWorker_createCursor(t *testing.T) { ) is.Equal( cursorDef, - fmt.Sprintf("DECLARE cursor123 CURSOR FOR(SELECT * FROM %s WHERE id > 10 AND id <= 15 ORDER BY id)", table), + fmt.Sprintf(`DECLARE cursor123 CURSOR FOR(SELECT * FROM %q WHERE "id" > 10 AND "id" <= 15 ORDER BY "id")`, table), ) is.NoErr(tx.Rollback(ctx)) diff --git a/source/snapshot/iterator.go b/source/snapshot/iterator.go index f9c312f..cc7e74e 100644 --- a/source/snapshot/iterator.go +++ b/source/snapshot/iterator.go @@ -80,24 +80,52 @@ func NewIterator(ctx context.Context, db *pgxpool.Pool, c Config) (*Iterator, er return i, nil } -func (i *Iterator) Next(ctx context.Context) (opencdc.Record, error) { +// NextN takes and returns up to n records from the queue. NextN is allowed to +// block until either at least one record is available or the context gets canceled. +func (i *Iterator) NextN(ctx context.Context, n int) ([]opencdc.Record, error) { + if n <= 0 { + return nil, fmt.Errorf("n must be greater than 0, got %d", n) + } + + var records []opencdc.Record + + // Get first record (blocking) select { case <-ctx.Done(): - return opencdc.Record{}, fmt.Errorf("iterator stopped: %w", ctx.Err()) + return nil, fmt.Errorf("iterator stopped: %w", ctx.Err()) case d, ok := <-i.data: if !ok { // closed if err := i.t.Err(); err != nil { - return opencdc.Record{}, fmt.Errorf("fetchers exited unexpectedly: %w", err) + return nil, fmt.Errorf("fetchers exited unexpectedly: %w", err) } if err := i.acks.Wait(ctx); err != nil { - return opencdc.Record{}, fmt.Errorf("failed to wait for acks: %w", err) + return nil, fmt.Errorf("failed to wait for acks: %w", err) } - return opencdc.Record{}, ErrIteratorDone + return nil, ErrIteratorDone } i.acks.Add(1) - return i.buildRecord(d), nil + records = append(records, i.buildRecord(d)) + } + + // Try to get remaining records non-blocking + for len(records) < n { + select { + case <-ctx.Done(): + return records, ctx.Err() + case d, ok := <-i.data: + if !ok { // closed + return records, nil + } + i.acks.Add(1) + records = append(records, i.buildRecord(d)) + default: + // No more records currently available + return records, nil + } } + + return records, nil } func (i *Iterator) Ack(_ context.Context, _ opencdc.Position) error { @@ -120,7 +148,7 @@ func (i *Iterator) buildRecord(d FetchData) opencdc.Record { pos := i.lastPosition.ToSDKPosition() metadata := make(opencdc.Metadata) - metadata["postgres.table"] = d.Table + metadata[opencdc.MetadataCollection] = d.Table rec := sdk.Util.Source.NewRecordSnapshot(pos, metadata, d.Key, d.Payload) if i.conf.WithAvroSchema { diff --git a/source/snapshot/iterator_test.go b/source/snapshot/iterator_test.go index 02079ed..610f2aa 100644 --- a/source/snapshot/iterator_test.go +++ b/source/snapshot/iterator_test.go @@ -26,7 +26,7 @@ import ( "github.com/matryer/is" ) -func Test_Iterator_Next(t *testing.T) { +func Test_Iterator_NextN(t *testing.T) { var ( ctx = test.Context(t) pool = test.ConnectPool(ctx, t, test.RegularConnString) @@ -48,18 +48,32 @@ func Test_Iterator_Next(t *testing.T) { is.NoErr(i.Teardown(ctx)) }() - for j := 1; j <= 4; j++ { - r, err := i.Next(ctx) - is.NoErr(err) + // Get first 2 records + records, err := i.NextN(ctx, 2) + is.NoErr(err) + is.Equal(len(records), 2) + for _, r := range records { + is.Equal(r.Operation, opencdc.OperationSnapshot) + is.Equal(r.Metadata[opencdc.MetadataCollection], table) + } + + // Get remaining 2 records + records, err = i.NextN(ctx, 2) + is.NoErr(err) + is.Equal(len(records), 2) + for _, r := range records { is.Equal(r.Operation, opencdc.OperationSnapshot) + is.Equal(r.Metadata[opencdc.MetadataCollection], table) } + // Ack all records for j := 1; j <= 4; j++ { err = i.Ack(ctx, nil) is.NoErr(err) } - _, err = i.Next(ctx) + // Should return ErrIteratorDone + _, err = i.NextN(ctx, 1) is.Equal(err, ErrIteratorDone) }) @@ -78,10 +92,15 @@ func Test_Iterator_Next(t *testing.T) { is.NoErr(i.Teardown(ctx)) }() - for j := 1; j <= 4; j++ { - _, err = i.Next(ctx) + // Get all 4 records in multiple calls since NextN is non-blocking + var allRecords []opencdc.Record + for len(allRecords) < 4 { + records, err := i.NextN(ctx, 4) is.NoErr(err) + allRecords = append(allRecords, records...) } + is.Equal(len(allRecords), 4) + // Only ack 3 records for j := 1; j <= 3; j++ { err = i.Ack(ctx, nil) @@ -91,16 +110,16 @@ func Test_Iterator_Next(t *testing.T) { ctxTimeout, cancel := context.WithTimeout(ctx, time.Millisecond*10) defer cancel() - // No more records, but Next blocks because we haven't acked all records - _, err = i.Next(ctxTimeout) + // No more records, but NextN blocks because we haven't acked all records + _, err = i.NextN(ctxTimeout, 1) is.True(errors.Is(err, context.DeadlineExceeded)) // Ack the last record err = i.Ack(ctx, nil) is.NoErr(err) - // Now Next won't block - _, err = i.Next(ctx) + // Now NextN won't block + _, err = i.NextN(ctx, 1) is.Equal(err, ErrIteratorDone) }) @@ -122,7 +141,7 @@ func Test_Iterator_Next(t *testing.T) { cancelCtx, cancel := context.WithCancel(ctx) cancel() - _, err = i.Next(cancelCtx) + _, err = i.NextN(cancelCtx, 1) is.Equal(err.Error(), "iterator stopped: context canceled") }) @@ -144,7 +163,31 @@ func Test_Iterator_Next(t *testing.T) { cancel() - _, err = i.Next(ctx) + _, err = i.NextN(ctx, 1) is.True(errors.Is(err, context.Canceled)) }) + + t.Run("invalid n", func(t *testing.T) { + is := is.New(t) + + i, err := NewIterator(ctx, pool, Config{ + Position: position.Position{}.ToSDKPosition(), + Tables: []string{table}, + TableKeys: map[string]string{ + table: "id", + }, + }) + is.NoErr(err) + defer func() { + is.NoErr(i.Teardown(ctx)) + }() + + _, err = i.NextN(ctx, 0) + is.True(err != nil) + is.Equal(err.Error(), "n must be greater than 0, got 0") + + _, err = i.NextN(ctx, -1) + is.True(err != nil) + is.Equal(err.Error(), "n must be greater than 0, got -1") + }) } diff --git a/source/types/numeric.go b/source/types/numeric.go index c7dfbf0..8fd2690 100644 --- a/source/types/numeric.go +++ b/source/types/numeric.go @@ -15,42 +15,23 @@ package types import ( + "math/big" + "github.com/jackc/pgx/v5/pgtype" ) type NumericFormatter struct{} -// Format coerces `pgtype.Numeric` to int or double depending on the exponent. -// Returns error when value is invalid. -func (NumericFormatter) Format(num pgtype.Numeric) (any, error) { - // N.B. The numeric type in pgx is represented by two ints. - // When the type in Postgres is defined as `NUMERIC(10)' the scale is assumed to be 0. - // However, pgx may represent the number as two ints e.g. 1200 -> (int=12,exp=2) = 12*10^2. as well - // as a type with zero exponent, e.g. 121 -> (int=121,exp=0). - // Thus, a Numeric type with positive or zero exponent is assumed to be an integer. - if num.Exp >= 0 { - i8v, err := num.Int64Value() - if err != nil { - return nil, err - } - - v, err := i8v.Value() - if err != nil { - return nil, err - } - - return v, nil +// Format converts a pgtype.Numeric to a big.Rat. +func (NumericFormatter) Format(num pgtype.Numeric) (*big.Rat, error) { + if num.Int == nil { + return nil, nil } - - f8v, err := num.Float64Value() + v := new(big.Rat) + driverVal, err := num.Value() if err != nil { return nil, err } - - v, err := f8v.Value() - if err != nil { - return nil, err - } - + v.SetString(driverVal.(string)) return v, nil } diff --git a/source/types/types_test.go b/source/types/types_test.go index 3c42d2f..fb1dea7 100644 --- a/source/types/types_test.go +++ b/source/types/types_test.go @@ -15,6 +15,7 @@ package types import ( + "math/big" "testing" "time" @@ -47,13 +48,13 @@ func Test_Format(t *testing.T) { { name: "pgtype.Numeric", input: []any{ - pgxNumeric(t, "12.2121"), pgxNumeric(t, "101"), &pgtype.Numeric{}, nil, + pgxNumeric(t, "12.2121"), pgxNumeric(t, "101"), pgxNumeric(t, "0"), &pgtype.Numeric{}, nil, }, inputOID: []uint32{ - 0, 0, 0, 0, + 0, 0, 0, 0, 0, }, expect: []any{ - float64(12.2121), int64(101), nil, nil, + big.NewRat(122121, 10000), big.NewRat(101, 1), big.NewRat(0, 1), nil, nil, }, }, { diff --git a/source_integration_test.go b/source_integration_test.go index c40cb5d..628769e 100644 --- a/source_integration_test.go +++ b/source_integration_test.go @@ -19,9 +19,12 @@ import ( "fmt" "testing" + "github.com/conduitio/conduit-commons/config" "github.com/conduitio/conduit-commons/opencdc" + "github.com/conduitio/conduit-connector-postgres/source" "github.com/conduitio/conduit-connector-postgres/source/logrepl" "github.com/conduitio/conduit-connector-postgres/test" + sdk "github.com/conduitio/conduit-connector-sdk" "github.com/matryer/is" ) @@ -34,7 +37,7 @@ func TestSource_Read(t *testing.T) { publicationName := "conduitpub1" s := NewSource() - err := s.Configure( + err := sdk.Util.ParseConfig( ctx, map[string]string{ "url": test.RepmgrConnString, @@ -44,6 +47,8 @@ func TestSource_Read(t *testing.T) { "logrepl.slotName": slotName, "logrepl.publicationName": publicationName, }, + s.Config(), + Connector.NewSpecification().SourceParams, ) is.NoErr(err) @@ -241,3 +246,46 @@ func insertRowAllColumns(ctx context.Context, t *testing.T, table string, rowNum _, err := conn.Exec(ctx, query) is.NoErr(err) } + +func TestSource_ParseConfig(t *testing.T) { + testCases := []struct { + name string + cfg config.Config + wantErr bool + }{ + { + name: "valid postgres replication slot name", + cfg: config.Config{ + "url": "postgresql://meroxauser:meroxapass@127.0.0.1:5432/meroxadb", + "tables": "table1,table2", + "cdcMode": "logrepl", + "logrepl.slotName": "valid_slot_name", + }, + wantErr: false, + }, { + name: "invalid postgres replication slot name", + cfg: config.Config{ + "url": "postgresql://meroxauser:meroxapass@127.0.0.1:5432/meroxadb", + "tables": "table1,table2", + "cdcMode": "logrepl", + "logrepl.slotName": "invalid:slot.name", + }, + wantErr: true, + }, + } + + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + is := is.New(t) + + var cfg source.Config + err := sdk.Util.ParseConfig(context.Background(), tc.cfg, cfg, Connector.NewSpecification().SourceParams) + + if tc.wantErr { + is.True(err != nil) + return + } + is.NoErr(err) + }) + } +} diff --git a/spec.go b/spec.go deleted file mode 100644 index c65e2f6..0000000 --- a/spec.go +++ /dev/null @@ -1,33 +0,0 @@ -// Copyright © 2022 Meroxa, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package postgres - -import ( - sdk "github.com/conduitio/conduit-connector-sdk" -) - -// version is set during the build process (i.e. the Makefile). -// Default version matches default from runtime/debug. -var version = "(devel)" - -// Specification returns the Plugin's Specification. -func Specification() sdk.Specification { - return sdk.Specification{ - Name: "postgres", - Summary: "A PostgreSQL source and destination plugin for Conduit.", - Version: version, - Author: "Meroxa, Inc.", - } -} diff --git a/test/docker-compose.yml b/test/docker-compose.yml index 8b6a1fc..16344bc 100644 --- a/test/docker-compose.yml +++ b/test/docker-compose.yml @@ -1,6 +1,6 @@ services: pg-0: - image: docker.io/bitnami/postgresql-repmgr:16 + image: docker.io/bitnami/postgresql-repmgr:17.5.0 ports: - "5433:5432" volumes: diff --git a/test/helper.go b/test/helper.go index 113fe44..6e72e21 100644 --- a/test/helper.go +++ b/test/helper.go @@ -18,6 +18,7 @@ import ( "context" "errors" "fmt" + "strconv" "strings" "testing" "time" @@ -45,6 +46,7 @@ const TestTableAvroSchemaV1 = `{ "name": "%s", "fields": [ + {"name":"UppercaseColumn1","type":"int"}, {"name":"column1","type":"string"}, {"name":"column2","type":"int"}, {"name":"column3","type":"boolean"}, @@ -80,6 +82,7 @@ const TestTableAvroSchemaV2 = `{ "name": "%s", "fields": [ + {"name":"UppercaseColumn1","type":"int"}, {"name":"column1","type":"string"}, {"name":"column101","type":{"type":"long","logicalType":"local-timestamp-micros"}}, {"name":"column2","type":"int"}, @@ -116,6 +119,7 @@ const TestTableAvroSchemaV3 = `{ "name": "%s", "fields": [ + {"name":"UppercaseColumn1","type":"int"}, {"name":"column1","type":"string"}, {"name":"column101","type":{"type":"long","logicalType":"local-timestamp-micros"}}, {"name":"column2","type":"int"}, @@ -139,7 +143,7 @@ const TestTableKeyAvroSchema = `{ // When updating this table, TestTableAvroSchemaV1 needs to be updated too. const testTableCreateQuery = ` - CREATE TABLE %s ( + CREATE TABLE %q ( id bigserial PRIMARY KEY, key bytea, column1 varchar(256), @@ -148,7 +152,8 @@ const testTableCreateQuery = ` column4 numeric(16,3), column5 numeric(5), column6 jsonb, - column7 json + column7 json, + "UppercaseColumn1" integer )` type Querier interface { @@ -182,39 +187,45 @@ func ConnectSimple(ctx context.Context, t *testing.T, connString string) *pgx.Co // SetupTestTable creates a new table and returns its name. func SetupEmptyTestTable(ctx context.Context, t *testing.T, conn Querier) string { - is := is.New(t) - table := RandomIdentifier(t) + SetupEmptyTestTableWithName(ctx, t, conn, table) + return table +} + +func SetupEmptyTestTableWithName(ctx context.Context, t *testing.T, conn Querier, table string) { + is := is.New(t) query := fmt.Sprintf(testTableCreateQuery, table) _, err := conn.Exec(ctx, query) is.NoErr(err) t.Cleanup(func() { - query := `DROP TABLE %s` + query := `DROP TABLE %q` query = fmt.Sprintf(query, table) _, err := conn.Exec(context.Background(), query) is.NoErr(err) }) - - return table } -// SetupTestTable creates a new table and returns its name. -func SetupTestTable(ctx context.Context, t *testing.T, conn Querier) string { +func SetupTestTableWithName(ctx context.Context, t *testing.T, conn Querier, table string) { is := is.New(t) - table := SetupEmptyTestTable(ctx, t, conn) + SetupEmptyTestTableWithName(ctx, t, conn, table) query := ` - INSERT INTO %s (key, column1, column2, column3, column4, column5, column6, column7) - VALUES ('1', 'foo', 123, false, 12.2, 4, '{"foo": "bar"}', '{"foo": "baz"}'), - ('2', 'bar', 456, true, 13.42, 8, '{"foo": "bar"}', '{"foo": "baz"}'), - ('3', 'baz', 789, false, null, 9, '{"foo": "bar"}', '{"foo": "baz"}'), - ('4', null, null, null, 91.1, null, null, null)` + INSERT INTO %q (key, column1, column2, column3, column4, column5, column6, column7, "UppercaseColumn1") + VALUES ('1', 'foo', 123, false, 12.2, 4, '{"foo": "bar"}', '{"foo": "baz"}', 1), + ('2', 'bar', 456, true, 13.42, 8, '{"foo": "bar"}', '{"foo": "baz"}', 2), + ('3', 'baz', 789, false, null, 9, '{"foo": "bar"}', '{"foo": "baz"}', 3), + ('4', null, null, null, 91.1, null, null, null, null)` query = fmt.Sprintf(query, table) _, err := conn.Exec(ctx, query) is.NoErr(err) +} +// SetupTestTable creates a new table and returns its name. +func SetupTestTable(ctx context.Context, t *testing.T, conn Querier) string { + table := RandomIdentifier(t) + SetupTestTableWithName(ctx, t, conn, table) return table } @@ -242,14 +253,20 @@ func CreateReplicationSlot(t *testing.T, conn Querier, slotName string) { func CreatePublication(t *testing.T, conn Querier, pubName string, tables []string) { is := is.New(t) + quotedTables := make([]string, 0, len(tables)) + for _, t := range tables { + // don't use internal.WrapSQLIdent to prevent import cycle + quotedTables = append(quotedTables, strconv.Quote(t)) + } + _, err := conn.Exec( context.Background(), - "CREATE PUBLICATION "+pubName+" FOR TABLE "+strings.Join(tables, ","), + fmt.Sprintf("CREATE PUBLICATION %s FOR TABLE %s", pubName, strings.Join(quotedTables, ",")), ) is.NoErr(err) t.Cleanup(func() { - _, err := conn.Exec(context.Background(), "DROP PUBLICATION IF EXISTS "+pubName) + _, err := conn.Exec(context.Background(), fmt.Sprintf("DROP PUBLICATION IF EXISTS %q", pubName)) is.NoErr(err) }) } diff --git a/tools/go.mod b/tools/go.mod new file mode 100644 index 0000000..aaead4b --- /dev/null +++ b/tools/go.mod @@ -0,0 +1,254 @@ +module github.com/conduitio/conduit-connector-postgres/tools + +go 1.24.2 + +tool ( + github.com/conduitio/conduit-connector-sdk/conn-sdk-cli + github.com/daixiang0/gci + github.com/golangci/golangci-lint/v2/cmd/golangci-lint + golang.org/x/tools/cmd/stringer + mvdan.cc/gofumpt +) + +require ( + 4d63.com/gocheckcompilerdirectives v1.3.0 // indirect + 4d63.com/gochecknoglobals v0.2.2 // indirect + dario.cat/mergo v1.0.1 // indirect + github.com/4meepo/tagalign v1.4.2 // indirect + github.com/Abirdcfly/dupword v0.1.3 // indirect + github.com/Antonboom/errname v1.1.0 // indirect + github.com/Antonboom/nilnil v1.1.0 // indirect + github.com/Antonboom/testifylint v1.6.1 // indirect + github.com/BurntSushi/toml v1.5.0 // indirect + github.com/Crocmagnon/fatcontext v0.7.1 // indirect + github.com/Djarvur/go-err113 v0.0.0-20210108212216-aea10b59be24 // indirect + github.com/GaijinEntertainment/go-exhaustruct/v3 v3.3.1 // indirect + github.com/Masterminds/goutils v1.1.1 // indirect + github.com/Masterminds/semver/v3 v3.3.1 // indirect + github.com/Masterminds/sprig/v3 v3.3.0 // indirect + github.com/OpenPeeDeeP/depguard/v2 v2.2.1 // indirect + github.com/alecthomas/chroma/v2 v2.17.2 // indirect + github.com/alecthomas/go-check-sumtype v0.3.1 // indirect + github.com/alexkohler/nakedret/v2 v2.0.6 // indirect + github.com/alexkohler/prealloc v1.0.0 // indirect + github.com/alingse/asasalint v0.0.11 // indirect + github.com/alingse/nilnesserr v0.2.0 // indirect + github.com/ashanbrown/forbidigo v1.6.0 // indirect + github.com/ashanbrown/makezero v1.2.0 // indirect + github.com/aymanbagabas/go-osc52/v2 v2.0.1 // indirect + github.com/beorn7/perks v1.0.1 // indirect + github.com/bkielbasa/cyclop v1.2.3 // indirect + github.com/blizzy78/varnamelen v0.8.0 // indirect + github.com/bombsimon/wsl/v4 v4.7.0 // indirect + github.com/breml/bidichk v0.3.3 // indirect + github.com/breml/errchkjson v0.4.1 // indirect + github.com/butuzov/ireturn v0.4.0 // indirect + github.com/butuzov/mirror v1.3.0 // indirect + github.com/catenacyber/perfsprint v0.9.1 // indirect + github.com/ccojocar/zxcvbn-go v1.0.2 // indirect + github.com/cespare/xxhash/v2 v2.3.0 // indirect + github.com/charithe/durationcheck v0.0.10 // indirect + github.com/charmbracelet/colorprofile v0.2.3-0.20250311203215-f60798e515dc // indirect + github.com/charmbracelet/lipgloss v1.1.0 // indirect + github.com/charmbracelet/x/ansi v0.8.0 // indirect + github.com/charmbracelet/x/cellbuf v0.0.13-0.20250311204145-2c3ea96c31dd // indirect + github.com/charmbracelet/x/term v0.2.1 // indirect + github.com/chavacava/garif v0.1.0 // indirect + github.com/ckaznocha/intrange v0.3.1 // indirect + github.com/conduitio/conduit-commons v0.5.2 // indirect + github.com/conduitio/conduit-connector-protocol v0.9.2 // indirect + github.com/conduitio/conduit-connector-sdk v0.13.3 // indirect + github.com/conduitio/evolviconf v0.1.0 // indirect + github.com/conduitio/evolviconf/evolviyaml v0.1.0 // indirect + github.com/conduitio/yaml/v3 v3.3.0 // indirect + github.com/curioswitch/go-reassign v0.3.0 // indirect + github.com/daixiang0/gci v0.13.6 // indirect + github.com/dave/dst v0.27.3 // indirect + github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc // indirect + github.com/denis-tingaikin/go-header v0.5.0 // indirect + github.com/dlclark/regexp2 v1.11.5 // indirect + github.com/ettle/strcase v0.2.0 // indirect + github.com/fatih/color v1.18.0 // indirect + github.com/fatih/structtag v1.2.0 // indirect + github.com/firefart/nonamedreturns v1.0.6 // indirect + github.com/fsnotify/fsnotify v1.7.0 // indirect + github.com/fzipp/gocyclo v0.6.0 // indirect + github.com/ghostiam/protogetter v0.3.15 // indirect + github.com/go-critic/go-critic v0.13.0 // indirect + github.com/go-toolsmith/astcast v1.1.0 // indirect + github.com/go-toolsmith/astcopy v1.1.0 // indirect + github.com/go-toolsmith/astequal v1.2.0 // indirect + github.com/go-toolsmith/astfmt v1.1.0 // indirect + github.com/go-toolsmith/astp v1.1.0 // indirect + github.com/go-toolsmith/strparse v1.1.0 // indirect + github.com/go-toolsmith/typep v1.1.0 // indirect + github.com/go-viper/mapstructure/v2 v2.2.1 // indirect + github.com/go-xmlfmt/xmlfmt v1.1.3 // indirect + github.com/gobwas/glob v0.2.3 // indirect + github.com/goccy/go-json v0.10.5 // indirect + github.com/gofrs/flock v0.12.1 // indirect + github.com/golang/protobuf v1.5.4 // indirect + github.com/golangci/dupl v0.0.0-20250308024227-f665c8d69b32 // indirect + github.com/golangci/go-printf-func-name v0.1.0 // indirect + github.com/golangci/gofmt v0.0.0-20250106114630-d62b90e6713d // indirect + github.com/golangci/golangci-lint v1.64.8 // indirect + github.com/golangci/golangci-lint/v2 v2.1.6 // indirect + github.com/golangci/golines v0.0.0-20250217134842-442fd0091d95 // indirect + github.com/golangci/misspell v0.6.0 // indirect + github.com/golangci/plugin-module-register v0.1.1 // indirect + github.com/golangci/revgrep v0.8.0 // indirect + github.com/golangci/unconvert v0.0.0-20250410112200-a129a6e6413e // indirect + github.com/google/go-cmp v0.7.0 // indirect + github.com/google/uuid v1.6.0 // indirect + github.com/gordonklaus/ineffassign v0.1.0 // indirect + github.com/gostaticanalysis/analysisutil v0.7.1 // indirect + github.com/gostaticanalysis/comment v1.5.0 // indirect + github.com/gostaticanalysis/forcetypeassert v0.2.0 // indirect + github.com/gostaticanalysis/nilerr v0.1.1 // indirect + github.com/hamba/avro/v2 v2.28.0 // indirect + github.com/hashicorp/go-hclog v1.6.3 // indirect + github.com/hashicorp/go-immutable-radix/v2 v2.1.0 // indirect + github.com/hashicorp/go-plugin v1.6.3 // indirect + github.com/hashicorp/go-version v1.7.0 // indirect + github.com/hashicorp/golang-lru/v2 v2.0.7 // indirect + github.com/hashicorp/hcl v1.0.0 // indirect + github.com/hashicorp/yamux v0.1.1 // indirect + github.com/hexops/gotextdiff v1.0.3 // indirect + github.com/huandu/xstrings v1.5.0 // indirect + github.com/inconshreveable/mousetrap v1.1.0 // indirect + github.com/jgautheron/goconst v1.8.1 // indirect + github.com/jingyugao/rowserrcheck v1.1.1 // indirect + github.com/jjti/go-spancheck v0.6.4 // indirect + github.com/jpillora/backoff v1.0.0 // indirect + github.com/json-iterator/go v1.1.12 // indirect + github.com/julz/importas v0.2.0 // indirect + github.com/karamaru-alpha/copyloopvar v1.2.1 // indirect + github.com/kisielk/errcheck v1.9.0 // indirect + github.com/kkHAIKE/contextcheck v1.1.6 // indirect + github.com/kulti/thelper v0.6.3 // indirect + github.com/kunwardeep/paralleltest v1.0.14 // indirect + github.com/lasiar/canonicalheader v1.1.2 // indirect + github.com/ldez/exptostd v0.4.3 // indirect + github.com/ldez/gomoddirectives v0.6.1 // indirect + github.com/ldez/grignotin v0.9.0 // indirect + github.com/ldez/tagliatelle v0.7.1 // indirect + github.com/ldez/usetesting v0.4.3 // indirect + github.com/leonklingele/grouper v1.1.2 // indirect + github.com/lucasb-eyer/go-colorful v1.2.0 // indirect + github.com/macabu/inamedparam v0.2.0 // indirect + github.com/magiconair/properties v1.8.7 // indirect + github.com/manuelarte/funcorder v0.2.1 // indirect + github.com/maratori/testableexamples v1.0.0 // indirect + github.com/maratori/testpackage v1.1.1 // indirect + github.com/matoous/godox v1.1.0 // indirect + github.com/matryer/is v1.4.1 // indirect + github.com/mattn/go-colorable v0.1.14 // indirect + github.com/mattn/go-isatty v0.0.20 // indirect + github.com/mattn/go-runewidth v0.0.16 // indirect + github.com/mgechev/revive v1.9.0 // indirect + github.com/mitchellh/copystructure v1.2.0 // indirect + github.com/mitchellh/go-homedir v1.1.0 // indirect + github.com/mitchellh/mapstructure v1.5.0 // indirect + github.com/mitchellh/reflectwalk v1.0.2 // indirect + github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd // indirect + github.com/modern-go/reflect2 v1.0.2 // indirect + github.com/moricho/tparallel v0.3.2 // indirect + github.com/muesli/termenv v0.16.0 // indirect + github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822 // indirect + github.com/nakabonne/nestif v0.3.1 // indirect + github.com/nishanths/exhaustive v0.12.0 // indirect + github.com/nishanths/predeclared v0.2.2 // indirect + github.com/nunnatsa/ginkgolinter v0.19.1 // indirect + github.com/oklog/run v1.1.0 // indirect + github.com/olekukonko/tablewriter v0.0.5 // indirect + github.com/pelletier/go-toml/v2 v2.2.4 // indirect + github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2 // indirect + github.com/polyfloyd/go-errorlint v1.8.0 // indirect + github.com/prometheus/client_golang v1.20.2 // indirect + github.com/prometheus/client_model v0.6.1 // indirect + github.com/prometheus/common v0.55.0 // indirect + github.com/prometheus/procfs v0.15.1 // indirect + github.com/quasilyte/go-ruleguard v0.4.4 // indirect + github.com/quasilyte/go-ruleguard/dsl v0.3.22 // indirect + github.com/quasilyte/gogrep v0.5.0 // indirect + github.com/quasilyte/regex/syntax v0.0.0-20210819130434-b3f0c404a727 // indirect + github.com/quasilyte/stdinfo v0.0.0-20220114132959-f7386bf02567 // indirect + github.com/raeperd/recvcheck v0.2.0 // indirect + github.com/rivo/uniseg v0.4.7 // indirect + github.com/rogpeppe/go-internal v1.14.1 // indirect + github.com/rs/zerolog v1.33.0 // indirect + github.com/ryancurrah/gomodguard v1.4.1 // indirect + github.com/ryanrolds/sqlclosecheck v0.5.1 // indirect + github.com/sagikazarmark/locafero v0.6.0 // indirect + github.com/sagikazarmark/slog-shim v0.1.0 // indirect + github.com/samber/lo v1.47.0 // indirect + github.com/samber/slog-common v0.18.1 // indirect + github.com/samber/slog-zerolog/v2 v2.7.3 // indirect + github.com/sanposhiho/wastedassign/v2 v2.1.0 // indirect + github.com/santhosh-tekuri/jsonschema/v6 v6.0.1 // indirect + github.com/sashamelentyev/interfacebloat v1.1.0 // indirect + github.com/sashamelentyev/usestdlibvars v1.28.0 // indirect + github.com/securego/gosec/v2 v2.22.3 // indirect + github.com/shopspring/decimal v1.4.0 // indirect + github.com/sirupsen/logrus v1.9.3 // indirect + github.com/sivchari/containedctx v1.0.3 // indirect + github.com/sivchari/tenv v1.12.1 // indirect + github.com/sonatard/noctx v0.1.0 // indirect + github.com/sourcegraph/conc v0.3.0 // indirect + github.com/sourcegraph/go-diff v0.7.0 // indirect + github.com/spf13/afero v1.14.0 // indirect + github.com/spf13/cast v1.7.0 // indirect + github.com/spf13/cobra v1.9.1 // indirect + github.com/spf13/pflag v1.0.6 // indirect + github.com/spf13/viper v1.19.0 // indirect + github.com/ssgreg/nlreturn/v2 v2.2.1 // indirect + github.com/stbenjam/no-sprintf-host-port v0.2.0 // indirect + github.com/stretchr/objx v0.5.2 // indirect + github.com/stretchr/testify v1.10.0 // indirect + github.com/subosito/gotenv v1.6.0 // indirect + github.com/tdakkota/asciicheck v0.4.1 // indirect + github.com/tetafro/godot v1.5.1 // indirect + github.com/timakin/bodyclose v0.0.0-20241222091800-1db5c5ca4d67 // indirect + github.com/timonwong/loggercheck v0.11.0 // indirect + github.com/tomarrell/wrapcheck/v2 v2.11.0 // indirect + github.com/tommy-muehle/go-mnd/v2 v2.5.1 // indirect + github.com/twmb/go-cache v1.2.1 // indirect + github.com/ultraware/funlen v0.2.0 // indirect + github.com/ultraware/whitespace v0.2.0 // indirect + github.com/uudashr/gocognit v1.2.0 // indirect + github.com/uudashr/iface v1.3.1 // indirect + github.com/xen0n/gosmopolitan v1.3.0 // indirect + github.com/xo/terminfo v0.0.0-20220910002029-abceb7e1c41e // indirect + github.com/yagipy/maintidx v1.0.0 // indirect + github.com/yeya24/promlinter v0.3.0 // indirect + github.com/ykadowak/zerologlint v0.1.5 // indirect + gitlab.com/bosi/decorder v0.4.2 // indirect + go-simpler.org/musttag v0.13.1 // indirect + go-simpler.org/sloglint v0.11.0 // indirect + go.augendre.info/fatcontext v0.8.0 // indirect + go.uber.org/automaxprocs v1.6.0 // indirect + go.uber.org/goleak v1.3.0 // indirect + go.uber.org/mock v0.5.0 // indirect + go.uber.org/multierr v1.11.0 // indirect + go.uber.org/zap v1.27.0 // indirect + golang.org/x/crypto v0.37.0 // indirect + golang.org/x/exp v0.0.0-20250228200357-dead58393ab7 // indirect + golang.org/x/exp/typeparams v0.0.0-20250210185358-939b2ce775ac // indirect + golang.org/x/mod v0.24.0 // indirect + golang.org/x/net v0.39.0 // indirect + golang.org/x/sync v0.13.0 // indirect + golang.org/x/sys v0.32.0 // indirect + golang.org/x/text v0.24.0 // indirect + golang.org/x/time v0.11.0 // indirect + golang.org/x/tools v0.32.0 // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20250313205543-e70fdf4c4cb4 // indirect + google.golang.org/grpc v1.71.0 // indirect + google.golang.org/protobuf v1.36.6 // indirect + gopkg.in/ini.v1 v1.67.0 // indirect + gopkg.in/tomb.v2 v2.0.0-20161208151619-d5d1b5820637 // indirect + gopkg.in/yaml.v2 v2.4.0 // indirect + gopkg.in/yaml.v3 v3.0.1 // indirect + honnef.co/go/tools v0.6.1 // indirect + mvdan.cc/gofumpt v0.8.0 // indirect + mvdan.cc/unparam v0.0.0-20250301125049-0df0534333a4 // indirect +) diff --git a/tools/go.sum b/tools/go.sum new file mode 100644 index 0000000..24b66e9 --- /dev/null +++ b/tools/go.sum @@ -0,0 +1,716 @@ +4d63.com/gocheckcompilerdirectives v1.3.0 h1:Ew5y5CtcAAQeTVKUVFrE7EwHMrTO6BggtEj8BZSjZ3A= +4d63.com/gocheckcompilerdirectives v1.3.0/go.mod h1:ofsJ4zx2QAuIP/NO/NAh1ig6R1Fb18/GI7RVMwz7kAY= +4d63.com/gochecknoglobals v0.2.2 h1:H1vdnwnMaZdQW/N+NrkT1SZMTBmcwHe9Vq8lJcYYTtU= +4d63.com/gochecknoglobals v0.2.2/go.mod h1:lLxwTQjL5eIesRbvnzIP3jZtG140FnTdz+AlMa+ogt0= +dario.cat/mergo v1.0.1 h1:Ra4+bf83h2ztPIQYNP99R6m+Y7KfnARDfID+a+vLl4s= +dario.cat/mergo v1.0.1/go.mod h1:uNxQE+84aUszobStD9th8a29P2fMDhsBdgRYvZOxGmk= +github.com/4meepo/tagalign v1.4.2 h1:0hcLHPGMjDyM1gHG58cS73aQF8J4TdVR96TZViorO9E= +github.com/4meepo/tagalign v1.4.2/go.mod h1:+p4aMyFM+ra7nb41CnFG6aSDXqRxU/w1VQqScKqDARI= +github.com/Abirdcfly/dupword v0.1.3 h1:9Pa1NuAsZvpFPi9Pqkd93I7LIYRURj+A//dFd5tgBeE= +github.com/Abirdcfly/dupword v0.1.3/go.mod h1:8VbB2t7e10KRNdwTVoxdBaxla6avbhGzb8sCTygUMhw= +github.com/Antonboom/errname v1.1.0 h1:A+ucvdpMwlo/myWrkHEUEBWc/xuXdud23S8tmTb/oAE= +github.com/Antonboom/errname v1.1.0/go.mod h1:O1NMrzgUcVBGIfi3xlVuvX8Q/VP/73sseCaAppfjqZw= +github.com/Antonboom/nilnil v1.1.0 h1:jGxJxjgYS3VUUtOTNk8Z1icwT5ESpLH/426fjmQG+ng= +github.com/Antonboom/nilnil v1.1.0/go.mod h1:b7sAlogQjFa1wV8jUW3o4PMzDVFLbTux+xnQdvzdcIE= +github.com/Antonboom/testifylint v1.6.1 h1:6ZSytkFWatT8mwZlmRCHkWz1gPi+q6UBSbieji2Gj/o= +github.com/Antonboom/testifylint v1.6.1/go.mod h1:k+nEkathI2NFjKO6HvwmSrbzUcQ6FAnbZV+ZRrnXPLI= +github.com/BurntSushi/toml v1.5.0 h1:W5quZX/G/csjUnuI8SUYlsHs9M38FC7znL0lIO+DvMg= +github.com/BurntSushi/toml v1.5.0/go.mod h1:ukJfTF/6rtPPRCnwkur4qwRxa8vTRFBF0uk2lLoLwho= +github.com/Crocmagnon/fatcontext v0.7.1 h1:SC/VIbRRZQeQWj/TcQBS6JmrXcfA+BU4OGSVUt54PjM= +github.com/Crocmagnon/fatcontext v0.7.1/go.mod h1:1wMvv3NXEBJucFGfwOJBxSVWcoIO6emV215SMkW9MFU= +github.com/Djarvur/go-err113 v0.0.0-20210108212216-aea10b59be24 h1:sHglBQTwgx+rWPdisA5ynNEsoARbiCBOyGcJM4/OzsM= +github.com/Djarvur/go-err113 v0.0.0-20210108212216-aea10b59be24/go.mod h1:4UJr5HIiMZrwgkSPdsjy2uOQExX/WEILpIrO9UPGuXs= +github.com/GaijinEntertainment/go-exhaustruct/v3 v3.3.1 h1:Sz1JIXEcSfhz7fUi7xHnhpIE0thVASYjvosApmHuD2k= +github.com/GaijinEntertainment/go-exhaustruct/v3 v3.3.1/go.mod h1:n/LSCXNuIYqVfBlVXyHfMQkZDdp1/mmxfSjADd3z1Zg= +github.com/Masterminds/goutils v1.1.1 h1:5nUrii3FMTL5diU80unEVvNevw1nH4+ZV4DSLVJLSYI= +github.com/Masterminds/goutils v1.1.1/go.mod h1:8cTjp+g8YejhMuvIA5y2vz3BpJxksy863GQaJW2MFNU= +github.com/Masterminds/semver/v3 v3.3.1 h1:QtNSWtVZ3nBfk8mAOu/B6v7FMJ+NHTIgUPi7rj+4nv4= +github.com/Masterminds/semver/v3 v3.3.1/go.mod h1:4V+yj/TJE1HU9XfppCwVMZq3I84lprf4nC11bSS5beM= +github.com/Masterminds/sprig/v3 v3.3.0 h1:mQh0Yrg1XPo6vjYXgtf5OtijNAKJRNcTdOOGZe3tPhs= +github.com/Masterminds/sprig/v3 v3.3.0/go.mod h1:Zy1iXRYNqNLUolqCpL4uhk6SHUMAOSCzdgBfDb35Lz0= +github.com/OpenPeeDeeP/depguard/v2 v2.2.1 h1:vckeWVESWp6Qog7UZSARNqfu/cZqvki8zsuj3piCMx4= +github.com/OpenPeeDeeP/depguard/v2 v2.2.1/go.mod h1:q4DKzC4UcVaAvcfd41CZh0PWpGgzrVxUYBlgKNGquUo= +github.com/alecthomas/assert/v2 v2.11.0 h1:2Q9r3ki8+JYXvGsDyBXwH3LcJ+WK5D0gc5E8vS6K3D0= +github.com/alecthomas/assert/v2 v2.11.0/go.mod h1:Bze95FyfUr7x34QZrjL+XP+0qgp/zg8yS+TtBj1WA3k= +github.com/alecthomas/chroma/v2 v2.17.2 h1:Rm81SCZ2mPoH+Q8ZCc/9YvzPUN/E7HgPiPJD8SLV6GI= +github.com/alecthomas/chroma/v2 v2.17.2/go.mod h1:RVX6AvYm4VfYe/zsk7mjHueLDZor3aWCNE14TFlepBk= +github.com/alecthomas/go-check-sumtype v0.3.1 h1:u9aUvbGINJxLVXiFvHUlPEaD7VDULsrxJb4Aq31NLkU= +github.com/alecthomas/go-check-sumtype v0.3.1/go.mod h1:A8TSiN3UPRw3laIgWEUOHHLPa6/r9MtoigdlP5h3K/E= +github.com/alecthomas/repr v0.4.0 h1:GhI2A8MACjfegCPVq9f1FLvIBS+DrQ2KQBFZP1iFzXc= +github.com/alecthomas/repr v0.4.0/go.mod h1:Fr0507jx4eOXV7AlPV6AVZLYrLIuIeSOWtW57eE/O/4= +github.com/alexkohler/nakedret/v2 v2.0.6 h1:ME3Qef1/KIKr3kWX3nti3hhgNxw6aqN5pZmQiFSsuzQ= +github.com/alexkohler/nakedret/v2 v2.0.6/go.mod h1:l3RKju/IzOMQHmsEvXwkqMDzHHvurNQfAgE1eVmT40Q= +github.com/alexkohler/prealloc v1.0.0 h1:Hbq0/3fJPQhNkN0dR95AVrr6R7tou91y0uHG5pOcUuw= +github.com/alexkohler/prealloc v1.0.0/go.mod h1:VetnK3dIgFBBKmg0YnD9F9x6Icjd+9cvfHR56wJVlKE= +github.com/alingse/asasalint v0.0.11 h1:SFwnQXJ49Kx/1GghOFz1XGqHYKp21Kq1nHad/0WQRnw= +github.com/alingse/asasalint v0.0.11/go.mod h1:nCaoMhw7a9kSJObvQyVzNTPBDbNpdocqrSP7t/cW5+I= +github.com/alingse/nilnesserr v0.2.0 h1:raLem5KG7EFVb4UIDAXgrv3N2JIaffeKNtcEXkEWd/w= +github.com/alingse/nilnesserr v0.2.0/go.mod h1:1xJPrXonEtX7wyTq8Dytns5P2hNzoWymVUIaKm4HNFg= +github.com/ashanbrown/forbidigo v1.6.0 h1:D3aewfM37Yb3pxHujIPSpTf6oQk9sc9WZi8gerOIVIY= +github.com/ashanbrown/forbidigo v1.6.0/go.mod h1:Y8j9jy9ZYAEHXdu723cUlraTqbzjKF1MUyfOKL+AjcU= +github.com/ashanbrown/makezero v1.2.0 h1:/2Lp1bypdmK9wDIq7uWBlDF1iMUpIIS4A+pF6C9IEUU= +github.com/ashanbrown/makezero v1.2.0/go.mod h1:dxlPhHbDMC6N6xICzFBSK+4njQDdK8euNO0qjQMtGY4= +github.com/aymanbagabas/go-osc52/v2 v2.0.1 h1:HwpRHbFMcZLEVr42D4p7XBqjyuxQH5SMiErDT4WkJ2k= +github.com/aymanbagabas/go-osc52/v2 v2.0.1/go.mod h1:uYgXzlJ7ZpABp8OJ+exZzJJhRNQ2ASbcXHWsFqH8hp8= +github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= +github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw= +github.com/bkielbasa/cyclop v1.2.3 h1:faIVMIGDIANuGPWH031CZJTi2ymOQBULs9H21HSMa5w= +github.com/bkielbasa/cyclop v1.2.3/go.mod h1:kHTwA9Q0uZqOADdupvcFJQtp/ksSnytRMe8ztxG8Fuo= +github.com/blizzy78/varnamelen v0.8.0 h1:oqSblyuQvFsW1hbBHh1zfwrKe3kcSj0rnXkKzsQ089M= +github.com/blizzy78/varnamelen v0.8.0/go.mod h1:V9TzQZ4fLJ1DSrjVDfl89H7aMnTvKkApdHeyESmyR7k= +github.com/bombsimon/wsl/v4 v4.7.0 h1:1Ilm9JBPRczjyUs6hvOPKvd7VL1Q++PL8M0SXBDf+jQ= +github.com/bombsimon/wsl/v4 v4.7.0/go.mod h1:uV/+6BkffuzSAVYD+yGyld1AChO7/EuLrCF/8xTiapg= +github.com/breml/bidichk v0.3.3 h1:WSM67ztRusf1sMoqH6/c4OBCUlRVTKq+CbSeo0R17sE= +github.com/breml/bidichk v0.3.3/go.mod h1:ISbsut8OnjB367j5NseXEGGgO/th206dVa427kR8YTE= +github.com/breml/errchkjson v0.4.1 h1:keFSS8D7A2T0haP9kzZTi7o26r7kE3vymjZNeNDRDwg= +github.com/breml/errchkjson v0.4.1/go.mod h1:a23OvR6Qvcl7DG/Z4o0el6BRAjKnaReoPQFciAl9U3s= +github.com/bufbuild/protocompile v0.14.1 h1:iA73zAf/fyljNjQKwYzUHD6AD4R8KMasmwa/FBatYVw= +github.com/bufbuild/protocompile v0.14.1/go.mod h1:ppVdAIhbr2H8asPk6k4pY7t9zB1OU5DoEw9xY/FUi1c= +github.com/butuzov/ireturn v0.4.0 h1:+s76bF/PfeKEdbG8b54aCocxXmi0wvYdOVsWxVO7n8E= +github.com/butuzov/ireturn v0.4.0/go.mod h1:ghI0FrCmap8pDWZwfPisFD1vEc56VKH4NpQUxDHta70= +github.com/butuzov/mirror v1.3.0 h1:HdWCXzmwlQHdVhwvsfBb2Au0r3HyINry3bDWLYXiKoc= +github.com/butuzov/mirror v1.3.0/go.mod h1:AEij0Z8YMALaq4yQj9CPPVYOyJQyiexpQEQgihajRfI= +github.com/catenacyber/perfsprint v0.9.1 h1:5LlTp4RwTooQjJCvGEFV6XksZvWE7wCOUvjD2z0vls0= +github.com/catenacyber/perfsprint v0.9.1/go.mod h1:q//VWC2fWbcdSLEY1R3l8n0zQCDPdE4IjZwyY1HMunM= +github.com/ccojocar/zxcvbn-go v1.0.2 h1:na/czXU8RrhXO4EZme6eQJLR4PzcGsahsBOAwU6I3Vg= +github.com/ccojocar/zxcvbn-go v1.0.2/go.mod h1:g1qkXtUSvHP8lhHp5GrSmTz6uWALGRMQdw6Qnz/hi60= +github.com/cespare/xxhash/v2 v2.3.0 h1:UL815xU9SqsFlibzuggzjXhog7bL6oX9BbNZnL2UFvs= +github.com/cespare/xxhash/v2 v2.3.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= +github.com/charithe/durationcheck v0.0.10 h1:wgw73BiocdBDQPik+zcEoBG/ob8uyBHf2iyoHGPf5w4= +github.com/charithe/durationcheck v0.0.10/go.mod h1:bCWXb7gYRysD1CU3C+u4ceO49LoGOY1C1L6uouGNreQ= +github.com/charmbracelet/colorprofile v0.2.3-0.20250311203215-f60798e515dc h1:4pZI35227imm7yK2bGPcfpFEmuY1gc2YSTShr4iJBfs= +github.com/charmbracelet/colorprofile v0.2.3-0.20250311203215-f60798e515dc/go.mod h1:X4/0JoqgTIPSFcRA/P6INZzIuyqdFY5rm8tb41s9okk= +github.com/charmbracelet/lipgloss v1.1.0 h1:vYXsiLHVkK7fp74RkV7b2kq9+zDLoEU4MZoFqR/noCY= +github.com/charmbracelet/lipgloss v1.1.0/go.mod h1:/6Q8FR2o+kj8rz4Dq0zQc3vYf7X+B0binUUBwA0aL30= +github.com/charmbracelet/x/ansi v0.8.0 h1:9GTq3xq9caJW8ZrBTe0LIe2fvfLR/bYXKTx2llXn7xE= +github.com/charmbracelet/x/ansi v0.8.0/go.mod h1:wdYl/ONOLHLIVmQaxbIYEC/cRKOQyjTkowiI4blgS9Q= +github.com/charmbracelet/x/cellbuf v0.0.13-0.20250311204145-2c3ea96c31dd h1:vy0GVL4jeHEwG5YOXDmi86oYw2yuYUGqz6a8sLwg0X8= +github.com/charmbracelet/x/cellbuf v0.0.13-0.20250311204145-2c3ea96c31dd/go.mod h1:xe0nKWGd3eJgtqZRaN9RjMtK7xUYchjzPr7q6kcvCCs= +github.com/charmbracelet/x/term v0.2.1 h1:AQeHeLZ1OqSXhrAWpYUtZyX1T3zVxfpZuEQMIQaGIAQ= +github.com/charmbracelet/x/term v0.2.1/go.mod h1:oQ4enTYFV7QN4m0i9mzHrViD7TQKvNEEkHUMCmsxdUg= +github.com/chavacava/garif v0.1.0 h1:2JHa3hbYf5D9dsgseMKAmc/MZ109otzgNFk5s87H9Pc= +github.com/chavacava/garif v0.1.0/go.mod h1:XMyYCkEL58DF0oyW4qDjjnPWONs2HBqYKI+UIPD+Gww= +github.com/ckaznocha/intrange v0.3.1 h1:j1onQyXvHUsPWujDH6WIjhyH26gkRt/txNlV7LspvJs= +github.com/ckaznocha/intrange v0.3.1/go.mod h1:QVepyz1AkUoFQkpEqksSYpNpUo3c5W7nWh/s6SHIJJk= +github.com/conduitio/conduit-commons v0.5.2 h1:Y56zXj1StoLVpwL5HELUhSmrHfFgSwJuGf+AQrdgVK8= +github.com/conduitio/conduit-commons v0.5.2/go.mod h1:hwVOkrOGbmI9DR4ZXail2f02lculzq1XRAUcC04tkmE= +github.com/conduitio/conduit-connector-protocol v0.9.2 h1:QYZQCYrbKz6Vbtv5KWfa4d7YQSoJq3AEnV1xVWnHk8Y= +github.com/conduitio/conduit-connector-protocol v0.9.2/go.mod h1:bgQ3cs1lGUWmO/TBxtP+mZIy4WL3vPSGEK7BVC5TEUg= +github.com/conduitio/conduit-connector-sdk v0.13.3 h1:wYkAMeIM02Djrat3bAF4Y5+U8rHjvMIGH6ALyM4jnaA= +github.com/conduitio/conduit-connector-sdk v0.13.3/go.mod h1:P4RwDuBmsrKi5lgRuryuufRAJMZJgUP3MHeIGqzoxuM= +github.com/conduitio/evolviconf v0.1.0 h1:rcG+hs6tlrYlX9qomOQJz+K+OnDhbMbioGx3ci55yo0= +github.com/conduitio/evolviconf v0.1.0/go.mod h1:RnbnSqDDYarKgG2p+krP71svG6qLms3+/TnKrPKWk+0= +github.com/conduitio/evolviconf/evolviyaml v0.1.0 h1:nMW7CROIMtHhscm/QLMpMs7uCPp6O2dS4CfU9bhugd4= +github.com/conduitio/evolviconf/evolviyaml v0.1.0/go.mod h1:22+FHPuroT5pPZpg0fuhE8ACIMCl1S+HsAFN1CM3Vho= +github.com/conduitio/yaml/v3 v3.3.0 h1:kbbaOSHcuH39gP4+rgbJGl6DSbLZcJgEaBvkEXJlCsI= +github.com/conduitio/yaml/v3 v3.3.0/go.mod h1:JNgFMOX1t8W4YJuRZOh6GggVtSMsgP9XgTw+7dIenpc= +github.com/coreos/go-systemd/v22 v22.5.0/go.mod h1:Y58oyj3AT4RCenI/lSvhwexgC+NSVTIJ3seZv2GcEnc= +github.com/cpuguy83/go-md2man/v2 v2.0.6/go.mod h1:oOW0eioCTA6cOiMLiUPZOpcVxMig6NIQQ7OS05n1F4g= +github.com/curioswitch/go-reassign v0.3.0 h1:dh3kpQHuADL3cobV/sSGETA8DOv457dwl+fbBAhrQPs= +github.com/curioswitch/go-reassign v0.3.0/go.mod h1:nApPCCTtqLJN/s8HfItCcKV0jIPwluBOvZP+dsJGA88= +github.com/daixiang0/gci v0.13.6 h1:RKuEOSkGpSadkGbvZ6hJ4ddItT3cVZ9Vn9Rybk6xjl8= +github.com/daixiang0/gci v0.13.6/go.mod h1:12etP2OniiIdP4q+kjUGrC/rUagga7ODbqsom5Eo5Yk= +github.com/dave/dst v0.27.3 h1:P1HPoMza3cMEquVf9kKy8yXsFirry4zEnWOdYPOoIzY= +github.com/dave/dst v0.27.3/go.mod h1:jHh6EOibnHgcUW3WjKHisiooEkYwqpHLBSX1iOBhEyc= +github.com/dave/jennifer v1.7.1 h1:B4jJJDHelWcDhlRQxWeo0Npa/pYKBLrirAQoTN45txo= +github.com/dave/jennifer v1.7.1/go.mod h1:nXbxhEmQfOZhWml3D1cDK5M1FLnMSozpbFN/m3RmGZc= +github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= +github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= +github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc h1:U9qPSI2PIWSS1VwoXQT9A3Wy9MM3WgvqSxFWenqJduM= +github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= +github.com/denis-tingaikin/go-header v0.5.0 h1:SRdnP5ZKvcO9KKRP1KJrhFR3RrlGuD+42t4429eC9k8= +github.com/denis-tingaikin/go-header v0.5.0/go.mod h1:mMenU5bWrok6Wl2UsZjy+1okegmwQ3UgWl4V1D8gjlY= +github.com/dlclark/regexp2 v1.11.5 h1:Q/sSnsKerHeCkc/jSTNq1oCm7KiVgUMZRDUoRu0JQZQ= +github.com/dlclark/regexp2 v1.11.5/go.mod h1:DHkYz0B9wPfa6wondMfaivmHpzrQ3v9q8cnmRbL6yW8= +github.com/ettle/strcase v0.2.0 h1:fGNiVF21fHXpX1niBgk0aROov1LagYsOwV/xqKDKR/Q= +github.com/ettle/strcase v0.2.0/go.mod h1:DajmHElDSaX76ITe3/VHVyMin4LWSJN5Z909Wp+ED1A= +github.com/fatih/color v1.13.0/go.mod h1:kLAiJbzzSOZDVNGyDpeOxJ47H46qBXwg5ILebYFFOfk= +github.com/fatih/color v1.18.0 h1:S8gINlzdQ840/4pfAwic/ZE0djQEH3wM94VfqLTZcOM= +github.com/fatih/color v1.18.0/go.mod h1:4FelSpRwEGDpQ12mAdzqdOukCy4u8WUtOY6lkT/6HfU= +github.com/fatih/structtag v1.2.0 h1:/OdNE99OxoI/PqaW/SuSK9uxxT3f/tcSZgon/ssNSx4= +github.com/fatih/structtag v1.2.0/go.mod h1:mBJUNpUnHmRKrKlQQlmCrh5PuhftFbNv8Ys4/aAZl94= +github.com/firefart/nonamedreturns v1.0.6 h1:vmiBcKV/3EqKY3ZiPxCINmpS431OcE1S47AQUwhrg8E= +github.com/firefart/nonamedreturns v1.0.6/go.mod h1:R8NisJnSIpvPWheCq0mNRXJok6D8h7fagJTF8EMEwCo= +github.com/frankban/quicktest v1.14.6 h1:7Xjx+VpznH+oBnejlPUj8oUpdxnVs4f8XU8WnHkI4W8= +github.com/frankban/quicktest v1.14.6/go.mod h1:4ptaffx2x8+WTWXmUCuVU6aPUX1/Mz7zb5vbUoiM6w0= +github.com/fsnotify/fsnotify v1.7.0 h1:8JEhPFa5W2WU7YfeZzPNqzMP6Lwt7L2715Ggo0nosvA= +github.com/fsnotify/fsnotify v1.7.0/go.mod h1:40Bi/Hjc2AVfZrqy+aj+yEI+/bRxZnMJyTJwOpGvigM= +github.com/fzipp/gocyclo v0.6.0 h1:lsblElZG7d3ALtGMx9fmxeTKZaLLpU8mET09yN4BBLo= +github.com/fzipp/gocyclo v0.6.0/go.mod h1:rXPyn8fnlpa0R2csP/31uerbiVBugk5whMdlyaLkLoA= +github.com/ghostiam/protogetter v0.3.15 h1:1KF5sXel0HE48zh1/vn0Loiw25A9ApyseLzQuif1mLY= +github.com/ghostiam/protogetter v0.3.15/go.mod h1:WZ0nw9pfzsgxuRsPOFQomgDVSWtDLJRfQJEhsGbmQMA= +github.com/go-critic/go-critic v0.13.0 h1:kJzM7wzltQasSUXtYyTl6UaPVySO6GkaR1thFnJ6afY= +github.com/go-critic/go-critic v0.13.0/go.mod h1:M/YeuJ3vOCQDnP2SU+ZhjgRzwzcBW87JqLpMJLrZDLI= +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-quicktest/qt v1.101.0 h1:O1K29Txy5P2OK0dGo59b7b0LR6wKfIhttaAhHUyn7eI= +github.com/go-quicktest/qt v1.101.0/go.mod h1:14Bz/f7NwaXPtdYEgzsx46kqSxVwTbzVZsDC26tQJow= +github.com/go-task/slim-sprig/v3 v3.0.0 h1:sUs3vkvUymDpBKi3qH1YSqBQk9+9D/8M2mN1vB6EwHI= +github.com/go-task/slim-sprig/v3 v3.0.0/go.mod h1:W848ghGpv3Qj3dhTPRyJypKRiqCdHZiAzKg9hl15HA8= +github.com/go-toolsmith/astcast v1.1.0 h1:+JN9xZV1A+Re+95pgnMgDboWNVnIMMQXwfBwLRPgSC8= +github.com/go-toolsmith/astcast v1.1.0/go.mod h1:qdcuFWeGGS2xX5bLM/c3U9lewg7+Zu4mr+xPwZIB4ZU= +github.com/go-toolsmith/astcopy v1.1.0 h1:YGwBN0WM+ekI/6SS6+52zLDEf8Yvp3n2seZITCUBt5s= +github.com/go-toolsmith/astcopy v1.1.0/go.mod h1:hXM6gan18VA1T/daUEHCFcYiW8Ai1tIwIzHY6srfEAw= +github.com/go-toolsmith/astequal v1.0.3/go.mod h1:9Ai4UglvtR+4up+bAD4+hCj7iTo4m/OXVTSLnCyTAx4= +github.com/go-toolsmith/astequal v1.1.0/go.mod h1:sedf7VIdCL22LD8qIvv7Nn9MuWJruQA/ysswh64lffQ= +github.com/go-toolsmith/astequal v1.2.0 h1:3Fs3CYZ1k9Vo4FzFhwwewC3CHISHDnVUPC4x0bI2+Cw= +github.com/go-toolsmith/astequal v1.2.0/go.mod h1:c8NZ3+kSFtFY/8lPso4v8LuJjdJiUFVnSuU3s0qrrDY= +github.com/go-toolsmith/astfmt v1.1.0 h1:iJVPDPp6/7AaeLJEruMsBUlOYCmvg0MoCfJprsOmcco= +github.com/go-toolsmith/astfmt v1.1.0/go.mod h1:OrcLlRwu0CuiIBp/8b5PYF9ktGVZUjlNMV634mhwuQ4= +github.com/go-toolsmith/astp v1.1.0 h1:dXPuCl6u2llURjdPLLDxJeZInAeZ0/eZwFJmqZMnpQA= +github.com/go-toolsmith/astp v1.1.0/go.mod h1:0T1xFGz9hicKs8Z5MfAqSUitoUYS30pDMsRVIDHs8CA= +github.com/go-toolsmith/pkgload v1.2.2 h1:0CtmHq/02QhxcF7E9N5LIFcYFsMR5rdovfqTtRKkgIk= +github.com/go-toolsmith/pkgload v1.2.2/go.mod h1:R2hxLNRKuAsiXCo2i5J6ZQPhnPMOVtU+f0arbFPWCus= +github.com/go-toolsmith/strparse v1.0.0/go.mod h1:YI2nUKP9YGZnL/L1/DLFBfixrcjslWct4wyljWhSRy8= +github.com/go-toolsmith/strparse v1.1.0 h1:GAioeZUK9TGxnLS+qfdqNbA4z0SSm5zVNtCQiyP2Bvw= +github.com/go-toolsmith/strparse v1.1.0/go.mod h1:7ksGy58fsaQkGQlY8WVoBFNyEPMGuJin1rfoPS4lBSQ= +github.com/go-toolsmith/typep v1.1.0 h1:fIRYDyF+JywLfqzyhdiHzRop/GQDxxNhLGQ6gFUNHus= +github.com/go-toolsmith/typep v1.1.0/go.mod h1:fVIw+7zjdsMxDA3ITWnH1yOiw1rnTQKCsF/sk2H/qig= +github.com/go-viper/mapstructure/v2 v2.2.1 h1:ZAaOCxANMuZx5RCeg0mBdEZk7DZasvvZIxtHqx8aGss= +github.com/go-viper/mapstructure/v2 v2.2.1/go.mod h1:oJDH3BJKyqBA2TXFhDsKDGDTlndYOZ6rGS0BRZIxGhM= +github.com/go-xmlfmt/xmlfmt v1.1.3 h1:t8Ey3Uy7jDSEisW2K3somuMKIpzktkWptA0iFCnRUWY= +github.com/go-xmlfmt/xmlfmt v1.1.3/go.mod h1:aUCEOzzezBEjDBbFBoSiya/gduyIiWYRP6CnSFIV8AM= +github.com/gobwas/glob v0.2.3 h1:A4xDbljILXROh+kObIiy5kIaPYD8e96x1tgBhUI5J+Y= +github.com/gobwas/glob v0.2.3/go.mod h1:d3Ez4x06l9bZtSvzIay5+Yzi0fmZzPgnTbPcKjJAkT8= +github.com/goccy/go-json v0.10.5 h1:Fq85nIqj+gXn/S5ahsiTlK3TmC85qgirsdTP/+DeaC4= +github.com/goccy/go-json v0.10.5/go.mod h1:oq7eo15ShAhp70Anwd5lgX2pLfOS3QCiwU/PULtXL6M= +github.com/godbus/dbus/v5 v5.0.4/go.mod h1:xhWf0FNVPg57R7Z0UbKHbJfkEywrmjJnf7w5xrFpKfA= +github.com/gofrs/flock v0.12.1 h1:MTLVXXHf8ekldpJk3AKicLij9MdwOWkZ+a/jHHZby9E= +github.com/gofrs/flock v0.12.1/go.mod h1:9zxTsyu5xtJ9DK+1tFZyibEV7y3uwDxPPfbxeeHCoD0= +github.com/golang/protobuf v1.5.4 h1:i7eJL8qZTpSEXOPTxNKhASYpMn+8e5Q6AdndVa1dWek= +github.com/golang/protobuf v1.5.4/go.mod h1:lnTiLA8Wa4RWRcIUkrtSVa5nRhsEGBg48fD6rSs7xps= +github.com/golangci/dupl v0.0.0-20250308024227-f665c8d69b32 h1:WUvBfQL6EW/40l6OmeSBYQJNSif4O11+bmWEz+C7FYw= +github.com/golangci/dupl v0.0.0-20250308024227-f665c8d69b32/go.mod h1:NUw9Zr2Sy7+HxzdjIULge71wI6yEg1lWQr7Evcu8K0E= +github.com/golangci/go-printf-func-name v0.1.0 h1:dVokQP+NMTO7jwO4bwsRwLWeudOVUPPyAKJuzv8pEJU= +github.com/golangci/go-printf-func-name v0.1.0/go.mod h1:wqhWFH5mUdJQhweRnldEywnR5021wTdZSNgwYceV14s= +github.com/golangci/gofmt v0.0.0-20250106114630-d62b90e6713d h1:viFft9sS/dxoYY0aiOTsLKO2aZQAPT4nlQCsimGcSGE= +github.com/golangci/gofmt v0.0.0-20250106114630-d62b90e6713d/go.mod h1:ivJ9QDg0XucIkmwhzCDsqcnxxlDStoTl89jDMIoNxKY= +github.com/golangci/golangci-lint v1.64.8 h1:y5TdeVidMtBGG32zgSC7ZXTFNHrsJkDnpO4ItB3Am+I= +github.com/golangci/golangci-lint v1.64.8/go.mod h1:5cEsUQBSr6zi8XI8OjmcY2Xmliqc4iYL7YoPrL+zLJ4= +github.com/golangci/golangci-lint/v2 v2.1.6 h1:LXqShFfAGM5BDzEOWD2SL1IzJAgUOqES/HRBsfKjI+w= +github.com/golangci/golangci-lint/v2 v2.1.6/go.mod h1:EPj+fgv4TeeBq3TcqaKZb3vkiV5dP4hHHKhXhEhzci8= +github.com/golangci/golines v0.0.0-20250217134842-442fd0091d95 h1:AkK+w9FZBXlU/xUmBtSJN1+tAI4FIvy5WtnUnY8e4p8= +github.com/golangci/golines v0.0.0-20250217134842-442fd0091d95/go.mod h1:k9mmcyWKSTMcPPvQUCfRWWQ9VHJ1U9Dc0R7kaXAgtnQ= +github.com/golangci/misspell v0.6.0 h1:JCle2HUTNWirNlDIAUO44hUsKhOFqGPoC4LZxlaSXDs= +github.com/golangci/misspell v0.6.0/go.mod h1:keMNyY6R9isGaSAu+4Q8NMBwMPkh15Gtc8UCVoDtAWo= +github.com/golangci/plugin-module-register v0.1.1 h1:TCmesur25LnyJkpsVrupv1Cdzo+2f7zX0H6Jkw1Ol6c= +github.com/golangci/plugin-module-register v0.1.1/go.mod h1:TTpqoB6KkwOJMV8u7+NyXMrkwwESJLOkfl9TxR1DGFc= +github.com/golangci/revgrep v0.8.0 h1:EZBctwbVd0aMeRnNUsFogoyayvKHyxlV3CdUA46FX2s= +github.com/golangci/revgrep v0.8.0/go.mod h1:U4R/s9dlXZsg8uJmaR1GrloUr14D7qDl8gi2iPXJH8k= +github.com/golangci/unconvert v0.0.0-20250410112200-a129a6e6413e h1:gD6P7NEo7Eqtt0ssnqSJNNndxe69DOQ24A5h7+i3KpM= +github.com/golangci/unconvert v0.0.0-20250410112200-a129a6e6413e/go.mod h1:h+wZwLjUTJnm/P2rwlbJdRPZXOzaT36/FwnPnY2inzc= +github.com/google/go-cmp v0.5.1/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.5.2/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.5.4/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.5.6/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.5.8/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= +github.com/google/go-cmp v0.7.0 h1:wk8382ETsv4JYUZwIsn6YpYiWiBsYLSJiTsyBybVuN8= +github.com/google/go-cmp v0.7.0/go.mod h1:pXiqmnSA92OHEEa9HXL2W4E7lf9JzCmGVUdgjX3N/iU= +github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= +github.com/google/pprof v0.0.0-20250302191652-9094ed2288e7 h1:+J3r2e8+RsmN3vKfo75g0YSY61ms37qzPglu4p0sGro= +github.com/google/pprof v0.0.0-20250302191652-9094ed2288e7/go.mod h1:vavhavw2zAxS5dIdcRluK6cSGGPlZynqzFM8NdvU144= +github.com/google/uuid v1.6.0 h1:NIvaJDMOsjHA8n1jAhLSgzrAzy1Hgr+hNrb57e+94F0= +github.com/google/uuid v1.6.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= +github.com/gordonklaus/ineffassign v0.1.0 h1:y2Gd/9I7MdY1oEIt+n+rowjBNDcLQq3RsH5hwJd0f9s= +github.com/gordonklaus/ineffassign v0.1.0/go.mod h1:Qcp2HIAYhR7mNUVSIxZww3Guk4it82ghYcEXIAk+QT0= +github.com/gostaticanalysis/analysisutil v0.7.1 h1:ZMCjoue3DtDWQ5WyU16YbjbQEQ3VuzwxALrpYd+HeKk= +github.com/gostaticanalysis/analysisutil v0.7.1/go.mod h1:v21E3hY37WKMGSnbsw2S/ojApNWb6C1//mXO48CXbVc= +github.com/gostaticanalysis/comment v1.4.1/go.mod h1:ih6ZxzTHLdadaiSnF5WY3dxUoXfXAlTaRzuaNDlSado= +github.com/gostaticanalysis/comment v1.4.2/go.mod h1:KLUTGDv6HOCotCH8h2erHKmpci2ZoR8VPu34YA2uzdM= +github.com/gostaticanalysis/comment v1.5.0 h1:X82FLl+TswsUMpMh17srGRuKaaXprTaytmEpgnKIDu8= +github.com/gostaticanalysis/comment v1.5.0/go.mod h1:V6eb3gpCv9GNVqb6amXzEUX3jXLVK/AdA+IrAMSqvEc= +github.com/gostaticanalysis/forcetypeassert v0.2.0 h1:uSnWrrUEYDr86OCxWa4/Tp2jeYDlogZiZHzGkWFefTk= +github.com/gostaticanalysis/forcetypeassert v0.2.0/go.mod h1:M5iPavzE9pPqWyeiVXSFghQjljW1+l/Uke3PXHS6ILY= +github.com/gostaticanalysis/nilerr v0.1.1 h1:ThE+hJP0fEp4zWLkWHWcRyI2Od0p7DlgYG3Uqrmrcpk= +github.com/gostaticanalysis/nilerr v0.1.1/go.mod h1:wZYb6YI5YAxxq0i1+VJbY0s2YONW0HU0GPE3+5PWN4A= +github.com/gostaticanalysis/testutil v0.3.1-0.20210208050101-bfb5c8eec0e4/go.mod h1:D+FIZ+7OahH3ePw/izIEeH5I06eKs1IKI4Xr64/Am3M= +github.com/gostaticanalysis/testutil v0.5.0 h1:Dq4wT1DdTwTGCQQv3rl3IvD5Ld0E6HiY+3Zh0sUGqw8= +github.com/gostaticanalysis/testutil v0.5.0/go.mod h1:OLQSbuM6zw2EvCcXTz1lVq5unyoNft372msDY0nY5Hs= +github.com/hamba/avro/v2 v2.28.0 h1:E8J5D27biyAulWKNiEBhV85QPc9xRMCUCGJewS0KYCE= +github.com/hamba/avro/v2 v2.28.0/go.mod h1:9TVrlt1cG1kkTUtm9u2eO5Qb7rZXlYzoKqPt8TSH+TA= +github.com/hashicorp/go-hclog v1.6.3 h1:Qr2kF+eVWjTiYmU7Y31tYlP1h0q/X3Nl3tPGdaB11/k= +github.com/hashicorp/go-hclog v1.6.3/go.mod h1:W4Qnvbt70Wk/zYJryRzDRU/4r0kIg0PVHBcfoyhpF5M= +github.com/hashicorp/go-immutable-radix/v2 v2.1.0 h1:CUW5RYIcysz+D3B+l1mDeXrQ7fUvGGCwJfdASSzbrfo= +github.com/hashicorp/go-immutable-radix/v2 v2.1.0/go.mod h1:hgdqLXA4f6NIjRVisM1TJ9aOJVNRqKZj+xDGF6m7PBw= +github.com/hashicorp/go-plugin v1.6.3 h1:xgHB+ZUSYeuJi96WtxEjzi23uh7YQpznjGh0U0UUrwg= +github.com/hashicorp/go-plugin v1.6.3/go.mod h1:MRobyh+Wc/nYy1V4KAXUiYfzxoYhs7V1mlH1Z7iY2h0= +github.com/hashicorp/go-uuid v1.0.3 h1:2gKiV6YVmrJ1i2CKKa9obLvRieoRGviZFL26PcT/Co8= +github.com/hashicorp/go-uuid v1.0.3/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= +github.com/hashicorp/go-version v1.2.1/go.mod h1:fltr4n8CU8Ke44wwGCBoEymUuxUHl09ZGVZPK5anwXA= +github.com/hashicorp/go-version v1.7.0 h1:5tqGy27NaOTB8yJKUZELlFAS/LTKJkrmONwQKeRZfjY= +github.com/hashicorp/go-version v1.7.0/go.mod h1:fltr4n8CU8Ke44wwGCBoEymUuxUHl09ZGVZPK5anwXA= +github.com/hashicorp/golang-lru/v2 v2.0.7 h1:a+bsQ5rvGLjzHuww6tVxozPZFVghXaHOwFs4luLUK2k= +github.com/hashicorp/golang-lru/v2 v2.0.7/go.mod h1:QeFd9opnmA6QUJc5vARoKUSoFhyfM2/ZepoAG6RGpeM= +github.com/hashicorp/hcl v1.0.0 h1:0Anlzjpi4vEasTeNFn2mLJgTSwt0+6sfsiTG8qcWGx4= +github.com/hashicorp/hcl v1.0.0/go.mod h1:E5yfLk+7swimpb2L/Alb/PJmXilQ/rhwaUYs4T20WEQ= +github.com/hashicorp/yamux v0.1.1 h1:yrQxtgseBDrq9Y652vSRDvsKCJKOUD+GzTS4Y0Y8pvE= +github.com/hashicorp/yamux v0.1.1/go.mod h1:CtWFDAQgb7dxtzFs4tWbplKIe2jSi3+5vKbgIO0SLnQ= +github.com/hexops/gotextdiff v1.0.3 h1:gitA9+qJrrTCsiCl7+kh75nPqQt1cx4ZkudSTLoUqJM= +github.com/hexops/gotextdiff v1.0.3/go.mod h1:pSWU5MAI3yDq+fZBTazCSJysOMbxWL1BSow5/V2vxeg= +github.com/huandu/xstrings v1.5.0 h1:2ag3IFq9ZDANvthTwTiqSSZLjDc+BedvHPAp5tJy2TI= +github.com/huandu/xstrings v1.5.0/go.mod h1:y5/lhBue+AyNmUVz9RLU9xbLR0o4KIIExikq4ovT0aE= +github.com/inconshreveable/mousetrap v1.1.0 h1:wN+x4NVGpMsO7ErUn/mUI3vEoE6Jt13X2s0bqwp9tc8= +github.com/inconshreveable/mousetrap v1.1.0/go.mod h1:vpF70FUmC8bwa3OWnCshd2FqLfsEA9PFc4w1p2J65bw= +github.com/jgautheron/goconst v1.8.1 h1:PPqCYp3K/xlOj5JmIe6O1Mj6r1DbkdbLtR3AJuZo414= +github.com/jgautheron/goconst v1.8.1/go.mod h1:A0oxgBCHy55NQn6sYpO7UdnA9p+h7cPtoOZUmvNIako= +github.com/jhump/protoreflect v1.15.1 h1:HUMERORf3I3ZdX05WaQ6MIpd/NJ434hTp5YiKgfCL6c= +github.com/jhump/protoreflect v1.15.1/go.mod h1:jD/2GMKKE6OqX8qTjhADU1e6DShO+gavG9e0Q693nKo= +github.com/jingyugao/rowserrcheck v1.1.1 h1:zibz55j/MJtLsjP1OF4bSdgXxwL1b+Vn7Tjzq7gFzUs= +github.com/jingyugao/rowserrcheck v1.1.1/go.mod h1:4yvlZSDb3IyDTUZJUmpZfm2Hwok+Dtp+nu2qOq+er9c= +github.com/jjti/go-spancheck v0.6.4 h1:Tl7gQpYf4/TMU7AT84MN83/6PutY21Nb9fuQjFTpRRc= +github.com/jjti/go-spancheck v0.6.4/go.mod h1:yAEYdKJ2lRkDA8g7X+oKUHXOWVAXSBJRv04OhF+QUjk= +github.com/jpillora/backoff v1.0.0 h1:uvFg412JmmHBHw7iwprIxkPMI+sGQ4kzOWsMeHnm2EA= +github.com/jpillora/backoff v1.0.0/go.mod h1:J/6gKK9jxlEcS3zixgDgUAsiuZ7yrSoa/FX5e0EB2j4= +github.com/json-iterator/go v1.1.12 h1:PV8peI4a0ysnczrg+LtxykD8LfKY9ML6u2jnxaEnrnM= +github.com/json-iterator/go v1.1.12/go.mod h1:e30LSqwooZae/UwlEbR2852Gd8hjQvJoHmT4TnhNGBo= +github.com/julz/importas v0.2.0 h1:y+MJN/UdL63QbFJHws9BVC5RpA2iq0kpjrFajTGivjQ= +github.com/julz/importas v0.2.0/go.mod h1:pThlt589EnCYtMnmhmRYY/qn9lCf/frPOK+WMx3xiJY= +github.com/karamaru-alpha/copyloopvar v1.2.1 h1:wmZaZYIjnJ0b5UoKDjUHrikcV0zuPyyxI4SVplLd2CI= +github.com/karamaru-alpha/copyloopvar v1.2.1/go.mod h1:nFmMlFNlClC2BPvNaHMdkirmTJxVCY0lhxBtlfOypMM= +github.com/kisielk/errcheck v1.9.0 h1:9xt1zI9EBfcYBvdU1nVrzMzzUPUtPKs9bVSIM3TAb3M= +github.com/kisielk/errcheck v1.9.0/go.mod h1:kQxWMMVZgIkDq7U8xtG/n2juOjbLgZtedi0D+/VL/i8= +github.com/kkHAIKE/contextcheck v1.1.6 h1:7HIyRcnyzxL9Lz06NGhiKvenXq7Zw6Q0UQu/ttjfJCE= +github.com/kkHAIKE/contextcheck v1.1.6/go.mod h1:3dDbMRNBFaq8HFXWC1JyvDSPm43CmE6IuHam8Wr0rkg= +github.com/klauspost/compress v1.18.0 h1:c/Cqfb0r+Yi+JtIEq73FWXVkRonBlf0CRNYc8Zttxdo= +github.com/klauspost/compress v1.18.0/go.mod h1:2Pp+KzxcywXVXMr50+X0Q/Lsb43OQHYWRCY2AiWywWQ= +github.com/kr/pretty v0.3.1 h1:flRD4NNwYAUpkphVc1HcthR4KEIFJ65n8Mw5qdRn3LE= +github.com/kr/pretty v0.3.1/go.mod h1:hoEshYVHaxMs3cyo3Yncou5ZscifuDolrwPKZanG3xk= +github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= +github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= +github.com/kulti/thelper v0.6.3 h1:ElhKf+AlItIu+xGnI990no4cE2+XaSu1ULymV2Yulxs= +github.com/kulti/thelper v0.6.3/go.mod h1:DsqKShOvP40epevkFrvIwkCMNYxMeTNjdWL4dqWHZ6I= +github.com/kunwardeep/paralleltest v1.0.14 h1:wAkMoMeGX/kGfhQBPODT/BL8XhK23ol/nuQ3SwFaUw8= +github.com/kunwardeep/paralleltest v1.0.14/go.mod h1:di4moFqtfz3ToSKxhNjhOZL+696QtJGCFe132CbBLGk= +github.com/lasiar/canonicalheader v1.1.2 h1:vZ5uqwvDbyJCnMhmFYimgMZnJMjwljN5VGY0VKbMXb4= +github.com/lasiar/canonicalheader v1.1.2/go.mod h1:qJCeLFS0G/QlLQ506T+Fk/fWMa2VmBUiEI2cuMK4djI= +github.com/ldez/exptostd v0.4.3 h1:Ag1aGiq2epGePuRJhez2mzOpZ8sI9Gimcb4Sb3+pk9Y= +github.com/ldez/exptostd v0.4.3/go.mod h1:iZBRYaUmcW5jwCR3KROEZ1KivQQp6PHXbDPk9hqJKCQ= +github.com/ldez/gomoddirectives v0.6.1 h1:Z+PxGAY+217f/bSGjNZr/b2KTXcyYLgiWI6geMBN2Qc= +github.com/ldez/gomoddirectives v0.6.1/go.mod h1:cVBiu3AHR9V31em9u2kwfMKD43ayN5/XDgr+cdaFaKs= +github.com/ldez/grignotin v0.9.0 h1:MgOEmjZIVNn6p5wPaGp/0OKWyvq42KnzAt/DAb8O4Ow= +github.com/ldez/grignotin v0.9.0/go.mod h1:uaVTr0SoZ1KBii33c47O1M8Jp3OP3YDwhZCmzT9GHEk= +github.com/ldez/tagliatelle v0.7.1 h1:bTgKjjc2sQcsgPiT902+aadvMjCeMHrY7ly2XKFORIk= +github.com/ldez/tagliatelle v0.7.1/go.mod h1:3zjxUpsNB2aEZScWiZTHrAXOl1x25t3cRmzfK1mlo2I= +github.com/ldez/usetesting v0.4.3 h1:pJpN0x3fMupdTf/IapYjnkhiY1nSTN+pox1/GyBRw3k= +github.com/ldez/usetesting v0.4.3/go.mod h1:eEs46T3PpQ+9RgN9VjpY6qWdiw2/QmfiDeWmdZdrjIQ= +github.com/leonklingele/grouper v1.1.2 h1:o1ARBDLOmmasUaNDesWqWCIFH3u7hoFlM84YrjT3mIY= +github.com/leonklingele/grouper v1.1.2/go.mod h1:6D0M/HVkhs2yRKRFZUoGjeDy7EZTfFBE9gl4kjmIGkA= +github.com/lucasb-eyer/go-colorful v1.2.0 h1:1nnpGOrhyZZuNyfu1QjKiUICQ74+3FNCN69Aj6K7nkY= +github.com/lucasb-eyer/go-colorful v1.2.0/go.mod h1:R4dSotOR9KMtayYi1e77YzuveK+i7ruzyGqttikkLy0= +github.com/macabu/inamedparam v0.2.0 h1:VyPYpOc10nkhI2qeNUdh3Zket4fcZjEWe35poddBCpE= +github.com/macabu/inamedparam v0.2.0/go.mod h1:+Pee9/YfGe5LJ62pYXqB89lJ+0k5bsR8Wgz/C0Zlq3U= +github.com/magiconair/properties v1.8.7 h1:IeQXZAiQcpL9mgcAe1Nu6cX9LLw6ExEHKjN0VQdvPDY= +github.com/magiconair/properties v1.8.7/go.mod h1:Dhd985XPs7jluiymwWYZ0G4Z61jb3vdS329zhj2hYo0= +github.com/manuelarte/funcorder v0.2.1 h1:7QJsw3qhljoZ5rH0xapIvjw31EcQeFbF31/7kQ/xS34= +github.com/manuelarte/funcorder v0.2.1/go.mod h1:BQQ0yW57+PF9ZpjpeJDKOffEsQbxDFKW8F8zSMe/Zd0= +github.com/maratori/testableexamples v1.0.0 h1:dU5alXRrD8WKSjOUnmJZuzdxWOEQ57+7s93SLMxb2vI= +github.com/maratori/testableexamples v1.0.0/go.mod h1:4rhjL1n20TUTT4vdh3RDqSizKLyXp7K2u6HgraZCGzE= +github.com/maratori/testpackage v1.1.1 h1:S58XVV5AD7HADMmD0fNnziNHqKvSdDuEKdPD1rNTU04= +github.com/maratori/testpackage v1.1.1/go.mod h1:s4gRK/ym6AMrqpOa/kEbQTV4Q4jb7WeLZzVhVVVOQMc= +github.com/matoous/godox v1.1.0 h1:W5mqwbyWrwZv6OQ5Z1a/DHGMOvXYCBP3+Ht7KMoJhq4= +github.com/matoous/godox v1.1.0/go.mod h1:jgE/3fUXiTurkdHOLT5WEkThTSuE7yxHv5iWPa80afs= +github.com/matryer/is v1.4.0/go.mod h1:8I/i5uYgLzgsgEloJE1U6xx5HkBQpAZvepWuujKwMRU= +github.com/matryer/is v1.4.1 h1:55ehd8zaGABKLXQUe2awZ99BD/PTc2ls+KV/dXphgEQ= +github.com/matryer/is v1.4.1/go.mod h1:8I/i5uYgLzgsgEloJE1U6xx5HkBQpAZvepWuujKwMRU= +github.com/mattn/go-colorable v0.1.9/go.mod h1:u6P/XSegPjTcexA+o6vUJrdnUu04hMope9wVRipJSqc= +github.com/mattn/go-colorable v0.1.12/go.mod h1:u5H1YNBxpqRaxsYJYSkiCWKzEfiAb1Gb520KVy5xxl4= +github.com/mattn/go-colorable v0.1.13/go.mod h1:7S9/ev0klgBDR4GtXTXX8a3vIGJpMovkB8vQcUbaXHg= +github.com/mattn/go-colorable v0.1.14 h1:9A9LHSqF/7dyVVX6g0U9cwm9pG3kP9gSzcuIPHPsaIE= +github.com/mattn/go-colorable v0.1.14/go.mod h1:6LmQG8QLFO4G5z1gPvYEzlUgJ2wF+stgPZH1UqBm1s8= +github.com/mattn/go-isatty v0.0.12/go.mod h1:cbi8OIDigv2wuxKPP5vlRcQ1OAZbq2CE4Kysco4FUpU= +github.com/mattn/go-isatty v0.0.14/go.mod h1:7GGIvUiUoEMVVmxf/4nioHXj79iQHKdU27kJ6hsGG94= +github.com/mattn/go-isatty v0.0.16/go.mod h1:kYGgaQfpe5nmfYZH+SKPsOc2e4SrIfOl2e/yFXSvRLM= +github.com/mattn/go-isatty v0.0.19/go.mod h1:W+V8PltTTMOvKvAeJH7IuucS94S2C6jfK/D7dTCTo3Y= +github.com/mattn/go-isatty v0.0.20 h1:xfD0iDuEKnDkl03q4limB+vH+GxLEtL/jb4xVJSWWEY= +github.com/mattn/go-isatty v0.0.20/go.mod h1:W+V8PltTTMOvKvAeJH7IuucS94S2C6jfK/D7dTCTo3Y= +github.com/mattn/go-runewidth v0.0.9/go.mod h1:H031xJmbD/WCDINGzjvQ9THkh0rPKHF+m2gUSrubnMI= +github.com/mattn/go-runewidth v0.0.16 h1:E5ScNMtiwvlvB5paMFdw9p4kSQzbXFikJ5SQO6TULQc= +github.com/mattn/go-runewidth v0.0.16/go.mod h1:Jdepj2loyihRzMpdS35Xk/zdY8IAYHsh153qUoGf23w= +github.com/mgechev/revive v1.9.0 h1:8LaA62XIKrb8lM6VsBSQ92slt/o92z5+hTw3CmrvSrM= +github.com/mgechev/revive v1.9.0/go.mod h1:LAPq3+MgOf7GcL5PlWIkHb0PT7XH4NuC2LdWymhb9Mo= +github.com/mitchellh/copystructure v1.2.0 h1:vpKXTN4ewci03Vljg/q9QvCGUDttBOGBIa15WveJJGw= +github.com/mitchellh/copystructure v1.2.0/go.mod h1:qLl+cE2AmVv+CoeAwDPye/v+N2HKCj9FbZEVFJRxO9s= +github.com/mitchellh/go-homedir v1.1.0 h1:lukF9ziXFxDFPkA1vsr5zpc1XuPDn/wFntq5mG+4E0Y= +github.com/mitchellh/go-homedir v1.1.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0= +github.com/mitchellh/mapstructure v1.5.0 h1:jeMsZIYE/09sWLaz43PL7Gy6RuMjD2eJVyuac5Z2hdY= +github.com/mitchellh/mapstructure v1.5.0/go.mod h1:bFUtVrKA4DC2yAKiSyO/QUcy7e+RRV2QTWOzhPopBRo= +github.com/mitchellh/reflectwalk v1.0.2 h1:G2LzWKi524PWgd3mLHV8Y5k7s6XUvT0Gef6zxSIeXaQ= +github.com/mitchellh/reflectwalk v1.0.2/go.mod h1:mSTlrgnPZtwu0c4WaC2kGObEpuNDbx0jmZXqmk4esnw= +github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= +github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd h1:TRLaZ9cD/w8PVh93nsPXa1VrQ6jlwL5oN8l14QlcNfg= +github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= +github.com/modern-go/reflect2 v1.0.2 h1:xBagoLtFs94CBntxluKeaWgTMpvLxC4ur3nMaC9Gz0M= +github.com/modern-go/reflect2 v1.0.2/go.mod h1:yWuevngMOJpCy52FWWMvUC8ws7m/LJsjYzDa0/r8luk= +github.com/moricho/tparallel v0.3.2 h1:odr8aZVFA3NZrNybggMkYO3rgPRcqjeQUlBBFVxKHTI= +github.com/moricho/tparallel v0.3.2/go.mod h1:OQ+K3b4Ln3l2TZveGCywybl68glfLEwFGqvnjok8b+U= +github.com/muesli/termenv v0.16.0 h1:S5AlUN9dENB57rsbnkPyfdGuWIlkmzJjbFf0Tf5FWUc= +github.com/muesli/termenv v0.16.0/go.mod h1:ZRfOIKPFDYQoDFF4Olj7/QJbW60Ol/kL1pU3VfY/Cnk= +github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822 h1:C3w9PqII01/Oq1c1nUAm88MOHcQC9l5mIlSMApZMrHA= +github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822/go.mod h1:+n7T8mK8HuQTcFwEeznm/DIxMOiR9yIdICNftLE1DvQ= +github.com/nakabonne/nestif v0.3.1 h1:wm28nZjhQY5HyYPx+weN3Q65k6ilSBxDb8v5S81B81U= +github.com/nakabonne/nestif v0.3.1/go.mod h1:9EtoZochLn5iUprVDmDjqGKPofoUEBL8U4Ngq6aY7OE= +github.com/nishanths/exhaustive v0.12.0 h1:vIY9sALmw6T/yxiASewa4TQcFsVYZQQRUQJhKRf3Swg= +github.com/nishanths/exhaustive v0.12.0/go.mod h1:mEZ95wPIZW+x8kC4TgC+9YCUgiST7ecevsVDTgc2obs= +github.com/nishanths/predeclared v0.2.2 h1:V2EPdZPliZymNAn79T8RkNApBjMmVKh5XRpLm/w98Vk= +github.com/nishanths/predeclared v0.2.2/go.mod h1:RROzoN6TnGQupbC+lqggsOlcgysk3LMK/HI84Mp280c= +github.com/nunnatsa/ginkgolinter v0.19.1 h1:mjwbOlDQxZi9Cal+KfbEJTCz327OLNfwNvoZ70NJ+c4= +github.com/nunnatsa/ginkgolinter v0.19.1/go.mod h1:jkQ3naZDmxaZMXPWaS9rblH+i+GWXQCaS/JFIWcOH2s= +github.com/oklog/run v1.1.0 h1:GEenZ1cK0+q0+wsJew9qUg/DyD8k3JzYsZAi5gYi2mA= +github.com/oklog/run v1.1.0/go.mod h1:sVPdnTZT1zYwAJeCMu2Th4T21pA3FPOQRfWjQlk7DVU= +github.com/olekukonko/tablewriter v0.0.5 h1:P2Ga83D34wi1o9J6Wh1mRuqd4mF/x/lgBS7N7AbDhec= +github.com/olekukonko/tablewriter v0.0.5/go.mod h1:hPp6KlRPjbx+hW8ykQs1w3UBbZlj6HuIJcUGPhkA7kY= +github.com/onsi/ginkgo/v2 v2.23.3 h1:edHxnszytJ4lD9D5Jjc4tiDkPBZ3siDeJJkUZJJVkp0= +github.com/onsi/ginkgo/v2 v2.23.3/go.mod h1:zXTP6xIp3U8aVuXN8ENK9IXRaTjFnpVB9mGmaSRvxnM= +github.com/onsi/gomega v1.36.3 h1:hID7cr8t3Wp26+cYnfcjR6HpJ00fdogN6dqZ1t6IylU= +github.com/onsi/gomega v1.36.3/go.mod h1:8D9+Txp43QWKhM24yyOBEdpkzN8FvJyAwecBgsU4KU0= +github.com/otiai10/copy v1.2.0/go.mod h1:rrF5dJ5F0t/EWSYODDu4j9/vEeYHMkc8jt0zJChqQWw= +github.com/otiai10/copy v1.14.0 h1:dCI/t1iTdYGtkvCuBG2BgR6KZa83PTclw4U5n2wAllU= +github.com/otiai10/copy v1.14.0/go.mod h1:ECfuL02W+/FkTWZWgQqXPWZgW9oeKCSQ5qVfSc4qc4w= +github.com/otiai10/curr v0.0.0-20150429015615-9b4961190c95/go.mod h1:9qAhocn7zKJG+0mI8eUu6xqkFDYS2kb2saOteoSB3cE= +github.com/otiai10/curr v1.0.0/go.mod h1:LskTG5wDwr8Rs+nNQ+1LlxRjAtTZZjtJW4rMXl6j4vs= +github.com/otiai10/mint v1.3.0/go.mod h1:F5AjcsTsWUqX+Na9fpHb52P8pcRX2CI6A3ctIT91xUo= +github.com/otiai10/mint v1.3.1/go.mod h1:/yxELlJQ0ufhjUwhshSj+wFjZ78CnZ48/1wtmBH1OTc= +github.com/pelletier/go-toml/v2 v2.2.4 h1:mye9XuhQ6gvn5h28+VilKrrPoQVanw5PMw/TB0t5Ec4= +github.com/pelletier/go-toml/v2 v2.2.4/go.mod h1:2gIqNv+qfxSVS7cM2xJQKtLSTLUE9V8t9Stt+h56mCY= +github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= +github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= +github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2 h1:Jamvg5psRIccs7FGNTlIRMkT8wgtp5eCXdBlqhYGL6U= +github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= +github.com/polyfloyd/go-errorlint v1.8.0 h1:DL4RestQqRLr8U4LygLw8g2DX6RN1eBJOpa2mzsrl1Q= +github.com/polyfloyd/go-errorlint v1.8.0/go.mod h1:G2W0Q5roxbLCt0ZQbdoxQxXktTjwNyDbEaj3n7jvl4s= +github.com/prashantv/gostub v1.1.0 h1:BTyx3RfQjRHnUWaGF9oQos79AlQ5k8WNktv7VGvVH4g= +github.com/prashantv/gostub v1.1.0/go.mod h1:A5zLQHz7ieHGG7is6LLXLz7I8+3LZzsrV0P1IAHhP5U= +github.com/prometheus/client_golang v1.20.2 h1:5ctymQzZlyOON1666svgwn3s6IKWgfbjsejTMiXIyjg= +github.com/prometheus/client_golang v1.20.2/go.mod h1:PIEt8X02hGcP8JWbeHyeZ53Y/jReSnHgO035n//V5WE= +github.com/prometheus/client_model v0.6.1 h1:ZKSh/rekM+n3CeS952MLRAdFwIKqeY8b62p8ais2e9E= +github.com/prometheus/client_model v0.6.1/go.mod h1:OrxVMOVHjw3lKMa8+x6HeMGkHMQyHDk9E3jmP2AmGiY= +github.com/prometheus/common v0.55.0 h1:KEi6DK7lXW/m7Ig5i47x0vRzuBsHuvJdi5ee6Y3G1dc= +github.com/prometheus/common v0.55.0/go.mod h1:2SECS4xJG1kd8XF9IcM1gMX6510RAEL65zxzNImwdc8= +github.com/prometheus/procfs v0.15.1 h1:YagwOFzUgYfKKHX6Dr+sHT7km/hxC76UB0learggepc= +github.com/prometheus/procfs v0.15.1/go.mod h1:fB45yRUv8NstnjriLhBQLuOUt+WW4BsoGhij/e3PBqk= +github.com/quasilyte/go-ruleguard v0.4.4 h1:53DncefIeLX3qEpjzlS1lyUmQoUEeOWPFWqaTJq9eAQ= +github.com/quasilyte/go-ruleguard v0.4.4/go.mod h1:Vl05zJ538vcEEwu16V/Hdu7IYZWyKSwIy4c88Ro1kRE= +github.com/quasilyte/go-ruleguard/dsl v0.3.22 h1:wd8zkOhSNr+I+8Qeciml08ivDt1pSXe60+5DqOpCjPE= +github.com/quasilyte/go-ruleguard/dsl v0.3.22/go.mod h1:KeCP03KrjuSO0H1kTuZQCWlQPulDV6YMIXmpQss17rU= +github.com/quasilyte/gogrep v0.5.0 h1:eTKODPXbI8ffJMN+W2aE0+oL0z/nh8/5eNdiO34SOAo= +github.com/quasilyte/gogrep v0.5.0/go.mod h1:Cm9lpz9NZjEoL1tgZ2OgeUKPIxL1meE7eo60Z6Sk+Ng= +github.com/quasilyte/regex/syntax v0.0.0-20210819130434-b3f0c404a727 h1:TCg2WBOl980XxGFEZSS6KlBGIV0diGdySzxATTWoqaU= +github.com/quasilyte/regex/syntax v0.0.0-20210819130434-b3f0c404a727/go.mod h1:rlzQ04UMyJXu/aOvhd8qT+hvDrFpiwqp8MRXDY9szc0= +github.com/quasilyte/stdinfo v0.0.0-20220114132959-f7386bf02567 h1:M8mH9eK4OUR4lu7Gd+PU1fV2/qnDNfzT635KRSObncs= +github.com/quasilyte/stdinfo v0.0.0-20220114132959-f7386bf02567/go.mod h1:DWNGW8A4Y+GyBgPuaQJuWiy0XYftx4Xm/y5Jqk9I6VQ= +github.com/raeperd/recvcheck v0.2.0 h1:GnU+NsbiCqdC2XX5+vMZzP+jAJC5fht7rcVTAhX74UI= +github.com/raeperd/recvcheck v0.2.0/go.mod h1:n04eYkwIR0JbgD73wT8wL4JjPC3wm0nFtzBnWNocnYU= +github.com/rivo/uniseg v0.2.0/go.mod h1:J6wj4VEh+S6ZtnVlnTBMWIodfgj8LQOQFoIToxlJtxc= +github.com/rivo/uniseg v0.4.7 h1:WUdvkW8uEhrYfLC4ZzdpI2ztxP1I582+49Oc5Mq64VQ= +github.com/rivo/uniseg v0.4.7/go.mod h1:FN3SvrM+Zdj16jyLfmOkMNblXMcoc8DfTHruCPUcx88= +github.com/rogpeppe/go-internal v1.14.1 h1:UQB4HGPB6osV0SQTLymcB4TgvyWu6ZyliaW0tI/otEQ= +github.com/rogpeppe/go-internal v1.14.1/go.mod h1:MaRKkUm5W0goXpeCfT7UZI6fk/L7L7so1lCWt35ZSgc= +github.com/rs/xid v1.5.0/go.mod h1:trrq9SKmegXys3aeAKXMUTdJsYXVwGY3RLcfgqegfbg= +github.com/rs/zerolog v1.33.0 h1:1cU2KZkvPxNyfgEmhHAz/1A9Bz+llsdYzklWFzgp0r8= +github.com/rs/zerolog v1.33.0/go.mod h1:/7mN4D5sKwJLZQ2b/znpjC3/GQWY/xaDXUM0kKWRHss= +github.com/russross/blackfriday/v2 v2.1.0/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= +github.com/ryancurrah/gomodguard v1.4.1 h1:eWC8eUMNZ/wM/PWuZBv7JxxqT5fiIKSIyTvjb7Elr+g= +github.com/ryancurrah/gomodguard v1.4.1/go.mod h1:qnMJwV1hX9m+YJseXEBhd2s90+1Xn6x9dLz11ualI1I= +github.com/ryanrolds/sqlclosecheck v0.5.1 h1:dibWW826u0P8jNLsLN+En7+RqWWTYrjCB9fJfSfdyCU= +github.com/ryanrolds/sqlclosecheck v0.5.1/go.mod h1:2g3dUjoS6AL4huFdv6wn55WpLIDjY7ZgUR4J8HOO/XQ= +github.com/sagikazarmark/locafero v0.6.0 h1:ON7AQg37yzcRPU69mt7gwhFEBwxI6P9T4Qu3N51bwOk= +github.com/sagikazarmark/locafero v0.6.0/go.mod h1:77OmuIc6VTraTXKXIs/uvUxKGUXjE1GbemJYHqdNjX0= +github.com/sagikazarmark/slog-shim v0.1.0 h1:diDBnUNK9N/354PgrxMywXnAwEr1QZcOr6gto+ugjYE= +github.com/sagikazarmark/slog-shim v0.1.0/go.mod h1:SrcSrq8aKtyuqEI1uvTDTK1arOWRIczQRv+GVI1AkeQ= +github.com/samber/lo v1.47.0 h1:z7RynLwP5nbyRscyvcD043DWYoOcYRv3mV8lBeqOCLc= +github.com/samber/lo v1.47.0/go.mod h1:RmDH9Ct32Qy3gduHQuKJ3gW1fMHAnE/fAzQuf6He5cU= +github.com/samber/slog-common v0.18.1 h1:c0EipD/nVY9HG5shgm/XAs67mgpWDMF+MmtptdJNCkQ= +github.com/samber/slog-common v0.18.1/go.mod h1:QNZiNGKakvrfbJ2YglQXLCZauzkI9xZBjOhWFKS3IKk= +github.com/samber/slog-zerolog/v2 v2.7.3 h1:/MkPDl/tJhijN2GvB1MWwBn2FU8RiL3rQ8gpXkQm2EY= +github.com/samber/slog-zerolog/v2 v2.7.3/go.mod h1:oWU7WHof4Xp8VguiNO02r1a4VzkgoOyOZhY5CuRke60= +github.com/sanposhiho/wastedassign/v2 v2.1.0 h1:crurBF7fJKIORrV85u9UUpePDYGWnwvv3+A96WvwXT0= +github.com/sanposhiho/wastedassign/v2 v2.1.0/go.mod h1:+oSmSC+9bQ+VUAxA66nBb0Z7N8CK7mscKTDYC6aIek4= +github.com/santhosh-tekuri/jsonschema/v6 v6.0.1 h1:PKK9DyHxif4LZo+uQSgXNqs0jj5+xZwwfKHgph2lxBw= +github.com/santhosh-tekuri/jsonschema/v6 v6.0.1/go.mod h1:JXeL+ps8p7/KNMjDQk3TCwPpBy0wYklyWTfbkIzdIFU= +github.com/sashamelentyev/interfacebloat v1.1.0 h1:xdRdJp0irL086OyW1H/RTZTr1h/tMEOsumirXcOJqAw= +github.com/sashamelentyev/interfacebloat v1.1.0/go.mod h1:+Y9yU5YdTkrNvoX0xHc84dxiN1iBi9+G8zZIhPVoNjQ= +github.com/sashamelentyev/usestdlibvars v1.28.0 h1:jZnudE2zKCtYlGzLVreNp5pmCdOxXUzwsMDBkR21cyQ= +github.com/sashamelentyev/usestdlibvars v1.28.0/go.mod h1:9nl0jgOfHKWNFS43Ojw0i7aRoS4j6EBye3YBhmAIRF8= +github.com/securego/gosec/v2 v2.22.3 h1:mRrCNmRF2NgZp4RJ8oJ6yPJ7G4x6OCiAXHd8x4trLRc= +github.com/securego/gosec/v2 v2.22.3/go.mod h1:42M9Xs0v1WseinaB/BmNGO8AVqG8vRfhC2686ACY48k= +github.com/sergi/go-diff v1.2.0 h1:XU+rvMAioB0UC3q1MFrIQy4Vo5/4VsRDQQXHsEya6xQ= +github.com/sergi/go-diff v1.2.0/go.mod h1:STckp+ISIX8hZLjrqAeVduY0gWCT9IjLuqbuNXdaHfM= +github.com/shopspring/decimal v1.4.0 h1:bxl37RwXBklmTi0C79JfXCEBD1cqqHt0bbgBAGFp81k= +github.com/shopspring/decimal v1.4.0/go.mod h1:gawqmDU56v4yIKSwfBSFip1HdCCXN8/+DMd9qYNcwME= +github.com/shurcooL/go v0.0.0-20180423040247-9e1955d9fb6e/go.mod h1:TDJrrUr11Vxrven61rcy3hJMUqaf/CLWYhHNPmT14Lk= +github.com/shurcooL/go-goon v0.0.0-20170922171312-37c2f522c041/go.mod h1:N5mDOmsrJOB+vfqUK+7DmDyjhSLIIBnXo9lvZJj3MWQ= +github.com/sirupsen/logrus v1.9.3 h1:dueUQJ1C2q9oE3F7wvmSGAaVtTmUizReu6fjN8uqzbQ= +github.com/sirupsen/logrus v1.9.3/go.mod h1:naHLuLoDiP4jHNo9R0sCBMtWGeIprob74mVsIT4qYEQ= +github.com/sivchari/containedctx v1.0.3 h1:x+etemjbsh2fB5ewm5FeLNi5bUjK0V8n0RB+Wwfd0XE= +github.com/sivchari/containedctx v1.0.3/go.mod h1:c1RDvCbnJLtH4lLcYD/GqwiBSSf4F5Qk0xld2rBqzJ4= +github.com/sivchari/tenv v1.12.1 h1:+E0QzjktdnExv/wwsnnyk4oqZBUfuh89YMQT1cyuvSY= +github.com/sivchari/tenv v1.12.1/go.mod h1:1LjSOUCc25snIr5n3DtGGrENhX3LuWefcplwVGC24mw= +github.com/sonatard/noctx v0.1.0 h1:JjqOc2WN16ISWAjAk8M5ej0RfExEXtkEyExl2hLW+OM= +github.com/sonatard/noctx v0.1.0/go.mod h1:0RvBxqY8D4j9cTTTWE8ylt2vqj2EPI8fHmrxHdsaZ2c= +github.com/sourcegraph/conc v0.3.0 h1:OQTbbt6P72L20UqAkXXuLOj79LfEanQ+YQFNpLA9ySo= +github.com/sourcegraph/conc v0.3.0/go.mod h1:Sdozi7LEKbFPqYX2/J+iBAM6HpqSLTASQIKqDmF7Mt0= +github.com/sourcegraph/go-diff v0.7.0 h1:9uLlrd5T46OXs5qpp8L/MTltk0zikUGi0sNNyCpA8G0= +github.com/sourcegraph/go-diff v0.7.0/go.mod h1:iBszgVvyxdc8SFZ7gm69go2KDdt3ag071iBaWPF6cjs= +github.com/spf13/afero v1.14.0 h1:9tH6MapGnn/j0eb0yIXiLjERO8RB6xIVZRDCX7PtqWA= +github.com/spf13/afero v1.14.0/go.mod h1:acJQ8t0ohCGuMN3O+Pv0V0hgMxNYDlvdk+VTfyZmbYo= +github.com/spf13/cast v1.7.0 h1:ntdiHjuueXFgm5nzDRdOS4yfT43P5Fnud6DH50rz/7w= +github.com/spf13/cast v1.7.0/go.mod h1:ancEpBxwJDODSW/UG4rDrAqiKolqNNh2DX3mk86cAdo= +github.com/spf13/cobra v1.9.1 h1:CXSaggrXdbHK9CF+8ywj8Amf7PBRmPCOJugH954Nnlo= +github.com/spf13/cobra v1.9.1/go.mod h1:nDyEzZ8ogv936Cinf6g1RU9MRY64Ir93oCnqb9wxYW0= +github.com/spf13/pflag v1.0.5/go.mod h1:McXfInJRrz4CZXVZOBLb0bTZqETkiAhM9Iw0y3An2Bg= +github.com/spf13/pflag v1.0.6 h1:jFzHGLGAlb3ruxLB8MhbI6A8+AQX/2eW4qeyNZXNp2o= +github.com/spf13/pflag v1.0.6/go.mod h1:McXfInJRrz4CZXVZOBLb0bTZqETkiAhM9Iw0y3An2Bg= +github.com/spf13/viper v1.19.0 h1:RWq5SEjt8o25SROyN3z2OrDB9l7RPd3lwTWU8EcEdcI= +github.com/spf13/viper v1.19.0/go.mod h1:GQUN9bilAbhU/jgc1bKs99f/suXKeUMct8Adx5+Ntkg= +github.com/ssgreg/nlreturn/v2 v2.2.1 h1:X4XDI7jstt3ySqGU86YGAURbxw3oTDPK9sPEi6YEwQ0= +github.com/ssgreg/nlreturn/v2 v2.2.1/go.mod h1:E/iiPB78hV7Szg2YfRgyIrk1AD6JVMTRkkxBiELzh2I= +github.com/stbenjam/no-sprintf-host-port v0.2.0 h1:i8pxvGrt1+4G0czLr/WnmyH7zbZ8Bg8etvARQ1rpyl4= +github.com/stbenjam/no-sprintf-host-port v0.2.0/go.mod h1:eL0bQ9PasS0hsyTyfTjjG+E80QIyPnBVQbYZyv20Jfk= +github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= +github.com/stretchr/objx v0.4.0/go.mod h1:YvHI0jy2hoMjB+UWwv71VJQ9isScKT/TqJzVSSt89Yw= +github.com/stretchr/objx v0.5.0/go.mod h1:Yh+to48EsGEfYuaHDzXPcE3xhTkx73EhmCGUpEOglKo= +github.com/stretchr/objx v0.5.2 h1:xuMeJ0Sdp5ZMRXx/aWO6RZxdr3beISkG5/G/aIRr3pY= +github.com/stretchr/objx v0.5.2/go.mod h1:FRsXN1f5AsAjCGJKqEizvkpNtU+EGNCLh3NxZ/8L+MA= +github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= +github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= +github.com/stretchr/testify v1.5.1/go.mod h1:5W2xD1RspED5o8YsWQXVCued0rvSQ+mT+I5cxcmMvtA= +github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= +github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= +github.com/stretchr/testify v1.7.2/go.mod h1:R6va5+xMeoiuVRoj+gSkQ7d3FALtqAAGI1FQKckRals= +github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU= +github.com/stretchr/testify v1.8.4/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXlSw2iwfAo= +github.com/stretchr/testify v1.10.0 h1:Xv5erBjTwe/5IxqUQTdXv5kgmIvbHo3QQyRwhJsOfJA= +github.com/stretchr/testify v1.10.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY= +github.com/subosito/gotenv v1.6.0 h1:9NlTDc1FTs4qu0DDq7AEtTPNw6SVm7uBMsUCUjABIf8= +github.com/subosito/gotenv v1.6.0/go.mod h1:Dk4QP5c2W3ibzajGcXpNraDfq2IrhjMIvMSWPKKo0FU= +github.com/tdakkota/asciicheck v0.4.1 h1:bm0tbcmi0jezRA2b5kg4ozmMuGAFotKI3RZfrhfovg8= +github.com/tdakkota/asciicheck v0.4.1/go.mod h1:0k7M3rCfRXb0Z6bwgvkEIMleKH3kXNz9UqJ9Xuqopr8= +github.com/tenntenn/modver v1.0.1 h1:2klLppGhDgzJrScMpkj9Ujy3rXPUspSjAcev9tSEBgA= +github.com/tenntenn/modver v1.0.1/go.mod h1:bePIyQPb7UeioSRkw3Q0XeMhYZSMx9B8ePqg6SAMGH0= +github.com/tenntenn/text/transform v0.0.0-20200319021203-7eef512accb3 h1:f+jULpRQGxTSkNYKJ51yaw6ChIqO+Je8UqsTKN/cDag= +github.com/tenntenn/text/transform v0.0.0-20200319021203-7eef512accb3/go.mod h1:ON8b8w4BN/kE1EOhwT0o+d62W65a6aPw1nouo9LMgyY= +github.com/tetafro/godot v1.5.1 h1:PZnjCol4+FqaEzvZg5+O8IY2P3hfY9JzRBNPv1pEDS4= +github.com/tetafro/godot v1.5.1/go.mod h1:cCdPtEndkmqqrhiCfkmxDodMQJ/f3L1BCNskCUZdTwk= +github.com/timakin/bodyclose v0.0.0-20241222091800-1db5c5ca4d67 h1:9LPGD+jzxMlnk5r6+hJnar67cgpDIz/iyD+rfl5r2Vk= +github.com/timakin/bodyclose v0.0.0-20241222091800-1db5c5ca4d67/go.mod h1:mkjARE7Yr8qU23YcGMSALbIxTQ9r9QBVahQOBRfU460= +github.com/timonwong/loggercheck v0.11.0 h1:jdaMpYBl+Uq9mWPXv1r8jc5fC3gyXx4/WGwTnnNKn4M= +github.com/timonwong/loggercheck v0.11.0/go.mod h1:HEAWU8djynujaAVX7QI65Myb8qgfcZ1uKbdpg3ZzKl8= +github.com/tomarrell/wrapcheck/v2 v2.11.0 h1:BJSt36snX9+4WTIXeJ7nvHBQBcm1h2SjQMSlmQ6aFSU= +github.com/tomarrell/wrapcheck/v2 v2.11.0/go.mod h1:wFL9pDWDAbXhhPZZt+nG8Fu+h29TtnZ2MW6Lx4BRXIU= +github.com/tommy-muehle/go-mnd/v2 v2.5.1 h1:NowYhSdyE/1zwK9QCLeRb6USWdoif80Ie+v+yU8u1Zw= +github.com/tommy-muehle/go-mnd/v2 v2.5.1/go.mod h1:WsUAkMJMYww6l/ufffCD3m+P7LEvr8TnZn9lwVDlgzw= +github.com/twmb/go-cache v1.2.1 h1:yUkLutow4S2x5NMbqFW24o14OsucoFI5Fzmlb6uBinM= +github.com/twmb/go-cache v1.2.1/go.mod h1:lArg9KhCl+GTFMikitLGhIBh/i11OK0lhSveqlMbbrY= +github.com/ultraware/funlen v0.2.0 h1:gCHmCn+d2/1SemTdYMiKLAHFYxTYz7z9VIDRaTGyLkI= +github.com/ultraware/funlen v0.2.0/go.mod h1:ZE0q4TsJ8T1SQcjmkhN/w+MceuatI6pBFSxxyteHIJA= +github.com/ultraware/whitespace v0.2.0 h1:TYowo2m9Nfj1baEQBjuHzvMRbp19i+RCcRYrSWoFa+g= +github.com/ultraware/whitespace v0.2.0/go.mod h1:XcP1RLD81eV4BW8UhQlpaR+SDc2givTvyI8a586WjW8= +github.com/uudashr/gocognit v1.2.0 h1:3BU9aMr1xbhPlvJLSydKwdLN3tEUUrzPSSM8S4hDYRA= +github.com/uudashr/gocognit v1.2.0/go.mod h1:k/DdKPI6XBZO1q7HgoV2juESI2/Ofj9AcHPZhBBdrTU= +github.com/uudashr/iface v1.3.1 h1:bA51vmVx1UIhiIsQFSNq6GZ6VPTk3WNMZgRiCe9R29U= +github.com/uudashr/iface v1.3.1/go.mod h1:4QvspiRd3JLPAEXBQ9AiZpLbJlrWWgRChOKDJEuQTdg= +github.com/xen0n/gosmopolitan v1.3.0 h1:zAZI1zefvo7gcpbCOrPSHJZJYA9ZgLfJqtKzZ5pHqQM= +github.com/xen0n/gosmopolitan v1.3.0/go.mod h1:rckfr5T6o4lBtM1ga7mLGKZmLxswUoH1zxHgNXOsEt4= +github.com/xo/terminfo v0.0.0-20220910002029-abceb7e1c41e h1:JVG44RsyaB9T2KIHavMF/ppJZNG9ZpyihvCd0w101no= +github.com/xo/terminfo v0.0.0-20220910002029-abceb7e1c41e/go.mod h1:RbqR21r5mrJuqunuUZ/Dhy/avygyECGrLceyNeo4LiM= +github.com/yagipy/maintidx v1.0.0 h1:h5NvIsCz+nRDapQ0exNv4aJ0yXSI0420omVANTv3GJM= +github.com/yagipy/maintidx v1.0.0/go.mod h1:0qNf/I/CCZXSMhsRsrEPDZ+DkekpKLXAJfsTACwgXLk= +github.com/yeya24/promlinter v0.3.0 h1:JVDbMp08lVCP7Y6NP3qHroGAO6z2yGKQtS5JsjqtoFs= +github.com/yeya24/promlinter v0.3.0/go.mod h1:cDfJQQYv9uYciW60QT0eeHlFodotkYZlL+YcPQN+mW4= +github.com/ykadowak/zerologlint v0.1.5 h1:Gy/fMz1dFQN9JZTPjv1hxEk+sRWm05row04Yoolgdiw= +github.com/ykadowak/zerologlint v0.1.5/go.mod h1:KaUskqF3e/v59oPmdq1U1DnKcuHokl2/K1U4pmIELKg= +github.com/yuin/goldmark v1.1.25/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= +github.com/yuin/goldmark v1.1.32/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= +github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= +github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= +github.com/yuin/goldmark v1.4.1/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= +github.com/yuin/goldmark v1.4.13/go.mod h1:6yULJ656Px+3vBD8DxQVa3kxgyrAnzto9xy5taEt/CY= +gitlab.com/bosi/decorder v0.4.2 h1:qbQaV3zgwnBZ4zPMhGLW4KZe7A7NwxEhJx39R3shffo= +gitlab.com/bosi/decorder v0.4.2/go.mod h1:muuhHoaJkA9QLcYHq4Mj8FJUwDZ+EirSHRiaTcTf6T8= +go-simpler.org/assert v0.9.0 h1:PfpmcSvL7yAnWyChSjOz6Sp6m9j5lyK8Ok9pEL31YkQ= +go-simpler.org/assert v0.9.0/go.mod h1:74Eqh5eI6vCK6Y5l3PI8ZYFXG4Sa+tkr70OIPJAUr28= +go-simpler.org/musttag v0.13.1 h1:lw2sJyu7S1X8lc8zWUAdH42y+afdcCnHhWpnkWvd6vU= +go-simpler.org/musttag v0.13.1/go.mod h1:8r450ehpMLQgvpb6sg+hV5Ur47eH6olp/3yEanfG97k= +go-simpler.org/sloglint v0.11.0 h1:JlR1X4jkbeaffiyjLtymeqmGDKBDO1ikC6rjiuFAOco= +go-simpler.org/sloglint v0.11.0/go.mod h1:CFDO8R1i77dlciGfPEPvYke2ZMx4eyGiEIWkyeW2Pvw= +go.augendre.info/fatcontext v0.8.0 h1:2dfk6CQbDGeu1YocF59Za5Pia7ULeAM6friJ3LP7lmk= +go.augendre.info/fatcontext v0.8.0/go.mod h1:oVJfMgwngMsHO+KB2MdgzcO+RvtNdiCEOlWvSFtax/s= +go.opentelemetry.io/auto/sdk v1.1.0 h1:cH53jehLUN6UFLY71z+NDOiNJqDdPRaXzTel0sJySYA= +go.opentelemetry.io/auto/sdk v1.1.0/go.mod h1:3wSPjt5PWp2RhlCcmmOial7AvC4DQqZb7a7wCow3W8A= +go.opentelemetry.io/otel v1.34.0 h1:zRLXxLCgL1WyKsPVrgbSdMN4c0FMkDAskSTQP+0hdUY= +go.opentelemetry.io/otel v1.34.0/go.mod h1:OWFPOQ+h4G8xpyjgqo4SxJYdDQ/qmRH+wivy7zzx9oI= +go.opentelemetry.io/otel/metric v1.34.0 h1:+eTR3U0MyfWjRDhmFMxe2SsW64QrZ84AOhvqS7Y+PoQ= +go.opentelemetry.io/otel/metric v1.34.0/go.mod h1:CEDrp0fy2D0MvkXE+dPV7cMi8tWZwX3dmaIhwPOaqHE= +go.opentelemetry.io/otel/sdk v1.34.0 h1:95zS4k/2GOy069d321O8jWgYsW3MzVV+KuSPKp7Wr1A= +go.opentelemetry.io/otel/sdk v1.34.0/go.mod h1:0e/pNiaMAqaykJGKbi+tSjWfNNHMTxoC9qANsCzbyxU= +go.opentelemetry.io/otel/sdk/metric v1.34.0 h1:5CeK9ujjbFVL5c1PhLuStg1wxA7vQv7ce1EK0Gyvahk= +go.opentelemetry.io/otel/sdk/metric v1.34.0/go.mod h1:jQ/r8Ze28zRKoNRdkjCZxfs6YvBTG1+YIqyFVFYec5w= +go.opentelemetry.io/otel/trace v1.34.0 h1:+ouXS2V8Rd4hp4580a8q23bg0azF2nI8cqLYnC8mh/k= +go.opentelemetry.io/otel/trace v1.34.0/go.mod h1:Svm7lSjQD7kG7KJ/MUHPVXSDGz2OX4h0M2jHBhmSfRE= +go.uber.org/automaxprocs v1.6.0 h1:O3y2/QNTOdbF+e/dpXNNW7Rx2hZ4sTIPyybbxyNqTUs= +go.uber.org/automaxprocs v1.6.0/go.mod h1:ifeIMSnPZuznNm6jmdzmU3/bfk01Fe2fotchwEFJ8r8= +go.uber.org/goleak v1.3.0 h1:2K3zAYmnTNqV73imy9J1T3WC+gmCePx2hEGkimedGto= +go.uber.org/goleak v1.3.0/go.mod h1:CoHD4mav9JJNrW/WLlf7HGZPjdw8EucARQHekz1X6bE= +go.uber.org/mock v0.5.0 h1:KAMbZvZPyBPWgD14IrIQ38QCyjwpvVVV6K/bHl1IwQU= +go.uber.org/mock v0.5.0/go.mod h1:ge71pBPLYDk7QIi1LupWxdAykm7KIEFchiOqd6z7qMM= +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/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-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= +golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= +golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= +golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= +golang.org/x/crypto v0.13.0/go.mod h1:y6Z2r+Rw4iayiXXAIxJIDAJ1zMW4yaTpebo8fPOliYc= +golang.org/x/crypto v0.14.0/go.mod h1:MVFd36DqK4CsrnJYDkBA3VC4m2GkXAM0PvzMCn4JQf4= +golang.org/x/crypto v0.37.0 h1:kJNSjF/Xp7kU0iB2Z+9viTPMW4EqqsrywMXLJOOsXSE= +golang.org/x/crypto v0.37.0/go.mod h1:vg+k43peMZ0pUMhYmVAWysMK35e6ioLh3wB8ZCAfbVc= +golang.org/x/exp v0.0.0-20250228200357-dead58393ab7 h1:aWwlzYV971S4BXRS9AmqwDLAD85ouC6X+pocatKY58c= +golang.org/x/exp v0.0.0-20250228200357-dead58393ab7/go.mod h1:BHOTPb3L19zxehTsLoJXVaTktb06DFgmdW6Wb9s8jqk= +golang.org/x/exp/typeparams v0.0.0-20220428152302-39d4317da171/go.mod h1:AbB0pIl9nAr9wVwH+Z2ZpaocVmF5I4GyWCDIsVjR0bk= +golang.org/x/exp/typeparams v0.0.0-20230203172020-98cc5a0785f9/go.mod h1:AbB0pIl9nAr9wVwH+Z2ZpaocVmF5I4GyWCDIsVjR0bk= +golang.org/x/exp/typeparams v0.0.0-20250210185358-939b2ce775ac h1:TSSpLIG4v+p0rPv1pNOQtl1I8knsO4S9trOxNMOLVP4= +golang.org/x/exp/typeparams v0.0.0-20250210185358-939b2ce775ac/go.mod h1:AbB0pIl9nAr9wVwH+Z2ZpaocVmF5I4GyWCDIsVjR0bk= +golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= +golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= +golang.org/x/mod v0.4.1/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= +golang.org/x/mod v0.4.2/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= +golang.org/x/mod v0.6.0-dev.0.20220106191415-9b9b3d81d5e3/go.mod h1:3p9vT2HGsQu2K1YbXdKPJLVgG5VJdoTa1poYQBtP1AY= +golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91VN4djpZkiMVwK6gcyfeH4XE8wZrZaV4= +golang.org/x/mod v0.7.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= +golang.org/x/mod v0.8.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= +golang.org/x/mod v0.9.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= +golang.org/x/mod v0.12.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= +golang.org/x/mod v0.13.0/go.mod h1:hTbmBsO62+eylJbnUtE2MGJUyE7QWk4xUqPFrRgJ+7c= +golang.org/x/mod v0.24.0 h1:ZfthKaKaT4NrhGVZHO1/WDTwGES4De8KtWO0SIbNJMU= +golang.org/x/mod v0.24.0/go.mod h1:IXM97Txy2VM4PJ3gI61r1YEk/gAj6zAHN3AdZt6S9Ww= +golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20200625001655-4c5254603344/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= +golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= +golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= +golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= +golang.org/x/net v0.0.0-20211015210444-4f30a5c0130f/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= +golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= +golang.org/x/net v0.2.0/go.mod h1:KqCZLdyyvdV855qA2rE3GC2aiw5xGR5TEjj8smXukLY= +golang.org/x/net v0.6.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= +golang.org/x/net v0.8.0/go.mod h1:QVkue5JL9kW//ek3r6jTKnTFis1tRmNAW2P1shuFdJc= +golang.org/x/net v0.10.0/go.mod h1:0qNGK6F8kojg2nk9dLZ2mShWaEBan6FAoqfSigmmuDg= +golang.org/x/net v0.15.0/go.mod h1:idbUs1IY1+zTqbi8yxTbhexhEEk5ur9LInksu6HrEpk= +golang.org/x/net v0.16.0/go.mod h1:NxSsAGuq816PNPmqtQdLE42eU2Fs7NoRIZrHJAlaCOE= +golang.org/x/net v0.39.0 h1:ZCu7HMWDxpXpaiKdhzIfaltL9Lp31x/3fCP11bc6/fY= +golang.org/x/net v0.39.0/go.mod h1:X7NRbYVEA+ewNkCNyJ513WmMdQ3BineSwVtN2zD/d+E= +golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20200625203802-6e8e738ad208/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.1.0/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.3.0/go.mod h1:FU7BRWz2tNW+3quACPkgCx/L+uEAv1htQ0V83Z9Rj+Y= +golang.org/x/sync v0.4.0/go.mod h1:FU7BRWz2tNW+3quACPkgCx/L+uEAv1htQ0V83Z9Rj+Y= +golang.org/x/sync v0.13.0 h1:AauUjRAJ9OSnvULf/ARrrVywoJDy0YS2AwQ98I37610= +golang.org/x/sync v0.13.0/go.mod h1:1dzgHSNfp02xaA81J2MS99Qcpr2w7fw1gpm99rleRqA= +golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200116001909-b77594299b42/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200223170610-d5e6a3e2c0ae/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200323222414-85ca7c5b95cd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210124154548-22da62e12c0c/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210423082822-04245dca01da/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20210630005230-0f9fa26af87c/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20210927094055-39ccf1dd6fa6/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20211019181941-9d821ace8654/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20211105183446-c75c47738b0c/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220503163025-988cb79eb6c6/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220715151400-c0bba94af5f8/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220811171246-fbc7d0a398ab/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.2.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.6.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.8.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.12.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.13.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.32.0 h1:s77OFDvIQeibCmezSnk/q6iAfkdiQaJi4VzroCFrN20= +golang.org/x/sys v0.32.0/go.mod h1:BJP2sWEmIv4KK5OTEluFJCKSidICx8ciO85XgH3Ak8k= +golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= +golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= +golang.org/x/term v0.2.0/go.mod h1:TVmDHMZPmdnySmBfhjOoOdhjzdE1h4u1VwSiw2l1Nuc= +golang.org/x/term v0.5.0/go.mod h1:jMB1sMXY+tzblOD4FWmEbocvup2/aLOaQEp7JmGp78k= +golang.org/x/term v0.6.0/go.mod h1:m6U89DPEgQRMq3DNkDClhWw02AUbt2daBVO4cn4Hv9U= +golang.org/x/term v0.8.0/go.mod h1:xPskH00ivmX89bAKVGSKKtLOWNx2+17Eiy94tnKShWo= +golang.org/x/term v0.12.0/go.mod h1:owVbMEjm3cBLCHdkQu9b1opXd4ETQWc3BhuQGKgXgvU= +golang.org/x/term v0.13.0/go.mod h1:LTmsnFJwVN6bCy1rVCoS+qHT1HhALEFxKncY3WNNh4U= +golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= +golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= +golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= +golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= +golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= +golang.org/x/text v0.4.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= +golang.org/x/text v0.7.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= +golang.org/x/text v0.8.0/go.mod h1:e1OnstbJyHTd6l/uOt8jFFHp6TRDWZR/bV3emEE/zU8= +golang.org/x/text v0.9.0/go.mod h1:e1OnstbJyHTd6l/uOt8jFFHp6TRDWZR/bV3emEE/zU8= +golang.org/x/text v0.13.0/go.mod h1:TvPlkZtksWOMsz7fbANvkp4WM8x/WCo/om8BMLbz+aE= +golang.org/x/text v0.24.0 h1:dd5Bzh4yt5KYA8f9CJHCP4FB4D51c2c6JvN37xJJkJ0= +golang.org/x/text v0.24.0/go.mod h1:L8rBsPeo2pSS+xqN0d5u2ikmjtmoJbDBT1b7nHvFCdU= +golang.org/x/time v0.11.0 h1:/bpjEDfN9tkoN/ryeYHnv5hcMlc8ncjMcM4XBk5NWV0= +golang.org/x/time v0.11.0/go.mod h1:CDIdPxbZBQxdj6cxyCIdrNogrJKMJ7pr37NYpMcMDSg= +golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20200324003944-a576cf524670/go.mod h1:Sl4aGygMT6LrqrWclx+PTx3U+LnKx/seiNR+3G19Ar8= +golang.org/x/tools v0.0.0-20200329025819-fd4102a86c65/go.mod h1:Sl4aGygMT6LrqrWclx+PTx3U+LnKx/seiNR+3G19Ar8= +golang.org/x/tools v0.0.0-20200724022722-7017fd6b1305/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= +golang.org/x/tools v0.0.0-20200820010801-b793a1359eac/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= +golang.org/x/tools v0.0.0-20201023174141-c8cfbd0f21e6/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= +golang.org/x/tools v0.1.1-0.20210205202024-ef80cdb6ec6d/go.mod h1:9bzcO0MWcOuT0tm1iBGzDVPshzfwoVvREIui8C+MHqU= +golang.org/x/tools v0.1.1-0.20210302220138-2ac05c832e1a/go.mod h1:9bzcO0MWcOuT0tm1iBGzDVPshzfwoVvREIui8C+MHqU= +golang.org/x/tools v0.1.1/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= +golang.org/x/tools v0.1.5/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= +golang.org/x/tools v0.1.10/go.mod h1:Uh6Zz+xoGYZom868N8YTex3t7RhtHDBrE8Gzo9bV56E= +golang.org/x/tools v0.1.12/go.mod h1:hNGJHUnrk76NpqgfD5Aqm5Crs+Hm0VOH/i9J2+nxYbc= +golang.org/x/tools v0.3.0/go.mod h1:/rWhSS2+zyEVwoJf8YAX6L2f0ntZ7Kn/mGgAWcipA5k= +golang.org/x/tools v0.6.0/go.mod h1:Xwgl3UAJ/d3gWutnCtw505GrjyAbvKui8lOU390QaIU= +golang.org/x/tools v0.7.0/go.mod h1:4pg6aUX35JBAogB10C9AtvVL+qowtN4pT3CGSQex14s= +golang.org/x/tools v0.13.0/go.mod h1:HvlwmtVNQAhOuCjW7xxvovg8wbNq7LwfXh/k7wXUl58= +golang.org/x/tools v0.14.0/go.mod h1:uYBEerGOWcJyEORxN+Ek8+TT266gXkNlHdJBwexUsBg= +golang.org/x/tools v0.32.0 h1:Q7N1vhpkQv7ybVzLFtTjvQya2ewbwNDZzUgfXGqtMWU= +golang.org/x/tools v0.32.0/go.mod h1:ZxrU41P/wAbZD8EDa6dDCa6XfpkhJ7HFMjHJXfBDu8s= +golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +google.golang.org/genproto/googleapis/rpc v0.0.0-20250313205543-e70fdf4c4cb4 h1:iK2jbkWL86DXjEx0qiHcRE9dE4/Ahua5k6V8OWFb//c= +google.golang.org/genproto/googleapis/rpc v0.0.0-20250313205543-e70fdf4c4cb4/go.mod h1:LuRYeWDFV6WOn90g357N17oMCaxpgCnbi/44qJvDn2I= +google.golang.org/grpc v1.71.0 h1:kF77BGdPTQ4/JZWMlb9VpJ5pa25aqvVqogsxNHHdeBg= +google.golang.org/grpc v1.71.0/go.mod h1:H0GRtasmQOh9LkFoCPDu3ZrwUtD1YGE+b2vYBYd/8Ec= +google.golang.org/protobuf v1.36.6 h1:z1NpPI8ku2WgiWnf+t9wTPsn6eP1L7ksHUlkfLvd9xY= +google.golang.org/protobuf v1.36.6/go.mod h1:jduwjTPXsFjZGTmRluh+L6NjiWu7pchiJ2/5YcXBHnY= +gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= +gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= +gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q= +gopkg.in/ini.v1 v1.67.0 h1:Dgnx+6+nfE+IfzjUEISNeydPJh9AXNNsWbGP9KzCsOA= +gopkg.in/ini.v1 v1.67.0/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k= +gopkg.in/tomb.v2 v2.0.0-20161208151619-d5d1b5820637 h1:yiW+nvdHb9LVqSHQBXfZCieqV4fzYhNBql77zY0ykqs= +gopkg.in/tomb.v2 v2.0.0-20161208151619-d5d1b5820637/go.mod h1:BHsqpu/nsuzkT5BpiH1EMZPLyqSMM8JbIavyFACoFNk= +gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v2 v2.4.0 h1:D8xgwECY7CYvx+Y2n4sBz93Jn9JRvxdiyyo8CTfuKaY= +gopkg.in/yaml.v2 v2.4.0/go.mod h1:RDklbk79AGWmwhnvt/jBztapEOGDOx6ZbXqjP6csGnQ= +gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= +gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA= +gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= +honnef.co/go/tools v0.6.1 h1:R094WgE8K4JirYjBaOpz/AvTyUu/3wbmAoskKN/pxTI= +honnef.co/go/tools v0.6.1/go.mod h1:3puzxxljPCe8RGJX7BIy1plGbxEOZni5mR2aXe3/uk4= +mvdan.cc/gofumpt v0.8.0 h1:nZUCeC2ViFaerTcYKstMmfysj6uhQrA2vJe+2vwGU6k= +mvdan.cc/gofumpt v0.8.0/go.mod h1:vEYnSzyGPmjvFkqJWtXkh79UwPWP9/HMxQdGEXZHjpg= +mvdan.cc/unparam v0.0.0-20250301125049-0df0534333a4 h1:WjUu4yQoT5BHT1w8Zu56SP8367OuBV5jvo+4Ulppyf8= +mvdan.cc/unparam v0.0.0-20250301125049-0df0534333a4/go.mod h1:rthT7OuvRbaGcd5ginj6dA2oLE7YNlta9qhBNNdCaLE= From 752982ffa232d9da6066178503233515d5a0e44b Mon Sep 17 00:00:00 2001 From: Haris Osmanagic Date: Thu, 29 May 2025 17:39:02 +0200 Subject: [PATCH 12/60] uncomment numeric, fix stuff --- source/schema/avro_integration_test.go | 2 +- source/snapshot/fetch_worker.go | 4 ++-- source/types/types.go | 11 +---------- source_integration_test.go | 24 +++++++++++++----------- 4 files changed, 17 insertions(+), 24 deletions(-) diff --git a/source/schema/avro_integration_test.go b/source/schema/avro_integration_test.go index 40ea3e2..e83845e 100644 --- a/source/schema/avro_integration_test.go +++ b/source/schema/avro_integration_test.go @@ -237,7 +237,7 @@ func avrolizeMap(fields []pgconn.FieldDescription, values []any) map[string]any switch f.DataTypeOID { case pgtype.NumericOID: n := new(big.Rat) - n.SetString(fmt.Sprint(types.Format(0, values[i]))) + n.SetString(fmt.Sprint(types.Format(0, values[i], true))) row[f.Name] = n case pgtype.UUIDOID: row[f.Name] = fmt.Sprint(values[i]) diff --git a/source/snapshot/fetch_worker.go b/source/snapshot/fetch_worker.go index 64b7231..bd9a965 100644 --- a/source/snapshot/fetch_worker.go +++ b/source/snapshot/fetch_worker.go @@ -134,7 +134,7 @@ func NewFetchWorker(db *pgxpool.Pool, out chan<- FetchData, c FetchConfig) *Fetc // * Table and keys exist // * Key is a primary key func (f *FetchWorker) Init(ctx context.Context) error { - err := f.validate(ctx) + err := f.Validate(ctx) if err != nil { return fmt.Errorf("validation failed: %w", err) } @@ -147,7 +147,7 @@ func (f *FetchWorker) Init(ctx context.Context) error { return nil } -func (f *FetchWorker) validate(ctx context.Context) error { +func (f *FetchWorker) Validate(ctx context.Context) error { if err := f.conf.Validate(); err != nil { return fmt.Errorf("failed to validate config: %w", err) } diff --git a/source/types/types.go b/source/types/types.go index 47aa031..4ca9e03 100644 --- a/source/types/types.go +++ b/source/types/types.go @@ -35,7 +35,7 @@ func Format(oid uint32, v any, isNotNull bool) (any, error) { if val == nil { return nil, nil } - + if reflect.TypeOf(val).Kind() != reflect.Ptr && !isNotNull { return &val, nil } @@ -43,15 +43,6 @@ func Format(oid uint32, v any, isNotNull bool) (any, error) { return val, nil } -func isPrimitive(v any) bool { - switch v.(type) { - case string, int, int8, int16, int32, int64, uint, uint8, uint16, uint32, uint64, float32, float64, bool: - return true - default: - return false - } -} - func format(oid uint32, v any) (any, error) { if oid == pgtype.UUIDOID { return UUID.Format(v) diff --git a/source_integration_test.go b/source_integration_test.go index 628769e..cde9373 100644 --- a/source_integration_test.go +++ b/source_integration_test.go @@ -63,20 +63,22 @@ func TestSource_Read(t *testing.T) { is.NoErr(s.Teardown(ctx)) }) - gotRecord, err := s.Read(ctx) + gotRecords, err := s.ReadN(ctx, 1) is.NoErr(err) - err = s.Ack(ctx, gotRecord.Position) + is.Equal(1, len(gotRecords)) + err = s.Ack(ctx, gotRecords[0].Position) is.NoErr(err) - assertRecordOK(is, tableName, gotRecord) + assertRecordOK(is, tableName, gotRecords[0]) insertRowNotNullColumnsOnly(ctx, t, tableName, 3) insertRowAllColumns(ctx, t, tableName, 4) - gotRecord, err = s.Read(ctx) + gotRecords, err = s.ReadN(ctx, 1) is.NoErr(err) - err = s.Ack(ctx, gotRecord.Position) + is.Equal(1, len(gotRecords)) + err = s.Ack(ctx, gotRecords[0].Position) is.NoErr(err) - assertRecordOK(is, tableName, gotRecord) + assertRecordOK(is, tableName, gotRecords[0]) } func assertRecordOK(is *is.I, tableName string, gotRecord opencdc.Record) { @@ -123,7 +125,7 @@ func prepareSourceIntegrationTestTable(ctx context.Context, t *testing.T) string col_int8 int8, col_int8_not_null int8 NOT NULL, col_numeric numeric(8,2), - -- col_numeric_not_null numeric(8,2) NOT NULL, + col_numeric_not_null numeric(8,2) NOT NULL, col_text text, col_text_not_null text NOT NULL, col_timestamp timestamp, @@ -163,7 +165,7 @@ func insertRowNotNullColumnsOnly(ctx context.Context, t *testing.T, table string col_int2_not_null, col_int4_not_null, col_int8_not_null, - -- col_numeric_not_null, + col_numeric_not_null, col_text_not_null, col_timestamp_not_null, col_timestamptz_not_null, @@ -177,7 +179,7 @@ func insertRowNotNullColumnsOnly(ctx context.Context, t *testing.T, table string %d, -- col_int2_not_null %d, -- col_int4_not_null %d, -- col_int8_not_null - -- %f, -- col_numeric_not_null + %f, -- col_numeric_not_null 'bar-%v', -- col_text_not_null now(), -- col_timestamp_not_null now(), -- col_timestamptz_not_null @@ -212,7 +214,7 @@ func insertRowAllColumns(ctx context.Context, t *testing.T, table string, rowNum col_int2, col_int2_not_null, col_int4, col_int4_not_null, col_int8, col_int8_not_null, - -- col_numeric, col_numeric_not_null, + col_numeric, col_numeric_not_null, col_text, col_text_not_null, col_timestamp, col_timestamp_not_null, col_timestamptz, col_timestamptz_not_null, @@ -226,7 +228,7 @@ func insertRowAllColumns(ctx context.Context, t *testing.T, table string, rowNum %d, %d, %d, %d, %d, %d, - -- %f, %f, + %f, %f, 'bar-%v', 'bar-%v', now(), now(), now(), now(), From d79dc993360497ff1916480744a417ddae89cf9d Mon Sep 17 00:00:00 2001 From: Haris Osmanagic Date: Thu, 29 May 2025 18:39:12 +0200 Subject: [PATCH 13/60] table_info: support capital case in table name --- source/common/table_info.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/source/common/table_info.go b/source/common/table_info.go index c016444..ce4d413 100644 --- a/source/common/table_info.go +++ b/source/common/table_info.go @@ -18,6 +18,7 @@ import ( "context" "fmt" + "github.com/conduitio/conduit-connector-postgres/internal" sdk "github.com/conduitio/conduit-connector-sdk" "github.com/jackc/pgx/v5/pgxpool" ) @@ -72,7 +73,7 @@ func (i TableInfoFetcher) Refresh(ctx context.Context, tableName string) error { ORDER BY a.attnum; ` - rows, err := tx.Query(context.Background(), query, tableName) + rows, err := tx.Query(context.Background(), query, internal.WrapSQLIdent(tableName)) if err != nil { sdk.Logger(ctx). Err(err). From 830eb6f16f92717bd8314dab94540e3141ac41e6 Mon Sep 17 00:00:00 2001 From: Haris Osmanagic Date: Thu, 29 May 2025 20:41:46 +0200 Subject: [PATCH 14/60] better way to take a pointer --- source/types/types.go | 28 +++++++++++++++++++++++++++- source/types/types_test.go | 7 ++++--- 2 files changed, 31 insertions(+), 4 deletions(-) diff --git a/source/types/types.go b/source/types/types.go index 4ca9e03..fa1687f 100644 --- a/source/types/types.go +++ b/source/types/types.go @@ -37,7 +37,7 @@ func Format(oid uint32, v any, isNotNull bool) (any, error) { } if reflect.TypeOf(val).Kind() != reflect.Ptr && !isNotNull { - return &val, nil + return GetPointer(val), nil } return val, nil @@ -62,3 +62,29 @@ func format(oid uint32, v any) (any, error) { return t, nil } } + +func GetPointer(v any) any { + rv := reflect.ValueOf(v) + + // If the value is nil or invalid, return nil + if !rv.IsValid() { + return nil + } + + // If it's already a pointer, return it as-is + if rv.Kind() == reflect.Ptr { + return rv.Interface() + } + + // For non-pointer values, we need to get the address + // If the value is addressable, return its address + if rv.CanAddr() { + return rv.Addr().Interface() + } + + // If we can't get the address directly, create an addressable copy + // This happens when the interface{} contains a literal value + ptr := reflect.New(rv.Type()) + ptr.Elem().Set(rv) + return ptr.Interface() +} diff --git a/source/types/types_test.go b/source/types/types_test.go index fb1dea7..25dbb5c 100644 --- a/source/types/types_test.go +++ b/source/types/types_test.go @@ -19,6 +19,7 @@ import ( "testing" "time" + "github.com/conduitio/conduit-commons/lang" "github.com/jackc/pgx/v5/pgtype" "github.com/matryer/is" ) @@ -36,13 +37,13 @@ func Test_Format(t *testing.T) { { name: "int float string bool", input: []any{ - 1021, 199.2, "foo", true, + 1021, }, inputOID: []uint32{ 0, 0, 0, 0, }, expect: []any{ - 1021, 199.2, "foo", true, + lang.Ptr(1021), }, }, { @@ -90,7 +91,7 @@ func Test_Format(t *testing.T) { is := is.New(t) for i, in := range tc.input { - v, err := Format(tc.inputOID[i], in) + v, err := Format(tc.inputOID[i], in, false) is.NoErr(err) is.Equal(v, tc.expect[i]) } From b7b60c947c065d23463d493307c8cfb8b0c55a52 Mon Sep 17 00:00:00 2001 From: Haris Osmanagic Date: Fri, 13 Jun 2025 11:41:01 +0200 Subject: [PATCH 15/60] fix types_test --- source/types/types_test.go | 30 ++++++++++++++++++++++-------- 1 file changed, 22 insertions(+), 8 deletions(-) diff --git a/source/types/types_test.go b/source/types/types_test.go index 25dbb5c..0253f6d 100644 --- a/source/types/types_test.go +++ b/source/types/types_test.go @@ -28,11 +28,11 @@ func Test_Format(t *testing.T) { now := time.Now().UTC() tests := []struct { - name string - input []any - inputOID []uint32 - expect []any - withBuiltin bool + name string + input []any + inputOID []uint32 + expect []any + expectNullable []any }{ { name: "int float string bool", @@ -43,6 +43,9 @@ func Test_Format(t *testing.T) { 0, 0, 0, 0, }, expect: []any{ + 1021, + }, + expectNullable: []any{ lang.Ptr(1021), }, }, @@ -57,6 +60,9 @@ func Test_Format(t *testing.T) { expect: []any{ big.NewRat(122121, 10000), big.NewRat(101, 1), big.NewRat(0, 1), nil, nil, }, + expectNullable: []any{ + big.NewRat(122121, 10000), big.NewRat(101, 1), big.NewRat(0, 1), nil, nil, + }, }, { name: "builtin time.Time", @@ -69,7 +75,9 @@ func Test_Format(t *testing.T) { expect: []any{ now, }, - withBuiltin: true, + expectNullable: []any{ + lang.Ptr(now), + }, }, { name: "uuid", @@ -82,18 +90,24 @@ func Test_Format(t *testing.T) { expect: []any{ "bd94ee0b-564f-4088-bf4e-8d5e626caf66", "", }, + expectNullable: []any{ + lang.Ptr("bd94ee0b-564f-4088-bf4e-8d5e626caf66"), lang.Ptr(""), + }, }, } - _ = time.Now() for _, tc := range tests { t.Run(tc.name, func(t *testing.T) { is := is.New(t) for i, in := range tc.input { - v, err := Format(tc.inputOID[i], in, false) + v, err := Format(tc.inputOID[i], in, true) is.NoErr(err) is.Equal(v, tc.expect[i]) + + vNullable, err := Format(tc.inputOID[i], in, false) + is.NoErr(err) + is.Equal(vNullable, tc.expectNullable[i]) } }) } From 98a0a3c196ad03601a7c04ea04c4dd04900e048d Mon Sep 17 00:00:00 2001 From: Haris Osmanagic Date: Fri, 13 Jun 2025 15:49:06 +0200 Subject: [PATCH 16/60] minor change --- source/logrepl/cdc_test.go | 3 +-- test/helper.go | 3 ++- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/source/logrepl/cdc_test.go b/source/logrepl/cdc_test.go index ff57442..d391ca2 100644 --- a/source/logrepl/cdc_test.go +++ b/source/logrepl/cdc_test.go @@ -498,8 +498,7 @@ func TestCDCIterator_NextN(t *testing.T) { for j, r := range allRecords { is.Equal(r.Operation, opencdc.OperationCreate) is.Equal(r.Key.(opencdc.StructuredData)["id"], int64(j+11)) - change := r.Payload - data := change.After.(opencdc.StructuredData) + data := r.Payload.After.(opencdc.StructuredData) is.Equal(data["column1"], fmt.Sprintf("test-%d", j+1)) //nolint:gosec // no risk to overflow is.Equal(data["column2"], (int32(j)+1)*100) diff --git a/test/helper.go b/test/helper.go index 6e72e21..39fa8ab 100644 --- a/test/helper.go +++ b/test/helper.go @@ -185,7 +185,7 @@ func ConnectSimple(ctx context.Context, t *testing.T, connString string) *pgx.Co return conn.Conn() } -// SetupTestTable creates a new table and returns its name. +// SetupEmptyTestTable creates an empty test table and returns its name. func SetupEmptyTestTable(ctx context.Context, t *testing.T, conn Querier) string { table := RandomIdentifier(t) SetupEmptyTestTableWithName(ctx, t, conn, table) @@ -207,6 +207,7 @@ func SetupEmptyTestTableWithName(ctx context.Context, t *testing.T, conn Querier }) } +// SetupTestTableWithName creates a test table with a few row inserted into it. func SetupTestTableWithName(ctx context.Context, t *testing.T, conn Querier, table string) { is := is.New(t) SetupEmptyTestTableWithName(ctx, t, conn, table) From cf60d5018bacd673ab25ef6f1afe53fa946c124a Mon Sep 17 00:00:00 2001 From: Haris Osmanagic Date: Fri, 13 Jun 2025 16:15:46 +0200 Subject: [PATCH 17/60] add more types to source integration test --- destination_integration_test.go | 4 +- source/logrepl/cdc_test.go | 12 +- source/logrepl/cleaner_test.go | 6 +- source/logrepl/combined_test.go | 4 +- source/logrepl/internal/publication_test.go | 8 +- source/logrepl/internal/subscription_test.go | 6 +- .../snapshot/fetch_worker_integration_test.go | 12 +- source/snapshot/iterator_test.go | 2 +- source_integration_test.go | 116 +++++++++++++----- test/helper.go | 20 +-- 10 files changed, 119 insertions(+), 71 deletions(-) diff --git a/destination_integration_test.go b/destination_integration_test.go index f97e7fe..14ba4e9 100644 --- a/destination_integration_test.go +++ b/destination_integration_test.go @@ -36,7 +36,7 @@ func TestDestination_Write(t *testing.T) { // tables with capital letters should be quoted tableName := strings.ToUpper(test.RandomIdentifier(t)) - test.SetupTestTableWithName(ctx, t, conn, tableName) + test.SetupTableWithName(ctx, t, conn, tableName) d := NewDestination() err := sdk.Util.ParseConfig( @@ -197,7 +197,7 @@ func TestDestination_Batch(t *testing.T) { conn := test.ConnectSimple(ctx, t, test.RegularConnString) tableName := strings.ToUpper(test.RandomIdentifier(t)) - test.SetupTestTableWithName(ctx, t, conn, tableName) + test.SetupTableWithName(ctx, t, conn, tableName) d := NewDestination() diff --git a/source/logrepl/cdc_test.go b/source/logrepl/cdc_test.go index d391ca2..aec2770 100644 --- a/source/logrepl/cdc_test.go +++ b/source/logrepl/cdc_test.go @@ -49,7 +49,7 @@ func TestCDCIterator_New(t *testing.T) { name: "publication already exists", setup: func(t *testing.T) CDCConfig { is := is.New(t) - table := test.SetupTestTable(ctx, t, pool) + table := test.SetupTable(ctx, t, pool) test.CreatePublication(t, pool, table, []string{table}) t.Cleanup(func() { @@ -79,7 +79,7 @@ func TestCDCIterator_New(t *testing.T) { name: "fails to create subscription", setup: func(t *testing.T) CDCConfig { is := is.New(t) - table := test.SetupTestTable(ctx, t, pool) + table := test.SetupTable(ctx, t, pool) t.Cleanup(func() { is.NoErr(Cleanup(ctx, CleanupConfig{ @@ -125,7 +125,7 @@ func TestCDCIterator_Operation_NextN(t *testing.T) { is := is.New(t) pool := test.ConnectPool(ctx, t, test.RepmgrConnString) - table := test.SetupTestTable(ctx, t, pool) + table := test.SetupTable(ctx, t, pool) i := testCDCIterator(ctx, t, pool, table, true) // wait for subscription to be ready @@ -374,7 +374,7 @@ func TestCDCIterator_EnsureLSN(t *testing.T) { is := is.New(t) pool := test.ConnectPool(ctx, t, test.RepmgrConnString) - table := test.SetupTestTable(ctx, t, pool) + table := test.SetupTable(ctx, t, pool) i := testCDCIterator(ctx, t, pool, table, true) <-i.sub.Ready() @@ -467,7 +467,7 @@ func TestCDCIterator_Ack(t *testing.T) { func TestCDCIterator_NextN(t *testing.T) { ctx := test.Context(t) pool := test.ConnectPool(ctx, t, test.RepmgrConnString) - table := test.SetupTestTable(ctx, t, pool) + table := test.SetupTable(ctx, t, pool) t.Run("retrieve exact N records", func(t *testing.T) { is := is.New(t) @@ -655,7 +655,7 @@ func TestCDCIterator_Schema(t *testing.T) { ctx := test.Context(t) pool := test.ConnectPool(ctx, t, test.RepmgrConnString) - table := test.SetupTestTable(ctx, t, pool) + table := test.SetupTable(ctx, t, pool) i := testCDCIterator(ctx, t, pool, table, true) <-i.sub.Ready() diff --git a/source/logrepl/cleaner_test.go b/source/logrepl/cleaner_test.go index 7c67058..c8cf121 100644 --- a/source/logrepl/cleaner_test.go +++ b/source/logrepl/cleaner_test.go @@ -42,7 +42,7 @@ func Test_Cleanup(t *testing.T) { PublicationName: "conduitpub1", }, setup: func(t *testing.T) { - table := test.SetupTestTable(context.Background(), t, conn) + table := test.SetupTable(context.Background(), t, conn) test.CreatePublication(t, conn, "conduitpub1", []string{table}) test.CreateReplicationSlot(t, conn, "conduitslot1") }, @@ -54,7 +54,7 @@ func Test_Cleanup(t *testing.T) { PublicationName: "conduitpub2", }, setup: func(t *testing.T) { - table := test.SetupTestTable(context.Background(), t, conn) + table := test.SetupTable(context.Background(), t, conn) test.CreatePublication(t, conn, "conduitpub2", []string{table}) }, }, @@ -76,7 +76,7 @@ func Test_Cleanup(t *testing.T) { PublicationName: "conduitpub4", }, setup: func(t *testing.T) { - table := test.SetupTestTable(context.Background(), t, conn) + table := test.SetupTable(context.Background(), t, conn) test.CreatePublication(t, conn, "conduitpub4", []string{table}) }, wantErr: errors.New(`replication slot "conduitslot4" does not exist`), diff --git a/source/logrepl/combined_test.go b/source/logrepl/combined_test.go index d48976d..dd01184 100644 --- a/source/logrepl/combined_test.go +++ b/source/logrepl/combined_test.go @@ -50,7 +50,7 @@ func TestConfig_Validate(t *testing.T) { func TestCombinedIterator_New(t *testing.T) { ctx := test.Context(t) pool := test.ConnectPool(ctx, t, test.RepmgrConnString) - table := test.SetupTestTable(ctx, t, pool) + table := test.SetupTable(ctx, t, pool) t.Run("fails to parse initial position", func(t *testing.T) { is := is.New(t) @@ -145,7 +145,7 @@ func TestCombinedIterator_NextN(t *testing.T) { is := is.New(t) pool := test.ConnectPool(ctx, t, test.RepmgrConnString) - table := test.SetupTestTable(ctx, t, pool) + table := test.SetupTable(ctx, t, pool) i, err := NewCombinedIterator(ctx, pool, Config{ Position: opencdc.Position{}, Tables: []string{table}, diff --git a/source/logrepl/internal/publication_test.go b/source/logrepl/internal/publication_test.go index f04a707..e4a109f 100644 --- a/source/logrepl/internal/publication_test.go +++ b/source/logrepl/internal/publication_test.go @@ -35,8 +35,8 @@ func TestCreatePublication(t *testing.T) { } tables := []string{ - test.SetupTestTable(ctx, t, pool), - test.SetupTestTable(ctx, t, pool), + test.SetupTable(ctx, t, pool), + test.SetupTable(ctx, t, pool), } for _, givenPubName := range pubNames { @@ -75,8 +75,8 @@ func TestCreatePublicationForTables(t *testing.T) { pool := test.ConnectPool(ctx, t, test.RegularConnString) tables := [][]string{ - {test.SetupTestTable(ctx, t, pool)}, - {test.SetupTestTable(ctx, t, pool), test.SetupTestTable(ctx, t, pool)}, + {test.SetupTable(ctx, t, pool)}, + {test.SetupTable(ctx, t, pool), test.SetupTable(ctx, t, pool)}, } for _, givenTables := range tables { diff --git a/source/logrepl/internal/subscription_test.go b/source/logrepl/internal/subscription_test.go index da6a587..16baae6 100644 --- a/source/logrepl/internal/subscription_test.go +++ b/source/logrepl/internal/subscription_test.go @@ -42,8 +42,8 @@ func TestSubscription_WithRepmgr(t *testing.T) { var ( ctx = test.Context(t) pool = test.ConnectPool(ctx, t, test.RepmgrConnString) - table1 = test.SetupTestTable(ctx, t, pool) - table2 = test.SetupTestTable(ctx, t, pool) + table1 = test.SetupTable(ctx, t, pool) + table2 = test.SetupTable(ctx, t, pool) ) sub, messages := setupSubscription(ctx, t, pool, table1, table2) @@ -156,7 +156,7 @@ func TestSubscription_ClosedContext(t *testing.T) { var ( is = is.New(t) pool = test.ConnectPool(ctx, t, test.RepmgrConnString) - table = test.SetupTestTable(ctx, t, pool) + table = test.SetupTable(ctx, t, pool) ) sub, messages := setupSubscription(ctx, t, pool, table) diff --git a/source/snapshot/fetch_worker_integration_test.go b/source/snapshot/fetch_worker_integration_test.go index 9b991a9..01047c7 100644 --- a/source/snapshot/fetch_worker_integration_test.go +++ b/source/snapshot/fetch_worker_integration_test.go @@ -126,7 +126,7 @@ func Test_FetcherValidate(t *testing.T) { ) // uppercase table name is required to test primary key fetching - test.SetupTestTableWithName(ctx, t, pool, table) + test.SetupTableWithName(ctx, t, pool, table) t.Run("success", func(t *testing.T) { is := is.New(t) @@ -215,7 +215,7 @@ func Test_FetcherRun_EmptySnapshot(t *testing.T) { is = is.New(t) ctx = test.Context(t) pool = test.ConnectPool(context.Background(), t, test.RegularConnString) - table = test.SetupEmptyTestTable(context.Background(), t, pool) + table = test.SetupEmptyTable(context.Background(), t, pool) out = make(chan FetchData) testTomb = &tomb.Tomb{} ) @@ -247,7 +247,7 @@ func Test_FetcherRun_EmptySnapshot(t *testing.T) { func Test_FetcherRun_Initial(t *testing.T) { var ( pool = test.ConnectPool(context.Background(), t, test.RegularConnString) - table = test.SetupTestTable(context.Background(), t, pool) + table = test.SetupTable(context.Background(), t, pool) is = is.New(t) out = make(chan FetchData) ctx = test.Context(t) @@ -313,7 +313,7 @@ func Test_FetcherRun_Initial(t *testing.T) { func Test_FetcherRun_Resume(t *testing.T) { var ( pool = test.ConnectPool(context.Background(), t, test.RegularConnString) - table = test.SetupTestTable(context.Background(), t, pool) + table = test.SetupTable(context.Background(), t, pool) is = is.New(t) out = make(chan FetchData) ctx = test.Context(t) @@ -498,7 +498,7 @@ func Test_FetchWorker_updateSnapshotEnd(t *testing.T) { table = strings.ToUpper(test.RandomIdentifier(t)) ) - test.SetupTestTableWithName(ctx, t, pool, table) + test.SetupTableWithName(ctx, t, pool, table) tx, err := pool.Begin(ctx) is.NoErr(err) @@ -560,7 +560,7 @@ func Test_FetchWorker_updateSnapshotEnd(t *testing.T) { func Test_FetchWorker_createCursor(t *testing.T) { var ( pool = test.ConnectPool(context.Background(), t, test.RegularConnString) - table = test.SetupTestTable(context.Background(), t, pool) + table = test.SetupTable(context.Background(), t, pool) is = is.New(t) ctx = test.Context(t) ) diff --git a/source/snapshot/iterator_test.go b/source/snapshot/iterator_test.go index 610f2aa..f1e9617 100644 --- a/source/snapshot/iterator_test.go +++ b/source/snapshot/iterator_test.go @@ -30,7 +30,7 @@ func Test_Iterator_NextN(t *testing.T) { var ( ctx = test.Context(t) pool = test.ConnectPool(ctx, t, test.RegularConnString) - table = test.SetupTestTable(ctx, t, pool) + table = test.SetupTable(ctx, t, pool) ) t.Run("success", func(t *testing.T) { diff --git a/source_integration_test.go b/source_integration_test.go index cde9373..cfadc07 100644 --- a/source_integration_test.go +++ b/source_integration_test.go @@ -28,11 +28,11 @@ import ( "github.com/matryer/is" ) -func TestSource_Read(t *testing.T) { +func TestSource_ReadN_Snapshot_CDC(t *testing.T) { is := is.New(t) ctx := test.Context(t) - tableName := prepareSourceIntegrationTestTable(ctx, t) + tableName := createTableWithManyTypes(ctx, t) slotName := "conduitslot1" publicationName := "conduitpub1" @@ -100,7 +100,7 @@ func assertRecordOK(is *is.I, tableName string, gotRecord opencdc.Record) { is.Equal(1, keySchemaVersion) } -func prepareSourceIntegrationTestTable(ctx context.Context, t *testing.T) string { +func createTableWithManyTypes(ctx context.Context, t *testing.T) string { is := is.New(t) conn := test.ConnectSimple(ctx, t, test.RepmgrConnString) @@ -133,7 +133,19 @@ func prepareSourceIntegrationTestTable(ctx context.Context, t *testing.T) string col_timestamptz timestamptz, col_timestamptz_not_null timestamptz NOT NULL, col_uuid uuid, - col_uuid_not_null uuid NOT NULL + col_uuid_not_null uuid NOT NULL, + col_json json, + col_json_not_null json NOT NULL, + col_jsonb jsonb, + col_jsonb_not_null jsonb NOT NULL, + col_bool bool, + col_bool_not_null bool NOT NULL, + col_serial serial, + col_serial_not_null serial NOT NULL, + col_smallserial smallserial, + col_smallserial_not_null smallserial NOT NULL, + col_bigserial bigserial, + col_bigserial_not_null bigserial NOT NULL )`, table) _, err := conn.Exec(ctx, query) is.NoErr(err) @@ -157,34 +169,46 @@ func insertRowNotNullColumnsOnly(ctx context.Context, t *testing.T, table string query := fmt.Sprintf( `INSERT INTO %s ( - col_bytea_not_null, - col_varchar_not_null, - col_date_not_null, - col_float4_not_null, - col_float8_not_null, - col_int2_not_null, - col_int4_not_null, - col_int8_not_null, - col_numeric_not_null, - col_text_not_null, - col_timestamp_not_null, - col_timestamptz_not_null, - col_uuid_not_null - ) VALUES ( - '%s'::bytea, -- col_bytea_not_null - 'foo-%v', -- col_varchar_not_null - now(), -- col_date_not_null - %f, -- col_float4_not_null - %f, -- col_float8_not_null - %d, -- col_int2_not_null - %d, -- col_int4_not_null - %d, -- col_int8_not_null - %f, -- col_numeric_not_null - 'bar-%v', -- col_text_not_null - now(), -- col_timestamp_not_null - now(), -- col_timestamptz_not_null - gen_random_uuid() -- col_uuid_not_null - )`, + col_bytea_not_null, + col_varchar_not_null, + col_date_not_null, + col_float4_not_null, + col_float8_not_null, + col_int2_not_null, + col_int4_not_null, + col_int8_not_null, + col_numeric_not_null, + col_text_not_null, + col_timestamp_not_null, + col_timestamptz_not_null, + col_uuid_not_null, + col_json_not_null, + col_jsonb_not_null, + col_bool_not_null, + col_serial_not_null, + col_smallserial_not_null, + col_bigserial_not_null + ) VALUES ( + '%s'::bytea, -- col_bytea_not_null + 'foo-%v', -- col_varchar_not_null + now(), -- col_date_not_null + %f, -- col_float4_not_null + %f, -- col_float8_not_null + %d, -- col_int2_not_null + %d, -- col_int4_not_null + %d, -- col_int8_not_null + %f, -- col_numeric_not_null + 'bar-%v', -- col_text_not_null + now(), -- col_timestamp_not_null + now(), -- col_timestamptz_not_null + gen_random_uuid(), -- col_uuid_not_null + '{"key": "value-%v"}'::json, -- col_json_not_null + '{"key": "value-%v"}'::jsonb, -- col_jsonb_not_null + %t, -- col_bool_not_null + %d, -- col_serial_not_null + %d, -- col_smallserial_not_null + %d -- col_bigserial_not_null + )`, table, fmt.Sprintf("col_bytea_-%v", rowNumber), rowNumber, @@ -195,6 +219,12 @@ func insertRowNotNullColumnsOnly(ctx context.Context, t *testing.T, table string rowNumber, float64(100+rowNumber)/10, rowNumber, + rowNumber, + rowNumber, + rowNumber%2 == 0, + rowNumber, + rowNumber, + rowNumber, ) _, err := conn.Exec(ctx, query) is.NoErr(err) @@ -218,7 +248,13 @@ func insertRowAllColumns(ctx context.Context, t *testing.T, table string, rowNum col_text, col_text_not_null, col_timestamp, col_timestamp_not_null, col_timestamptz, col_timestamptz_not_null, - col_uuid, col_uuid_not_null + col_uuid, col_uuid_not_null, + col_json, col_json_not_null, + col_jsonb, col_jsonb_not_null, + col_bool, col_bool_not_null, + col_serial, col_serial_not_null, + col_smallserial, col_smallserial_not_null, + col_bigserial, col_bigserial_not_null ) VALUES ( '%s'::bytea, '%s'::bytea, 'foo-%v', 'foo-%v', @@ -232,7 +268,13 @@ func insertRowAllColumns(ctx context.Context, t *testing.T, table string, rowNum 'bar-%v', 'bar-%v', now(), now(), now(), now(), - gen_random_uuid(), gen_random_uuid() + gen_random_uuid(), gen_random_uuid(), + '{"key": "value-%v"}'::json, '{"key": "value-%v"}'::json, + '{"key": "value-%v"}'::jsonb, '{"key": "value-%v"}'::jsonb, + %t, %t, + %d, %d, + %d, %d, + %d, %d )`, table, fmt.Sprintf("col_bytea_-%v", rowNumber), fmt.Sprintf("col_bytea_-%v", rowNumber), @@ -244,6 +286,12 @@ func insertRowAllColumns(ctx context.Context, t *testing.T, table string, rowNum rowNumber, rowNumber, float64(100+rowNumber)/10, float64(100+rowNumber)/10, rowNumber, rowNumber, + rowNumber, rowNumber, + rowNumber, rowNumber, + rowNumber%2 == 0, rowNumber%2 == 0, + rowNumber, rowNumber, + rowNumber, rowNumber, + rowNumber, rowNumber, ) _, err := conn.Exec(ctx, query) is.NoErr(err) diff --git a/test/helper.go b/test/helper.go index 39fa8ab..8956726 100644 --- a/test/helper.go +++ b/test/helper.go @@ -185,14 +185,14 @@ func ConnectSimple(ctx context.Context, t *testing.T, connString string) *pgx.Co return conn.Conn() } -// SetupEmptyTestTable creates an empty test table and returns its name. -func SetupEmptyTestTable(ctx context.Context, t *testing.T, conn Querier) string { +// SetupEmptyTable creates an empty test table and returns its name. +func SetupEmptyTable(ctx context.Context, t *testing.T, conn Querier) string { table := RandomIdentifier(t) - SetupEmptyTestTableWithName(ctx, t, conn, table) + SetupEmptyTableWithName(ctx, t, conn, table) return table } -func SetupEmptyTestTableWithName(ctx context.Context, t *testing.T, conn Querier, table string) { +func SetupEmptyTableWithName(ctx context.Context, t *testing.T, conn Querier, table string) { is := is.New(t) query := fmt.Sprintf(testTableCreateQuery, table) @@ -207,10 +207,10 @@ func SetupEmptyTestTableWithName(ctx context.Context, t *testing.T, conn Querier }) } -// SetupTestTableWithName creates a test table with a few row inserted into it. -func SetupTestTableWithName(ctx context.Context, t *testing.T, conn Querier, table string) { +// SetupTableWithName creates a test table with a few row inserted into it. +func SetupTableWithName(ctx context.Context, t *testing.T, conn Querier, table string) { is := is.New(t) - SetupEmptyTestTableWithName(ctx, t, conn, table) + SetupEmptyTableWithName(ctx, t, conn, table) query := ` INSERT INTO %q (key, column1, column2, column3, column4, column5, column6, column7, "UppercaseColumn1") @@ -223,10 +223,10 @@ func SetupTestTableWithName(ctx context.Context, t *testing.T, conn Querier, tab is.NoErr(err) } -// SetupTestTable creates a new table and returns its name. -func SetupTestTable(ctx context.Context, t *testing.T, conn Querier) string { +// SetupTable creates a new table and returns its name. +func SetupTable(ctx context.Context, t *testing.T, conn Querier) string { table := RandomIdentifier(t) - SetupTestTableWithName(ctx, t, conn, table) + SetupTableWithName(ctx, t, conn, table) return table } From 74275b4222888c87a717c8cc95a6761030cffa25 Mon Sep 17 00:00:00 2001 From: Haris Osmanagic Date: Fri, 13 Jun 2025 17:04:44 +0200 Subject: [PATCH 18/60] update avro_integration_test --- source/schema/avro_integration_test.go | 315 ++++++++++++++++++++----- 1 file changed, 250 insertions(+), 65 deletions(-) diff --git a/source/schema/avro_integration_test.go b/source/schema/avro_integration_test.go index e83845e..da8b122 100644 --- a/source/schema/avro_integration_test.go +++ b/source/schema/avro_integration_test.go @@ -65,7 +65,7 @@ func Test_AvroExtract(t *testing.T) { t.Run("schema is parsable", func(t *testing.T) { is := is.New(t) is.NoErr(err) - is.Equal(sch, avroTestSchema(t, table)) + is.Equal(sch.String(), avroTestSchema(t, table).String()) _, err = avro.Parse(sch.String()) is.NoErr(err) @@ -123,22 +123,47 @@ func setupAvroTestTable(ctx context.Context, t *testing.T, conn test.Querier) st table := test.RandomIdentifier(t) query := ` - CREATE TABLE %s ( - col_boolean boolean, - col_bytea bytea, - col_varchar varchar(10), - col_date date, - col_float4 float4, - col_float8 float8, - col_int2 int2, - col_int4 int4, - col_int8 int8, - col_numeric numeric(8,2), - col_text text, - col_timestamp timestamp, - col_timestamptz timestamptz, - col_uuid uuid - )` + CREATE TABLE %s ( + id bigserial PRIMARY KEY, + col_bytea bytea, + col_bytea_not_null bytea NOT NULL, + col_varchar varchar(10), + col_varchar_not_null varchar(10) NOT NULL, + col_date date, + col_date_not_null date NOT NULL, + col_float4 float4, + col_float4_not_null float4 NOT NULL, + col_float8 float8, + col_float8_not_null float8 NOT NULL, + col_int2 int2, + col_int2_not_null int2 NOT NULL, + col_int4 int4, + col_int4_not_null int4 NOT NULL, + col_int8 int8, + col_int8_not_null int8 NOT NULL, + col_numeric numeric(8,2), + col_numeric_not_null numeric(8,2) NOT NULL, + col_text text, + col_text_not_null text NOT NULL, + col_timestamp timestamp, + col_timestamp_not_null timestamp NOT NULL, + col_timestamptz timestamptz, + col_timestamptz_not_null timestamptz NOT NULL, + col_uuid uuid, + col_uuid_not_null uuid NOT NULL, + col_json json, + col_json_not_null json NOT NULL, + col_jsonb jsonb, + col_jsonb_not_null jsonb NOT NULL, + col_bool bool, + col_bool_not_null bool NOT NULL, + col_serial serial, + col_serial_not_null serial NOT NULL, + col_smallserial smallserial, + col_smallserial_not_null smallserial NOT NULL, + col_bigserial bigserial, + col_bigserial_not_null bigserial NOT NULL + )` query = fmt.Sprintf(query, table) _, err := conn.Exec(ctx, query) is.NoErr(err) @@ -149,37 +174,85 @@ func setupAvroTestTable(ctx context.Context, t *testing.T, conn test.Querier) st func insertAvroTestRow(ctx context.Context, t *testing.T, conn test.Querier, table string) { is := is.New(t) query := ` - INSERT INTO %s ( - col_boolean, - col_bytea, - col_varchar, - col_date, - col_float4, - col_float8, - col_int2, - col_int4, - col_int8, - col_numeric, - col_text, - col_timestamp, - col_timestamptz, - col_uuid - ) VALUES ( - true, -- col_boolean - '\x07', -- col_bytea - '9', -- col_varchar - '2022-03-14', -- col_date - 15, -- col_float4 - 16.16, -- col_float8 - 32767, -- col_int2 - 2147483647, -- col_int4 - 9223372036854775807, -- col_int8 - '292929.29', -- col_numeric - 'foo bar baz', -- col_text - '2022-03-14 15:16:17', -- col_timestamp - '2022-03-14 15:16:17-08', -- col_timestamptz - 'bd94ee0b-564f-4088-bf4e-8d5e626caf66' -- col_uuid - )` + INSERT INTO %s ( + col_bytea, + col_bytea_not_null, + col_varchar, + col_varchar_not_null, + col_date, + col_date_not_null, + col_float4, + col_float4_not_null, + col_float8, + col_float8_not_null, + col_int2, + col_int2_not_null, + col_int4, + col_int4_not_null, + col_int8, + col_int8_not_null, + col_numeric, + col_numeric_not_null, + col_text, + col_text_not_null, + col_timestamp, + col_timestamp_not_null, + col_timestamptz, + col_timestamptz_not_null, + col_uuid, + col_uuid_not_null, + col_json, + col_json_not_null, + col_jsonb, + col_jsonb_not_null, + col_bool, + col_bool_not_null, + col_serial, + col_serial_not_null, + col_smallserial, + col_smallserial_not_null, + col_bigserial, + col_bigserial_not_null + ) VALUES ( + '\x07', -- col_bytea + '\x08', -- col_bytea_not_null + '9', -- col_varchar + '10', -- col_varchar_not_null + '2022-03-14', -- col_date + '2022-03-15', -- col_date_not_null + 15, -- col_float4 + 16, -- col_float4_not_null + 16.16, -- col_float8 + 17.17, -- col_float8_not_null + 32767, -- col_int2 + 32766, -- col_int2_not_null + 2147483647, -- col_int4 + 2147483646, -- col_int4_not_null + 9223372036854775807, -- col_int8 + 9223372036854775806, -- col_int8_not_null + '292929.29', -- col_numeric + '292928.28', -- col_numeric_not_null + 'foo bar baz', -- col_text + 'foo bar baz not null', -- col_text_not_null + '2022-03-14 15:16:17', -- col_timestamp + '2022-03-14 15:16:18', -- col_timestamp_not_null + '2022-03-14 15:16:17-08', -- col_timestamptz + '2022-03-14 15:16:18-08', -- col_timestamptz_not_null + 'bd94ee0b-564f-4088-bf4e-8d5e626caf66', -- col_uuid + 'bd94ee0b-564f-4088-bf4e-8d5e626caf67', -- col_uuid_not_null + '{"key": "value"}', -- col_json + '{"key": "value_not_null"}', -- col_json_not_null + '{"key": "value"}', -- col_jsonb + '{"key": "value_not_null"}', -- col_jsonb_not_null + true, -- col_bool + false, -- col_bool_not_null + 100, -- col_serial + 101, -- col_serial_not_null + 200, -- col_smallserial + 201, -- col_smallserial_not_null + 300, -- col_bigserial + 301 -- col_bigserial_not_null + )` query = fmt.Sprintf(query, table) _, err := conn.Exec(ctx, query) is.NoErr(err) @@ -189,35 +262,147 @@ func avroTestSchema(t *testing.T, table string) avro.Schema { is := is.New(t) fields := []*avro.Field{ - assert(avro.NewField("col_boolean", avro.NewPrimitiveSchema(avro.Boolean, nil))), - assert(avro.NewField("col_bytea", avro.NewPrimitiveSchema(avro.Bytes, nil))), - assert(avro.NewField("col_varchar", avro.NewPrimitiveSchema(avro.String, nil))), - assert(avro.NewField("col_float4", avro.NewPrimitiveSchema(avro.Float, nil))), - assert(avro.NewField("col_float8", avro.NewPrimitiveSchema(avro.Double, nil))), - assert(avro.NewField("col_int2", avro.NewPrimitiveSchema(avro.Int, nil))), - assert(avro.NewField("col_int4", avro.NewPrimitiveSchema(avro.Int, nil))), - assert(avro.NewField("col_int8", avro.NewPrimitiveSchema(avro.Long, nil))), - assert(avro.NewField("col_text", avro.NewPrimitiveSchema(avro.String, nil))), - assert(avro.NewField("col_numeric", avro.NewPrimitiveSchema( - avro.Bytes, - avro.NewDecimalLogicalSchema(8, 2), - ))), - assert(avro.NewField("col_date", avro.NewPrimitiveSchema( + // Primary key - bigserial (not null) + assert(avro.NewField("id", avro.NewPrimitiveSchema(avro.Long, nil))), + + // bytea fields + assert(avro.NewField("col_bytea", assert(avro.NewUnionSchema([]avro.Schema{ + avro.NewPrimitiveSchema(avro.Null, nil), + avro.NewPrimitiveSchema(avro.Bytes, nil), + })))), + assert(avro.NewField("col_bytea_not_null", avro.NewPrimitiveSchema(avro.Bytes, nil))), + + // varchar fields + assert(avro.NewField("col_varchar", assert(avro.NewUnionSchema([]avro.Schema{ + avro.NewPrimitiveSchema(avro.Null, nil), + avro.NewPrimitiveSchema(avro.String, nil), + })))), + assert(avro.NewField("col_varchar_not_null", avro.NewPrimitiveSchema(avro.String, nil))), + + // date fields + assert(avro.NewField("col_date", assert(avro.NewUnionSchema([]avro.Schema{ + avro.NewPrimitiveSchema(avro.Null, nil), + avro.NewPrimitiveSchema(avro.Int, avro.NewPrimitiveLogicalSchema(avro.Date)), + })))), + assert(avro.NewField("col_date_not_null", avro.NewPrimitiveSchema( avro.Int, avro.NewPrimitiveLogicalSchema(avro.Date), ))), - assert(avro.NewField("col_timestamp", avro.NewPrimitiveSchema( + + // float4 fields + assert(avro.NewField("col_float4", assert(avro.NewUnionSchema([]avro.Schema{ + avro.NewPrimitiveSchema(avro.Null, nil), + avro.NewPrimitiveSchema(avro.Float, nil), + })))), + assert(avro.NewField("col_float4_not_null", avro.NewPrimitiveSchema(avro.Float, nil))), + + // float8 fields + assert(avro.NewField("col_float8", assert(avro.NewUnionSchema([]avro.Schema{ + avro.NewPrimitiveSchema(avro.Null, nil), + avro.NewPrimitiveSchema(avro.Double, nil), + })))), + assert(avro.NewField("col_float8_not_null", avro.NewPrimitiveSchema(avro.Double, nil))), + + // int2 fields + assert(avro.NewField("col_int2", assert(avro.NewUnionSchema([]avro.Schema{ + avro.NewPrimitiveSchema(avro.Null, nil), + avro.NewPrimitiveSchema(avro.Int, nil), + })))), + assert(avro.NewField("col_int2_not_null", avro.NewPrimitiveSchema(avro.Int, nil))), + + // int4 fields + assert(avro.NewField("col_int4", assert(avro.NewUnionSchema([]avro.Schema{ + avro.NewPrimitiveSchema(avro.Null, nil), + avro.NewPrimitiveSchema(avro.Int, nil), + })))), + assert(avro.NewField("col_int4_not_null", avro.NewPrimitiveSchema(avro.Int, nil))), + + // int8 fields + assert(avro.NewField("col_int8", assert(avro.NewUnionSchema([]avro.Schema{ + avro.NewPrimitiveSchema(avro.Null, nil), + avro.NewPrimitiveSchema(avro.Long, nil), + })))), + assert(avro.NewField("col_int8_not_null", avro.NewPrimitiveSchema(avro.Long, nil))), + + // numeric fields + assert(avro.NewField("col_numeric", assert(avro.NewUnionSchema([]avro.Schema{ + avro.NewPrimitiveSchema(avro.Null, nil), + avro.NewPrimitiveSchema(avro.Bytes, avro.NewDecimalLogicalSchema(8, 2)), + })))), + assert(avro.NewField("col_numeric_not_null", avro.NewPrimitiveSchema( + avro.Bytes, + avro.NewDecimalLogicalSchema(8, 2), + ))), + + // text fields + assert(avro.NewField("col_text", assert(avro.NewUnionSchema([]avro.Schema{ + avro.NewPrimitiveSchema(avro.Null, nil), + avro.NewPrimitiveSchema(avro.String, nil), + })))), + assert(avro.NewField("col_text_not_null", avro.NewPrimitiveSchema(avro.String, nil))), + + // timestamp fields + assert(avro.NewField("col_timestamp", assert(avro.NewUnionSchema([]avro.Schema{ + avro.NewPrimitiveSchema(avro.Null, nil), + avro.NewPrimitiveSchema(avro.Long, avro.NewPrimitiveLogicalSchema(avro.LocalTimestampMicros)), + })))), + assert(avro.NewField("col_timestamp_not_null", avro.NewPrimitiveSchema( avro.Long, avro.NewPrimitiveLogicalSchema(avro.LocalTimestampMicros), ))), - assert(avro.NewField("col_timestamptz", avro.NewPrimitiveSchema( + + // timestamptz fields + assert(avro.NewField("col_timestamptz", assert(avro.NewUnionSchema([]avro.Schema{ + avro.NewPrimitiveSchema(avro.Null, nil), + avro.NewPrimitiveSchema(avro.Long, avro.NewPrimitiveLogicalSchema(avro.TimestampMicros)), + })))), + assert(avro.NewField("col_timestamptz_not_null", avro.NewPrimitiveSchema( avro.Long, avro.NewPrimitiveLogicalSchema(avro.TimestampMicros), ))), - assert(avro.NewField("col_uuid", avro.NewPrimitiveSchema( + + // uuid fields + assert(avro.NewField("col_uuid", assert(avro.NewUnionSchema([]avro.Schema{ + avro.NewPrimitiveSchema(avro.Null, nil), + avro.NewPrimitiveSchema(avro.String, avro.NewPrimitiveLogicalSchema(avro.UUID)), + })))), + assert(avro.NewField("col_uuid_not_null", avro.NewPrimitiveSchema( avro.String, avro.NewPrimitiveLogicalSchema(avro.UUID), ))), + + // json fields (represented as strings in Avro) + assert(avro.NewField("col_json", assert(avro.NewUnionSchema([]avro.Schema{ + avro.NewPrimitiveSchema(avro.Null, nil), + avro.NewPrimitiveSchema(avro.Bytes, nil), + })))), + assert(avro.NewField("col_json_not_null", avro.NewPrimitiveSchema(avro.Bytes, nil))), + + // jsonb fields (represented as strings in Avro) + assert(avro.NewField("col_jsonb", assert(avro.NewUnionSchema([]avro.Schema{ + avro.NewPrimitiveSchema(avro.Null, nil), + avro.NewPrimitiveSchema(avro.Bytes, nil), + })))), + assert(avro.NewField("col_jsonb_not_null", avro.NewPrimitiveSchema(avro.Bytes, nil))), + + // bool fields + assert(avro.NewField("col_bool", assert(avro.NewUnionSchema([]avro.Schema{ + avro.NewPrimitiveSchema(avro.Null, nil), + avro.NewPrimitiveSchema(avro.Boolean, nil), + })))), + assert(avro.NewField("col_bool_not_null", avro.NewPrimitiveSchema(avro.Boolean, nil))), + + // serial fields (represented as int in Avro) + assert(avro.NewField("col_serial", avro.NewPrimitiveSchema(avro.Int, nil))), + assert(avro.NewField("col_serial_not_null", avro.NewPrimitiveSchema(avro.Int, nil))), + + // smallserial fields (represented as int in Avro) + assert(avro.NewField("col_smallserial", avro.NewPrimitiveSchema(avro.Int, nil))), + assert(avro.NewField("col_smallserial_not_null", avro.NewPrimitiveSchema(avro.Int, nil))), + + // bigserial fields (represented as long in Avro) + assert(avro.NewField("col_bigserial", avro.NewPrimitiveSchema(avro.Long, nil))), + assert(avro.NewField("col_bigserial_not_null", avro.NewPrimitiveSchema(avro.Long, nil))), } slices.SortFunc(fields, func(a, b *avro.Field) int { From 2f9de012f5c56878a677eb4df4a82fe7c4df9568 Mon Sep 17 00:00:00 2001 From: Haris Osmanagic Date: Fri, 13 Jun 2025 19:42:15 +0200 Subject: [PATCH 19/60] fix tests --- destination_integration_test.go | 28 +++-- source/logrepl/cdc_test.go | 58 ++++------ source/logrepl/combined_test.go | 40 ++----- source/logrepl/internal/relationset_test.go | 101 +++++++++--------- source/logrepl/internal/subscription_test.go | 18 ++-- .../snapshot/fetch_worker_integration_test.go | 18 +--- test/helper.go | 68 ++++-------- 7 files changed, 135 insertions(+), 196 deletions(-) diff --git a/destination_integration_test.go b/destination_integration_test.go index 14ba4e9..fe58045 100644 --- a/destination_integration_test.go +++ b/destination_integration_test.go @@ -70,10 +70,11 @@ func TestDestination_Write(t *testing.T) { Key: opencdc.StructuredData{"id": 5000}, Payload: opencdc.Change{ After: opencdc.StructuredData{ + "key": []uint8("123"), "column1": "foo", "column2": 123, "column3": true, - "column4": nil, + "column4": big.NewRat(123, 10), "UppercaseColumn1": 222, }, }, @@ -88,10 +89,11 @@ func TestDestination_Write(t *testing.T) { Key: opencdc.StructuredData{"id": 5}, Payload: opencdc.Change{ After: opencdc.StructuredData{ + "key": []uint8("234"), "column1": "foo", "column2": 456, "column3": false, - "column4": nil, + "column4": big.NewRat(123, 10), "UppercaseColumn1": 333, }, }, @@ -106,10 +108,11 @@ func TestDestination_Write(t *testing.T) { Key: opencdc.StructuredData{"id": 6}, Payload: opencdc.Change{ After: opencdc.StructuredData{ + "key": []uint8("345"), "column1": "bar", "column2": 567, "column3": true, - "column4": nil, + "column4": big.NewRat(123, 10), "UppercaseColumn1": 444, }, }, @@ -124,10 +127,11 @@ func TestDestination_Write(t *testing.T) { Key: opencdc.StructuredData{"id": 1}, Payload: opencdc.Change{ After: opencdc.StructuredData{ + "key": []uint8("456"), "column1": "foobar", "column2": 567, "column3": true, - "column4": nil, + "column4": big.NewRat(123, 10), "UppercaseColumn1": 555, }, }, @@ -151,6 +155,7 @@ func TestDestination_Write(t *testing.T) { Key: opencdc.StructuredData{"id": 123}, Payload: opencdc.Change{ After: opencdc.StructuredData{ + "key": []uint8("567"), "column1": "abcdef", "column2": 567, "column3": true, @@ -223,10 +228,11 @@ func TestDestination_Batch(t *testing.T) { Key: opencdc.StructuredData{"id": 5}, Payload: opencdc.Change{ After: opencdc.StructuredData{ + "key": []uint8("123"), "column1": "foo1", "column2": 1, "column3": false, - "column4": nil, + "column4": big.NewRat(123, 10), "UppercaseColumn1": 111, }, }, @@ -237,10 +243,11 @@ func TestDestination_Batch(t *testing.T) { Key: opencdc.StructuredData{"id": 6}, Payload: opencdc.Change{ After: opencdc.StructuredData{ + "key": []uint8("234"), "column1": "foo2", "column2": 2, "column3": true, - "column4": nil, + "column4": big.NewRat(123, 10), "UppercaseColumn1": 222, }, }, @@ -251,10 +258,11 @@ func TestDestination_Batch(t *testing.T) { Key: opencdc.StructuredData{"id": 7}, Payload: opencdc.Change{ After: opencdc.StructuredData{ + "key": []uint8("345"), "column1": "foo3", "column2": 3, "column3": false, - "column4": nil, + "column4": big.NewRat(123, 10), "UppercaseColumn1": 333, }, }, @@ -275,11 +283,12 @@ func TestDestination_Batch(t *testing.T) { func queryTestTable(ctx context.Context, conn test.Querier, tableName string, id any) (opencdc.StructuredData, error) { row := conn.QueryRow( ctx, - fmt.Sprintf(`SELECT column1, column2, column3, column4, "UppercaseColumn1" FROM %q WHERE id = $1`, tableName), + fmt.Sprintf(`SELECT key, column1, column2, column3, column4, "UppercaseColumn1" FROM %q WHERE id = $1`, tableName), id, ) var ( + key []uint8 col1 string col2 int col3 bool @@ -287,7 +296,7 @@ func queryTestTable(ctx context.Context, conn test.Querier, tableName string, id uppercaseCol1 int ) - err := row.Scan(&col1, &col2, &col3, &col4Str, &uppercaseCol1) + err := row.Scan(&key, &col1, &col2, &col3, &col4Str, &uppercaseCol1) if err != nil { return nil, err } @@ -301,6 +310,7 @@ func queryTestTable(ctx context.Context, conn test.Querier, tableName string, id } return opencdc.StructuredData{ + "key": key, "column1": col1, "column2": col2, "column3": col3, diff --git a/source/logrepl/cdc_test.go b/source/logrepl/cdc_test.go index aec2770..6abf391 100644 --- a/source/logrepl/cdc_test.go +++ b/source/logrepl/cdc_test.go @@ -141,8 +141,8 @@ func TestCDCIterator_Operation_NextN(t *testing.T) { name: "should detect insert", setup: func(t *testing.T) { is := is.New(t) - query := fmt.Sprintf(`INSERT INTO %s (id, column1, column2, column3, column4, column5, column6, column7) - VALUES (6, 'bizz', 456, false, 12.3, 14, '{"foo2": "bar2"}', '{"foo2": "baz2"}')`, table) + query := fmt.Sprintf(`INSERT INTO %s (id, key, column1, column2, column3, column4, "UppercaseColumn1") + VALUES (6, '6', 'bizz', 456, false, 12.3, 61)`, table) _, err := pool.Exec(ctx, query) is.NoErr(err) }, @@ -165,11 +165,8 @@ func TestCDCIterator_Operation_NextN(t *testing.T) { "column2": int32(456), "column3": false, "column4": big.NewRat(123, 10), - "column5": big.NewRat(14, 1), - "column6": []byte(`{"foo2": "bar2"}`), - "column7": []byte(`{"foo2": "baz2"}`), - "key": nil, - "UppercaseColumn1": nil, + "key": []uint8("6"), + "UppercaseColumn1": int32(61), }, }, }, @@ -200,9 +197,6 @@ func TestCDCIterator_Operation_NextN(t *testing.T) { "column2": int32(123), "column3": false, "column4": big.NewRat(122, 10), - "column5": big.NewRat(4, 1), - "column6": []byte(`{"foo": "bar"}`), - "column7": []byte(`{"foo": "baz"}`), "key": []uint8("1"), "UppercaseColumn1": int32(1), }, @@ -237,9 +231,6 @@ func TestCDCIterator_Operation_NextN(t *testing.T) { "column2": int32(123), "column3": false, "column4": big.NewRat(122, 10), - "column5": big.NewRat(4, 1), - "column6": []byte(`{"foo": "bar"}`), - "column7": []byte(`{"foo": "baz"}`), "key": []uint8("1"), "UppercaseColumn1": int32(1), }, @@ -249,9 +240,6 @@ func TestCDCIterator_Operation_NextN(t *testing.T) { "column2": int32(123), "column3": false, "column4": big.NewRat(122, 10), - "column5": big.NewRat(4, 1), - "column6": []byte(`{"foo": "bar"}`), - "column7": []byte(`{"foo": "baz"}`), "key": []uint8("1"), "UppercaseColumn1": int32(1), }, @@ -286,9 +274,6 @@ func TestCDCIterator_Operation_NextN(t *testing.T) { "column2": nil, "column3": nil, "column4": nil, - "column5": nil, - "column6": nil, - "column7": nil, "key": nil, "UppercaseColumn1": nil, }, @@ -323,10 +308,7 @@ func TestCDCIterator_Operation_NextN(t *testing.T) { "column1": "baz", "column2": int32(789), "column3": false, - "column4": nil, - "column5": big.NewRat(9, 1), - "column6": []byte(`{"foo": "bar"}`), - "column7": []byte(`{"foo": "baz"}`), + "column4": big.NewRat(836, 25), "UppercaseColumn1": int32(3), }, }, @@ -379,8 +361,8 @@ func TestCDCIterator_EnsureLSN(t *testing.T) { i := testCDCIterator(ctx, t, pool, table, true) <-i.sub.Ready() - _, err := pool.Exec(ctx, fmt.Sprintf(`INSERT INTO %s (id, column1, column2, column3, column4, column5) - VALUES (6, 'bizz', 456, false, 12.3, 14)`, table)) + _, err := pool.Exec(ctx, fmt.Sprintf(`INSERT INTO %s (id, key, column1, column2, column3, column4, "UppercaseColumn1") + VALUES (6, '6', 'bizz', 456, false, 12.3, 6)`, table)) is.NoErr(err) rr, err := i.NextN(ctx, 1) @@ -475,8 +457,8 @@ func TestCDCIterator_NextN(t *testing.T) { <-i.sub.Ready() for j := 1; j <= 3; j++ { - _, err := pool.Exec(ctx, fmt.Sprintf(`INSERT INTO %s (id, column1, column2, column3, column4, column5) - VALUES (%d, 'test-%d', %d, false, 12.3, 14)`, table, j+10, j, j*100)) + _, err := pool.Exec(ctx, fmt.Sprintf(`INSERT INTO %s (id, key, column1, column2, column3, column4, "UppercaseColumn1") + VALUES (%d, '%d', 'test-%d', %d, false, 12.3, 4)`, table, j+10, j+10, j, j*100)) is.NoErr(err) } @@ -511,8 +493,8 @@ func TestCDCIterator_NextN(t *testing.T) { <-i.sub.Ready() for j := 1; j <= 2; j++ { - _, err := pool.Exec(ctx, fmt.Sprintf(`INSERT INTO %s (id, column1, column2, column3, column4, column5) - VALUES (%d, 'test-%d', %d, false, 12.3, 14)`, table, j+20, j, j*100)) + _, err := pool.Exec(ctx, fmt.Sprintf(`INSERT INTO %s (id, key, column1, column2, column3, column4, "UppercaseColumn1") + VALUES (%d, '%d', 'test-%d', %d, false, 12.3, 4)`, table, j+20, j+20, j, j*100)) is.NoErr(err) } @@ -578,8 +560,8 @@ func TestCDCIterator_NextN(t *testing.T) { i := testCDCIterator(ctx, t, pool, table, true) <-i.sub.Ready() - _, err := pool.Exec(ctx, fmt.Sprintf(`INSERT INTO %s (id, column1, column2, column3, column4, column5) - VALUES (30, 'test-1', 100, false, 12.3, 14)`, table)) + _, err := pool.Exec(ctx, fmt.Sprintf(`INSERT INTO %s (id, key, column1, column2, column3, column4, "UppercaseColumn1") + VALUES (30, '30', 'test-1', 100, false, 12.3, 14)`, table)) is.NoErr(err) go func() { @@ -665,8 +647,8 @@ func TestCDCIterator_Schema(t *testing.T) { _, err := pool.Exec( ctx, - fmt.Sprintf(`INSERT INTO %s (id, column1, column2, column3, column4, column5) - VALUES (6, 'bizz', 456, false, 12.3, 14)`, table), + fmt.Sprintf(`INSERT INTO %s (id, key, column1, column2, column3, column4, "UppercaseColumn1") + VALUES (6, '6', 'bizz', 456, false, 12.3, 6)`, table), ) is.NoErr(err) @@ -688,8 +670,8 @@ func TestCDCIterator_Schema(t *testing.T) { _, err = pool.Exec( ctx, - fmt.Sprintf(`INSERT INTO %s (id, key, column1, column2, column3, column4, column5, column6, column7, column101) - VALUES (7, decode('aabbcc', 'hex'), 'example data 1', 100, true, 12345.678, 12345, '{"foo":"bar"}', '{"foo2":"baz2"}', '2023-09-09 10:00:00');`, table), + fmt.Sprintf(`INSERT INTO %s (id, key, column1, column2, column3, column4, column101, "UppercaseColumn1") + VALUES (7, decode('aabbcc', 'hex'), 'example data 1', 100, true, 12345.678, '2023-09-09 10:00:00', 7);`, table), ) is.NoErr(err) @@ -706,13 +688,13 @@ func TestCDCIterator_Schema(t *testing.T) { t.Run("column removed", func(t *testing.T) { is := is.New(t) - _, err := pool.Exec(ctx, fmt.Sprintf(`ALTER TABLE %s DROP COLUMN column4, DROP COLUMN column5;`, table)) + _, err := pool.Exec(ctx, fmt.Sprintf(`ALTER TABLE %s DROP COLUMN column4;`, table)) is.NoErr(err) _, err = pool.Exec( ctx, - fmt.Sprintf(`INSERT INTO %s (id, key, column1, column2, column3, column6, column7, column101) - VALUES (8, decode('aabbcc', 'hex'), 'example data 1', 100, true, '{"foo":"bar"}', '{"foo2":"baz2"}', '2023-09-09 10:00:00');`, table), + fmt.Sprintf(`INSERT INTO %s (id, key, column1, column2, column3, column101, "UppercaseColumn1") + VALUES (8, decode('aabbcc', 'hex'), 'example data 1', 100, true, '2023-09-09 10:00:00', 8);`, table), ) is.NoErr(err) diff --git a/source/logrepl/combined_test.go b/source/logrepl/combined_test.go index dd01184..6669762 100644 --- a/source/logrepl/combined_test.go +++ b/source/logrepl/combined_test.go @@ -158,8 +158,8 @@ func TestCombinedIterator_NextN(t *testing.T) { // Add a record to the table for CDC mode testing _, err = pool.Exec(ctx, fmt.Sprintf( - `INSERT INTO %s (id, column1, column2, column3, column4, column5, column6, column7) - VALUES (6, 'bizz', 1010, false, 872.2, 101, '{"foo12": "bar12"}', '{"foo13": "bar13"}')`, + `INSERT INTO %s (id, key, column1, column2, column3, column4, "UppercaseColumn1") + VALUES (6, '6', 'bizz', 1010, false, 872.2, 6)`, table, )) is.NoErr(err) @@ -276,15 +276,15 @@ func TestCombinedIterator_NextN(t *testing.T) { // Insert two more records for testing CDC batch _, err = pool.Exec(ctx, fmt.Sprintf( - `INSERT INTO %s (id, column1, column2, column3, column4, column5, column6, column7) - VALUES (7, 'buzz', 10101, true, 121.9, 51, '{"foo7": "bar7"}', '{"foo8": "bar8"}')`, + `INSERT INTO %s (id, key, column1, column2, column3, column4, "UppercaseColumn1") + VALUES (7, '7', 'buzz', 10101, true, 121.9, 7)`, table, )) is.NoErr(err) _, err = pool.Exec(ctx, fmt.Sprintf( - `INSERT INTO %s (id, column1, column2, column3, column4, column5, column6, column7) - VALUES (8, 'fizz', 20202, false, 232.8, 62, '{"foo9": "bar9"}', '{"foo10": "bar10"}')`, + `INSERT INTO %s (id, key, column1, column2, column3, column4, "UppercaseColumn1") + VALUES (8, '8', 'fizz', 20202, false, 232.8, 8)`, table, )) is.NoErr(err) @@ -369,9 +369,6 @@ func testRecords() []opencdc.StructuredData { "column2": int32(123), "column3": false, "column4": big.NewRat(122, 10), - "column5": big.NewRat(4, 1), - "column6": []byte(`{"foo": "bar"}`), - "column7": []byte(`{"foo": "baz"}`), "UppercaseColumn1": int32(1), }, { @@ -381,9 +378,6 @@ func testRecords() []opencdc.StructuredData { "column2": int32(456), "column3": true, "column4": big.NewRat(1342, 100), // 13.42 - "column5": big.NewRat(8, 1), - "column6": []byte(`{"foo": "bar"}`), - "column7": []byte(`{"foo": "baz"}`), "UppercaseColumn1": int32(2), }, { @@ -393,9 +387,6 @@ func testRecords() []opencdc.StructuredData { "column2": int32(789), "column3": false, "column4": nil, - "column5": big.NewRat(9, 1), - "column6": []byte(`{"foo": "bar"}`), - "column7": []byte(`{"foo": "baz"}`), "UppercaseColumn1": int32(3), }, { @@ -405,34 +396,25 @@ func testRecords() []opencdc.StructuredData { "column2": nil, "column3": nil, "column4": big.NewRat(911, 10), // 91.1 - "column5": nil, - "column6": nil, - "column7": nil, - "UppercaseColumn1": nil, + "UppercaseColumn1": int32(4), }, { "id": int64(6), - "key": nil, + "key": []uint8("6"), "column1": "bizz", "column2": int32(1010), "column3": false, "column4": big.NewRat(8722, 10), // 872.2 - "column5": big.NewRat(101, 1), - "column6": []byte(`{"foo12": "bar12"}`), - "column7": []byte(`{"foo13": "bar13"}`), - "UppercaseColumn1": nil, + "UppercaseColumn1": int32(6), }, { "id": int64(7), - "key": nil, + "key": []uint8("7"), "column1": "buzz", "column2": int32(10101), "column3": true, "column4": big.NewRat(1219, 10), // 121.9 - "column5": big.NewRat(51, 1), - "column6": []byte(`{"foo7": "bar7"}`), - "column7": []byte(`{"foo8": "bar8"}`), - "UppercaseColumn1": nil, + "UppercaseColumn1": int32(7), }, } } diff --git a/source/logrepl/internal/relationset_test.go b/source/logrepl/internal/relationset_test.go index c120bfe..e548494 100644 --- a/source/logrepl/internal/relationset_test.go +++ b/source/logrepl/internal/relationset_test.go @@ -23,6 +23,7 @@ import ( "testing" "time" + "github.com/conduitio/conduit-connector-postgres/source/common" "github.com/conduitio/conduit-connector-postgres/test" "github.com/google/go-cmp/cmp" "github.com/jackc/pglogrepl" @@ -49,7 +50,6 @@ func TestRelationSetAllTypes(t *testing.T) { is := is.New(t) pool := test.ConnectPool(ctx, t, test.RepmgrConnString) - table := setupTableAllTypes(ctx, t, pool) _, messages := setupSubscription(ctx, t, pool, table) insertRowAllTypes(ctx, t, pool, table) @@ -77,6 +77,10 @@ func TestRelationSetAllTypes(t *testing.T) { break } + tableInfo := common.NewTableInfoFetcher(pool) + err := tableInfo.Refresh(ctx, table) + is.NoErr(err) + rs := NewRelationSet() rs.Add(rel) @@ -87,7 +91,7 @@ func TestRelationSetAllTypes(t *testing.T) { t.Run("with builtin plugin", func(t *testing.T) { is := is.New(t) - values, err := rs.Values(ins.RelationID, ins.Tuple) + values, err := rs.Values(ins.RelationID, ins.Tuple, tableInfo.GetTable(table)) is.NoErr(err) isValuesAllTypes(is, values) }) @@ -95,7 +99,7 @@ func TestRelationSetAllTypes(t *testing.T) { t.Run("with standalone plugin", func(t *testing.T) { is := is.New(t) - values, err := rs.Values(ins.RelationID, ins.Tuple) + values, err := rs.Values(ins.RelationID, ins.Tuple, tableInfo.GetTable(table)) is.NoErr(err) isValuesAllTypesStandalone(is, values) }) @@ -105,52 +109,51 @@ func TestRelationSetAllTypes(t *testing.T) { func setupTableAllTypes(ctx context.Context, t *testing.T, conn test.Querier) string { is := is.New(t) table := test.RandomIdentifier(t) - query := ` - CREATE TABLE %s ( - id bigserial PRIMARY KEY, - col_bit bit(8), - col_varbit varbit(8), - col_boolean boolean, - col_box box, - col_bytea bytea, - col_char char(3), - col_varchar varchar(10), - col_cidr cidr, - col_circle circle, - col_date date, - col_float4 float4, - col_float8 float8, - col_inet inet, - col_int2 int2, - col_int4 int4, - col_int8 int8, - col_interval interval, - col_json json, - col_jsonb jsonb, - col_line line, - col_lseg lseg, - col_macaddr macaddr, - col_macaddr8 macaddr8, - col_money money, - col_numeric numeric(8,2), - col_path path, - col_pg_lsn pg_lsn, - col_pg_snapshot pg_snapshot, - col_point point, - col_polygon polygon, - col_serial2 serial2, - col_serial4 serial4, - col_serial8 serial8, - col_text text, - col_time time, - col_timetz timetz, - col_timestamp timestamp, - col_timestamptz timestamptz, - col_tsquery tsquery, - col_tsvector tsvector, - col_uuid uuid, - col_xml xml - )` + query := `CREATE TABLE %s ( + id bigserial PRIMARY KEY, + col_bit bit(8) NOT NULL, + col_varbit varbit(8) NOT NULL, + col_boolean boolean NOT NULL, + col_box box NOT NULL, + col_bytea bytea NOT NULL, + col_char char(3) NOT NULL, + col_varchar varchar(10) NOT NULL, + col_cidr cidr NOT NULL, + col_circle circle NOT NULL, + col_date date NOT NULL, + col_float4 float4 NOT NULL, + col_float8 float8 NOT NULL, + col_inet inet NOT NULL, + col_int2 int2 NOT NULL, + col_int4 int4 NOT NULL, + col_int8 int8 NOT NULL, + col_interval interval NOT NULL, + col_json json NOT NULL, + col_jsonb jsonb NOT NULL, + col_line line NOT NULL, + col_lseg lseg NOT NULL, + col_macaddr macaddr NOT NULL, + col_macaddr8 macaddr8 NOT NULL, + col_money money NOT NULL, + col_numeric numeric(8,2) NOT NULL, + col_path path NOT NULL, + col_pg_lsn pg_lsn NOT NULL, + col_pg_snapshot pg_snapshot NOT NULL, + col_point point NOT NULL, + col_polygon polygon NOT NULL, + col_serial2 serial2 NOT NULL, + col_serial4 serial4 NOT NULL, + col_serial8 serial8 NOT NULL, + col_text text NOT NULL, + col_time time NOT NULL, + col_timetz timetz NOT NULL, + col_timestamp timestamp NOT NULL, + col_timestamptz timestamptz NOT NULL, + col_tsquery tsquery NOT NULL, + col_tsvector tsvector NOT NULL, + col_uuid uuid NOT NULL, + col_xml xml NOT NULL +)` query = fmt.Sprintf(query, table) _, err := conn.Exec(ctx, query) is.NoErr(err) diff --git a/source/logrepl/internal/subscription_test.go b/source/logrepl/internal/subscription_test.go index 16baae6..ad6ae99 100644 --- a/source/logrepl/internal/subscription_test.go +++ b/source/logrepl/internal/subscription_test.go @@ -64,9 +64,9 @@ func TestSubscription_WithRepmgr(t *testing.T) { t.Run("first insert table1", func(t *testing.T) { is := is.New(t) - query := `INSERT INTO %s (id, column1, column2, column3) - VALUES (6, 'bizz', 456, false)` - _, err := pool.Exec(ctx, fmt.Sprintf(query, table1)) + query := fmt.Sprintf(`INSERT INTO %s (id, key, column1, column2, column3, column4, "UppercaseColumn1") + VALUES (6, '6', 'bizz', 456, false, 12.3, 61)`, table1) + _, err := pool.Exec(ctx, query) is.NoErr(err) _ = fetchAndAssertMessageTypes( @@ -82,9 +82,9 @@ func TestSubscription_WithRepmgr(t *testing.T) { t.Run("second insert table1", func(t *testing.T) { is := is.New(t) - query := `INSERT INTO %s (id, column1, column2, column3) - VALUES (7, 'bizz', 456, false)` - _, err := pool.Exec(ctx, fmt.Sprintf(query, table1)) + query := fmt.Sprintf(`INSERT INTO %s (id, key, column1, column2, column3, column4, "UppercaseColumn1") + VALUES (7, '7', 'bizz', 456, false, 12.3, 61)`, table1) + _, err := pool.Exec(ctx, query) is.NoErr(err) _ = fetchAndAssertMessageTypes( @@ -162,9 +162,9 @@ func TestSubscription_ClosedContext(t *testing.T) { sub, messages := setupSubscription(ctx, t, pool, table) // insert to get new messages into publication - query := `INSERT INTO %s (id, column1, column2, column3) - VALUES (6, 'bizz', 456, false)` - _, err := pool.Exec(ctx, fmt.Sprintf(query, table)) + query := fmt.Sprintf(`INSERT INTO %s (id, key, column1, column2, column3, column4, "UppercaseColumn1") + VALUES (6, '6', 'bizz', 456, false, 12.3, 61)`, table) + _, err := pool.Exec(ctx, query) is.NoErr(err) cancel() diff --git a/source/snapshot/fetch_worker_integration_test.go b/source/snapshot/fetch_worker_integration_test.go index 01047c7..09966c1 100644 --- a/source/snapshot/fetch_worker_integration_test.go +++ b/source/snapshot/fetch_worker_integration_test.go @@ -277,16 +277,11 @@ func Test_FetcherRun_Initial(t *testing.T) { is.NoErr(tt.Err()) is.Equal(len(gotFetchData), 4) - var ( - value6 = []byte(`{"foo": "bar"}`) - value7 = []byte(`{"foo": "baz"}`) - ) - expectedMatch := []opencdc.StructuredData{ - {"id": int64(1), "key": []uint8{49}, "column1": "foo", "column2": int32(123), "column3": false, "column4": big.NewRat(122, 10), "column5": big.NewRat(4, 1), "column6": value6, "column7": value7, "UppercaseColumn1": int32(1)}, - {"id": int64(2), "key": []uint8{50}, "column1": "bar", "column2": int32(456), "column3": true, "column4": big.NewRat(1342, 100), "column5": big.NewRat(8, 1), "column6": value6, "column7": value7, "UppercaseColumn1": int32(2)}, - {"id": int64(3), "key": []uint8{51}, "column1": "baz", "column2": int32(789), "column3": false, "column4": nil, "column5": big.NewRat(9, 1), "column6": value6, "column7": value7, "UppercaseColumn1": int32(3)}, - {"id": int64(4), "key": []uint8{52}, "column1": nil, "column2": nil, "column3": nil, "column4": big.NewRat(911, 10), "column5": nil, "column6": nil, "column7": nil, "UppercaseColumn1": nil}, + {"id": int64(1), "key": []uint8{49}, "column1": "foo", "column2": int32(123), "column3": false, "column4": big.NewRat(122, 10), "UppercaseColumn1": int32(1)}, + {"id": int64(2), "key": []uint8{50}, "column1": "bar", "column2": int32(456), "column3": true, "column4": big.NewRat(1342, 100), "UppercaseColumn1": int32(2)}, + {"id": int64(3), "key": []uint8{51}, "column1": "baz", "column2": int32(789), "column3": false, "column4": big.NewRat(836, 25), "UppercaseColumn1": int32(3)}, + {"id": int64(4), "key": []uint8{52}, "column1": "qux", "column2": int32(444), "column3": false, "column4": big.NewRat(911, 10), "UppercaseColumn1": int32(4)}, } for i, got := range gotFetchData { @@ -364,10 +359,7 @@ func Test_FetcherRun_Resume(t *testing.T) { "column1": "baz", "column2": int32(789), "column3": false, - "column4": nil, - "column5": big.NewRat(9, 1), - "column6": []byte(`{"foo": "bar"}`), - "column7": []byte(`{"foo": "baz"}`), + "column4": big.NewRat(836, 25), "UppercaseColumn1": int32(3), }, cmp.Comparer(func(x, y *big.Rat) bool { diff --git a/test/helper.go b/test/helper.go index 8956726..616fa78 100644 --- a/test/helper.go +++ b/test/helper.go @@ -44,9 +44,8 @@ const RegularConnString = "postgres://meroxauser:meroxapass@127.0.0.1:5433/merox const TestTableAvroSchemaV1 = `{ "type": "record", "name": "%s", - "fields": - [ - {"name":"UppercaseColumn1","type":"int"}, + "fields": [ + {"name":"UppercaseColumn1","type":"int"}, {"name":"column1","type":"string"}, {"name":"column2","type":"int"}, {"name":"column3","type":"boolean"}, @@ -60,17 +59,6 @@ const TestTableAvroSchemaV1 = `{ "scale": 3 } }, - { - "name": "column5", - "type": - { - "type": "bytes", - "logicalType": "decimal", - "precision": 5 - } - }, - {"name":"column6","type":"bytes"}, - {"name":"column7","type":"bytes"}, {"name":"id","type":"long"}, {"name":"key","type":"bytes"} ] @@ -80,11 +68,10 @@ const TestTableAvroSchemaV1 = `{ const TestTableAvroSchemaV2 = `{ "type": "record", "name": "%s", - "fields": - [ - {"name":"UppercaseColumn1","type":"int"}, + "fields": [ + {"name":"UppercaseColumn1","type":"int"}, {"name":"column1","type":"string"}, - {"name":"column101","type":{"type":"long","logicalType":"local-timestamp-micros"}}, + {"name":"column101","type":["null", {"type":"long","logicalType":"local-timestamp-micros"}]}, {"name":"column2","type":"int"}, {"name":"column3","type":"boolean"}, { @@ -97,17 +84,6 @@ const TestTableAvroSchemaV2 = `{ "scale": 3 } }, - { - "name": "column5", - "type": - { - "type": "bytes", - "logicalType": "decimal", - "precision": 5 - } - }, - {"name":"column6","type":"bytes"}, - {"name":"column7","type":"bytes"}, {"name":"id","type":"long"}, {"name":"key","type":"bytes"} ] @@ -117,15 +93,12 @@ const TestTableAvroSchemaV2 = `{ const TestTableAvroSchemaV3 = `{ "type": "record", "name": "%s", - "fields": - [ - {"name":"UppercaseColumn1","type":"int"}, + "fields": [ + {"name":"UppercaseColumn1","type":"int"}, {"name":"column1","type":"string"}, - {"name":"column101","type":{"type":"long","logicalType":"local-timestamp-micros"}}, + {"name":"column101","type":["null", {"type":"long","logicalType":"local-timestamp-micros"}]}, {"name":"column2","type":"int"}, {"name":"column3","type":"boolean"}, - {"name":"column6","type":"bytes"}, - {"name":"column7","type":"bytes"}, {"name":"id","type":"long"}, {"name":"key","type":"bytes"} ] @@ -145,15 +118,12 @@ const TestTableKeyAvroSchema = `{ const testTableCreateQuery = ` CREATE TABLE %q ( id bigserial PRIMARY KEY, - key bytea, - column1 varchar(256), - column2 integer, - column3 boolean, - column4 numeric(16,3), - column5 numeric(5), - column6 jsonb, - column7 json, - "UppercaseColumn1" integer + key bytea not null, + column1 varchar(256) not null, + column2 integer not null, + column3 boolean not null, + column4 numeric(16,3) not null, + "UppercaseColumn1" integer not null )` type Querier interface { @@ -213,11 +183,11 @@ func SetupTableWithName(ctx context.Context, t *testing.T, conn Querier, table s SetupEmptyTableWithName(ctx, t, conn, table) query := ` - INSERT INTO %q (key, column1, column2, column3, column4, column5, column6, column7, "UppercaseColumn1") - VALUES ('1', 'foo', 123, false, 12.2, 4, '{"foo": "bar"}', '{"foo": "baz"}', 1), - ('2', 'bar', 456, true, 13.42, 8, '{"foo": "bar"}', '{"foo": "baz"}', 2), - ('3', 'baz', 789, false, null, 9, '{"foo": "bar"}', '{"foo": "baz"}', 3), - ('4', null, null, null, 91.1, null, null, null, null)` + INSERT INTO %q (key, column1, column2, column3, column4, "UppercaseColumn1") + VALUES ('1', 'foo', 123, false, 12.2, 1), + ('2', 'bar', 456, true, 13.42, 2), + ('3', 'baz', 789, false, 33.44, 3), + ('4', 'qux', 444, false, 91.1, 4)` query = fmt.Sprintf(query, table) _, err := conn.Exec(ctx, query) is.NoErr(err) From 84d8a206f6995b6ca8583623307822b54eaca4c5 Mon Sep 17 00:00:00 2001 From: Haris Osmanagic Date: Fri, 13 Jun 2025 19:56:07 +0200 Subject: [PATCH 20/60] more tests fixed --- .../snapshot/fetch_worker_integration_test.go | 27 ------------------- 1 file changed, 27 deletions(-) diff --git a/source/snapshot/fetch_worker_integration_test.go b/source/snapshot/fetch_worker_integration_test.go index 09966c1..0c585fb 100644 --- a/source/snapshot/fetch_worker_integration_test.go +++ b/source/snapshot/fetch_worker_integration_test.go @@ -21,14 +21,12 @@ import ( "math/big" "strings" "testing" - "time" "github.com/conduitio/conduit-commons/opencdc" "github.com/conduitio/conduit-connector-postgres/source/position" "github.com/conduitio/conduit-connector-postgres/test" "github.com/google/go-cmp/cmp" "github.com/jackc/pgx/v5" - "github.com/jackc/pgx/v5/pgconn" "github.com/jackc/pgx/v5/pgxpool" "github.com/matryer/is" "gopkg.in/tomb.v2" @@ -457,31 +455,6 @@ func Test_send(t *testing.T) { is.Equal(err, context.Canceled) } -func Test_FetchWorker_buildRecordData(t *testing.T) { - var ( - is = is.New(t) - now = time.Now().UTC() - - // special case fields - fields = []pgconn.FieldDescription{{Name: "id"}, {Name: "time"}} - values = []any{1, now} - expectValues = []any{1, now} - ) - - key, payload, err := (&FetchWorker{ - conf: FetchConfig{Table: "mytable", Key: "id"}, - }).buildRecordData(fields, values) - - is.NoErr(err) - is.Equal(len(payload), 2) - for i, fd := range fields { - is.Equal(payload[fd.Name], expectValues[i]) - } - - is.Equal(len(key), 1) - is.Equal(key["id"], 1) -} - func Test_FetchWorker_updateSnapshotEnd(t *testing.T) { var ( is = is.New(t) From 28ed11b53cd2545471bef8b2a374d341290b214d Mon Sep 17 00:00:00 2001 From: Haris Osmanagic Date: Mon, 16 Jun 2025 10:55:51 +0200 Subject: [PATCH 21/60] fix fetch worker int test --- source/snapshot/fetch_worker_integration_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/snapshot/fetch_worker_integration_test.go b/source/snapshot/fetch_worker_integration_test.go index 0c585fb..d45838d 100644 --- a/source/snapshot/fetch_worker_integration_test.go +++ b/source/snapshot/fetch_worker_integration_test.go @@ -489,7 +489,7 @@ func Test_FetchWorker_updateSnapshotEnd(t *testing.T) { Table: table, Key: "UppercaseColumn1", }}, - expected: 3, + expected: 4, }, { desc: "skip update when set", From dcf70c4064e81dc6b3face74cb541275f2ed6e40 Mon Sep 17 00:00:00 2001 From: Haris Osmanagic Date: Mon, 16 Jun 2025 14:38:11 +0200 Subject: [PATCH 22/60] updates --- source/schema/avro_integration_test.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/source/schema/avro_integration_test.go b/source/schema/avro_integration_test.go index da8b122..f36e863 100644 --- a/source/schema/avro_integration_test.go +++ b/source/schema/avro_integration_test.go @@ -371,14 +371,14 @@ func avroTestSchema(t *testing.T, table string) avro.Schema { avro.NewPrimitiveLogicalSchema(avro.UUID), ))), - // json fields (represented as strings in Avro) + // json fields (represented as bytes in Avro) assert(avro.NewField("col_json", assert(avro.NewUnionSchema([]avro.Schema{ avro.NewPrimitiveSchema(avro.Null, nil), avro.NewPrimitiveSchema(avro.Bytes, nil), })))), assert(avro.NewField("col_json_not_null", avro.NewPrimitiveSchema(avro.Bytes, nil))), - // jsonb fields (represented as strings in Avro) + // jsonb fields (represented as bytes in Avro) assert(avro.NewField("col_jsonb", assert(avro.NewUnionSchema([]avro.Schema{ avro.NewPrimitiveSchema(avro.Null, nil), avro.NewPrimitiveSchema(avro.Bytes, nil), From 4297c35e2ce5073db4f29f95a52778549992e588 Mon Sep 17 00:00:00 2001 From: Haris Osmanagic Date: Tue, 17 Jun 2025 12:34:50 +0200 Subject: [PATCH 23/60] fix usage of pointers in tests --- source/schema/avro_integration_test.go | 78 +++++++++++++++++--------- source/types/types.go | 4 ++ 2 files changed, 56 insertions(+), 26 deletions(-) diff --git a/source/schema/avro_integration_test.go b/source/schema/avro_integration_test.go index f36e863..1503204 100644 --- a/source/schema/avro_integration_test.go +++ b/source/schema/avro_integration_test.go @@ -19,7 +19,9 @@ import ( "context" "fmt" "math/big" + "reflect" "slices" + "strings" "testing" "time" @@ -29,7 +31,6 @@ import ( "github.com/conduitio/conduit-connector-postgres/test" "github.com/hamba/avro/v2" "github.com/jackc/pgx/v5/pgconn" - "github.com/jackc/pgx/v5/pgtype" "github.com/matryer/is" ) @@ -59,15 +60,15 @@ func Test_AvroExtract(t *testing.T) { fields := rows.FieldDescriptions() - sch, err := Avro.Extract(table, tableInfoFetcher.GetTable(table), fields) + schemaExtracted, err := Avro.Extract(table, tableInfoFetcher.GetTable(table), fields) is.NoErr(err) t.Run("schema is parsable", func(t *testing.T) { is := is.New(t) is.NoErr(err) - is.Equal(sch.String(), avroTestSchema(t, table).String()) + is.Equal(schemaExtracted.String(), avroTestSchema(t, table).String()) - _, err = avro.Parse(sch.String()) + _, err = avro.Parse(schemaExtracted.String()) is.NoErr(err) }) @@ -76,7 +77,7 @@ func Test_AvroExtract(t *testing.T) { row := avrolizeMap(fields, values) - sch, err := avro.Parse(sch.String()) + sch, err := avro.Parse(schemaExtracted.String()) is.NoErr(err) data, err := avro.Marshal(sch, row) @@ -89,35 +90,53 @@ func Test_AvroExtract(t *testing.T) { is.Equal(len(decoded), len(row)) is.Equal(row["col_boolean"], decoded["col_boolean"]) is.Equal(row["col_bytea"], decoded["col_bytea"]) - is.Equal(row["col_varchar"], decoded["col_varchar"]) - is.Equal(row["col_date"], decoded["col_date"]) - is.Equal(row["col_float4"], decoded["col_float4"]) - is.Equal(row["col_float8"], decoded["col_float8"]) + is.Equal(dereference(row["col_varchar"]), decoded["col_varchar"]) + is.Equal(dereference(row["col_date"]), decoded["col_date"]) + is.Equal(dereference(row["col_float4"]), decoded["col_float4"]) + is.Equal(dereference(row["col_float8"]), decoded["col_float8"]) - colInt2 := int(row["col_int2"].(int16)) + colInt2 := int(*row["col_int2"].(*int16)) is.Equal(colInt2, decoded["col_int2"]) - colInt4 := int(row["col_int4"].(int32)) + colInt4 := int(*row["col_int4"].(*int32)) is.Equal(colInt4, decoded["col_int4"]) - is.Equal(row["col_int8"], decoded["col_int8"]) + is.Equal(dereference(row["col_int8"]), decoded["col_int8"]) numRow := row["col_numeric"].(*big.Rat) numDecoded := decoded["col_numeric"].(*big.Rat) is.Equal(numRow.RatString(), numDecoded.RatString()) - is.Equal(row["col_text"], decoded["col_text"]) + is.Equal(dereference(row["col_text"]), decoded["col_text"]) + is.Equal(dereference(row["col_uuid"]), decoded["col_uuid"]) - rowTS, colTS := row["col_timestamp"].(time.Time), decoded["col_timestamp"].(time.Time) + rowTS, colTS := row["col_timestamp"].(*time.Time), decoded["col_timestamp"].(time.Time) is.Equal(rowTS.UTC().String(), colTS.UTC().String()) rowTSTZ, colTSTZ := row["col_timestamptz"].(time.Time), decoded["col_timestamptz"].(time.Time) is.Equal(rowTSTZ.UTC().String(), colTSTZ.UTC().String()) - - is.Equal(row["col_uuid"], decoded["col_uuid"]) }) } +func dereference(v any) any { + if v == nil { + return nil + } + + dereferenced := v + + rowValueReflect := reflect.ValueOf(v) + if rowValueReflect.Kind() == reflect.Ptr { + if rowValueReflect.IsNil() { + dereferenced = nil + } else { + dereferenced = rowValueReflect.Elem().Interface() + } + } + + return dereferenced +} + func setupAvroTestTable(ctx context.Context, t *testing.T, conn test.Querier) string { is := is.New(t) table := test.RandomIdentifier(t) @@ -419,16 +438,23 @@ func avrolizeMap(fields []pgconn.FieldDescription, values []any) map[string]any row := make(map[string]any) for i, f := range fields { - switch f.DataTypeOID { - case pgtype.NumericOID: - n := new(big.Rat) - n.SetString(fmt.Sprint(types.Format(0, values[i], true))) - row[f.Name] = n - case pgtype.UUIDOID: - row[f.Name] = fmt.Sprint(values[i]) - default: - row[f.Name] = values[i] - } + isNotNull := f.Name == "id" || + f.Name == "col_bigserial" || + f.Name == "col_serial" || + f.Name == "col_smallserial" || + strings.HasSuffix(f.Name, "_not_null") + + row[f.Name] = assert(types.Format(f.DataTypeOID, values[i], isNotNull)) + // switch f.DataTypeOID { + // case pgtype.NumericOID: + // n := new(big.Rat) + // n.SetString(fmt.Sprint(types.Format(0, values[i], true))) + // row[f.Name] = n + // case pgtype.UUIDOID: + // row[f.Name] = fmt.Sprint(values[i]) + // default: + // row[f.Name] = values[i] + // } } return row diff --git a/source/types/types.go b/source/types/types.go index fa1687f..ba34904 100644 --- a/source/types/types.go +++ b/source/types/types.go @@ -76,6 +76,10 @@ func GetPointer(v any) any { return rv.Interface() } + if rv.Kind() == reflect.Slice || rv.Kind() == reflect.Map || rv.Kind() == reflect.Array { + return rv.Interface() + } + // For non-pointer values, we need to get the address // If the value is addressable, return its address if rv.CanAddr() { From 0027279b19d357ab963f369089b61cf9309d7144 Mon Sep 17 00:00:00 2001 From: Haris Osmanagic Date: Tue, 17 Jun 2025 15:17:37 +0200 Subject: [PATCH 24/60] fix usage of pointers in tests --- source/schema/avro_integration_test.go | 167 ++++++++++++++++++++----- 1 file changed, 134 insertions(+), 33 deletions(-) diff --git a/source/schema/avro_integration_test.go b/source/schema/avro_integration_test.go index 1503204..582bb58 100644 --- a/source/schema/avro_integration_test.go +++ b/source/schema/avro_integration_test.go @@ -88,53 +88,154 @@ func Test_AvroExtract(t *testing.T) { is.NoErr(avro.Unmarshal(sch, data, &decoded)) is.Equal(len(decoded), len(row)) - is.Equal(row["col_boolean"], decoded["col_boolean"]) - is.Equal(row["col_bytea"], decoded["col_bytea"]) - is.Equal(dereference(row["col_varchar"]), decoded["col_varchar"]) - is.Equal(dereference(row["col_date"]), decoded["col_date"]) - is.Equal(dereference(row["col_float4"]), decoded["col_float4"]) - is.Equal(dereference(row["col_float8"]), decoded["col_float8"]) - colInt2 := int(*row["col_int2"].(*int16)) - is.Equal(colInt2, decoded["col_int2"]) + // Compare all fields + compareValue(is, row, decoded, "col_bytea") + compareValue(is, row, decoded, "col_bytea_not_null") + compareValue(is, row, decoded, "col_varchar") + compareValue(is, row, decoded, "col_varchar_not_null") + compareValue(is, row, decoded, "col_date") + compareValue(is, row, decoded, "col_date_not_null") + compareValue(is, row, decoded, "col_float4") + compareValue(is, row, decoded, "col_float4_not_null") + compareValue(is, row, decoded, "col_float8") + compareValue(is, row, decoded, "col_float8_not_null") + + compareIntValue(is, row, decoded, "col_int2") + compareIntValue(is, row, decoded, "col_int2_not_null") + compareIntValue(is, row, decoded, "col_int4") + compareIntValue(is, row, decoded, "col_int4_not_null") + + compareValue(is, row, decoded, "col_int8") + compareValue(is, row, decoded, "col_int8_not_null") + + compareNumericValue(is, row, decoded, "col_numeric") + compareNumericValue(is, row, decoded, "col_numeric_not_null") + + compareValue(is, row, decoded, "col_text") + compareValue(is, row, decoded, "col_text_not_null") + + compareTimestampValue(is, row, decoded, "col_timestamp") + compareTimestampValue(is, row, decoded, "col_timestamp_not_null") + compareTimestampValue(is, row, decoded, "col_timestamptz") + compareTimestampValue(is, row, decoded, "col_timestamptz_not_null") + + compareValue(is, row, decoded, "col_uuid") + compareValue(is, row, decoded, "col_uuid_not_null") + compareValue(is, row, decoded, "col_json") + compareValue(is, row, decoded, "col_json_not_null") + compareValue(is, row, decoded, "col_jsonb") + compareValue(is, row, decoded, "col_jsonb_not_null") + compareValue(is, row, decoded, "col_bool") + compareValue(is, row, decoded, "col_bool_not_null") + + // Serial types are integers, so use compareIntValue + compareIntValue(is, row, decoded, "col_serial") + compareIntValue(is, row, decoded, "col_serial_not_null") + compareIntValue(is, row, decoded, "col_smallserial") + compareIntValue(is, row, decoded, "col_smallserial_not_null") + compareValue(is, row, decoded, "col_bigserial") + compareValue(is, row, decoded, "col_bigserial_not_null") + }) +} + +// Extracted comparison functions +func compareValue(is *is.I, wantMap, gotMap map[string]any, key string) { + is.Helper() + + want := wantMap[key] + got := gotMap[key] + + if want == nil { + is.Equal(nil, got) + return + } - colInt4 := int(*row["col_int4"].(*int32)) - is.Equal(colInt4, decoded["col_int4"]) + // If row value is a pointer, dereference it + wantReflect := reflect.ValueOf(want) + if wantReflect.Kind() == reflect.Ptr { + if wantReflect.IsNil() { + is.Equal(nil, got) + return + } + want = wantReflect.Elem().Interface() + } - is.Equal(dereference(row["col_int8"]), decoded["col_int8"]) + is.Equal(want, got) +} - numRow := row["col_numeric"].(*big.Rat) - numDecoded := decoded["col_numeric"].(*big.Rat) - is.Equal(numRow.RatString(), numDecoded.RatString()) +func compareIntValue(is *is.I, wantMap, gotMap map[string]any, key string) { + is.Helper() - is.Equal(dereference(row["col_text"]), decoded["col_text"]) - is.Equal(dereference(row["col_uuid"]), decoded["col_uuid"]) + want := wantMap[key] + got := gotMap[key] - rowTS, colTS := row["col_timestamp"].(*time.Time), decoded["col_timestamp"].(time.Time) - is.Equal(rowTS.UTC().String(), colTS.UTC().String()) + if want == nil { + is.Equal(nil, got) + return + } - rowTSTZ, colTSTZ := row["col_timestamptz"].(time.Time), decoded["col_timestamptz"].(time.Time) - is.Equal(rowTSTZ.UTC().String(), colTSTZ.UTC().String()) - }) + switch v := want.(type) { + case *int16: + case *int32: + is.Equal(int(*v), got) + case int16: + case int32: + is.Equal(int(v), got) + default: + is.Equal(want, got) + } } -func dereference(v any) any { - if v == nil { - return nil +func compareNumericValue(is *is.I, wantMap, gotMap map[string]any, key string) { + is.Helper() + + want := wantMap[key] + got := gotMap[key] + + if want == nil { + is.Equal(nil, got) + return + } + + numRow, ok := want.(*big.Rat) + if !ok || numRow == nil { + is.Equal(nil, got) + return } - dereferenced := v + numDecoded := got.(*big.Rat) + is.Equal(numRow.RatString(), numDecoded.RatString()) +} - rowValueReflect := reflect.ValueOf(v) - if rowValueReflect.Kind() == reflect.Ptr { - if rowValueReflect.IsNil() { - dereferenced = nil - } else { - dereferenced = rowValueReflect.Elem().Interface() - } +func compareTimestampValue(is *is.I, wantMap, gotMap map[string]any, key string) { + is.Helper() + + want := wantMap[key] + got := gotMap[key] + + if want == nil { + is.Equal(nil, got) + return + } + + var wantTS time.Time + switch v := want.(type) { + case *time.Time: + wantTS = *v + case time.Time: + wantTS = v + } + + var gotTS time.Time + switch v := got.(type) { + case map[string]interface{}: + gotTS = got.(map[string]interface{})["long.local-timestamp-micros"].(time.Time) + case time.Time: + gotTS = v } - return dereferenced + is.Equal(wantTS.UTC().String(), gotTS.UTC().String()) } func setupAvroTestTable(ctx context.Context, t *testing.T, conn test.Querier) string { From 6b79643c69573818653aba9e19997eef891c1394 Mon Sep 17 00:00:00 2001 From: Haris Osmanagic Date: Tue, 17 Jun 2025 16:42:35 +0200 Subject: [PATCH 25/60] fixes --- source/logrepl/cdc_test.go | 132 +----------------- source/logrepl/handler_test.go | 6 +- .../snapshot/fetch_worker_integration_test.go | 10 +- 3 files changed, 10 insertions(+), 138 deletions(-) diff --git a/source/logrepl/cdc_test.go b/source/logrepl/cdc_test.go index 0d9894f..61f82be 100644 --- a/source/logrepl/cdc_test.go +++ b/source/logrepl/cdc_test.go @@ -446,6 +446,7 @@ func TestCDCIterator_Ack(t *testing.T) { }) } } + func TestCDCIterator_NextN(t *testing.T) { ctx := test.Context(t) pool := test.ConnectPool(ctx, t, test.RepmgrConnString) @@ -581,7 +582,7 @@ func TestCDCIterator_NextN(t *testing.T) { func TestCDCIterator_NextN_InternalBatching(t *testing.T) { ctx := test.Context(t) pool := test.ConnectPool(ctx, t, test.RepmgrConnString) - table := test.SetupEmptyTestTable(ctx, t, pool) + table := test.SetupEmptyTable(ctx, t, pool) is := is.New(t) underTest := testCDCIterator(ctx, t, pool, table, true) @@ -643,7 +644,6 @@ func verifyOpenCDCRecords(is *is.I, got []opencdc.Record, tableName string, from "column2": int32(i) * 100, //nolint:gosec // fine, we know the value is small enough "column3": false, "column4": big.NewRat(123, 10), - "column5": big.NewRat(14, 1), "column6": nil, "column7": nil, "UppercaseColumn1": nil, @@ -667,134 +667,6 @@ func verifyOpenCDCRecords(is *is.I, got []opencdc.Record, tableName string, from is.Equal("", cmp.Diff(want, got, cmpOpts...)) // mismatch (-want +got) } -func TestCDCIterator_NextN(t *testing.T) { - ctx := test.Context(t) - pool := test.ConnectPool(ctx, t, test.RepmgrConnString) - table := test.SetupTestTable(ctx, t, pool) - - t.Run("retrieve exact N records", func(t *testing.T) { - is := is.New(t) - i := testCDCIterator(ctx, t, pool, table, true) - <-i.sub.Ready() - - for j := 1; j <= 3; j++ { - _, err := pool.Exec(ctx, fmt.Sprintf(`INSERT INTO %s (id, column1, column2, column3, column4, column5) - VALUES (%d, 'test-%d', %d, false, 12.3, 14)`, table, j+10, j, j*100)) - is.NoErr(err) - } - - var allRecords []opencdc.Record - attemptCtx, cancel := context.WithTimeout(ctx, 5*time.Second) - defer cancel() - - // Collect records until we have all 3 - for len(allRecords) < 3 { - records, err := i.NextN(attemptCtx, 3-len(allRecords)) - is.NoErr(err) - // Only proceed if we got at least one record - is.True(len(records) > 0) - allRecords = append(allRecords, records...) - } - - is.Equal(len(allRecords), 3) - - for j, r := range allRecords { - is.Equal(r.Operation, opencdc.OperationCreate) - is.Equal(r.Key.(opencdc.StructuredData)["id"], int64(j+11)) - change := r.Payload - data := change.After.(opencdc.StructuredData) - is.Equal(data["column1"], fmt.Sprintf("test-%d", j+1)) - //nolint:gosec // no risk to overflow - is.Equal(data["column2"], (int32(j)+1)*100) - } - }) - - t.Run("retrieve fewer records than requested", func(t *testing.T) { - is := is.New(t) - i := testCDCIterator(ctx, t, pool, table, true) - <-i.sub.Ready() - - for j := 1; j <= 2; j++ { - _, err := pool.Exec(ctx, fmt.Sprintf(`INSERT INTO %s (id, column1, column2, column3, column4, column5) - VALUES (%d, 'test-%d', %d, false, 12.3, 14)`, table, j+20, j, j*100)) - is.NoErr(err) - } - - // Will keep calling NextN until all records are received - records := make([]opencdc.Record, 0, 2) - for len(records) < 2 { - recordsTmp, err := i.NextN(ctx, 5) - is.NoErr(err) - records = append(records, recordsTmp...) - } - - // nothing else to fetch - ctxWithTimeout, cancel := context.WithTimeout(ctx, 500*time.Millisecond) - defer cancel() - _, err := i.NextN(ctxWithTimeout, 5) - is.True(errors.Is(err, context.DeadlineExceeded)) - - for j, r := range records { - is.Equal(r.Operation, opencdc.OperationCreate) - is.Equal(r.Key.(opencdc.StructuredData)["id"], int64(j+21)) - change := r.Payload - data := change.After.(opencdc.StructuredData) - is.Equal(data["column1"], fmt.Sprintf("test-%d", j+1)) - //nolint:gosec // no risk to overflow - is.Equal(data["column2"], (int32(j)+1)*100) - } - }) - - t.Run("context cancellation", func(t *testing.T) { - is := is.New(t) - i := testCDCIterator(ctx, t, pool, table, true) - <-i.sub.Ready() - - ctxTimeout, cancel := context.WithTimeout(ctx, 100*time.Millisecond) - defer cancel() - - _, err := i.NextN(ctxTimeout, 5) - is.True(errors.Is(err, context.DeadlineExceeded)) - }) - - t.Run("subscriber not started", func(t *testing.T) { - is := is.New(t) - i := testCDCIterator(ctx, t, pool, table, false) - - _, err := i.NextN(ctx, 5) - is.Equal(err.Error(), "logical replication has not been started") - }) - - t.Run("invalid N values", func(t *testing.T) { - is := is.New(t) - i := testCDCIterator(ctx, t, pool, table, true) - <-i.sub.Ready() - - _, err := i.NextN(ctx, 0) - is.True(strings.Contains(err.Error(), "n must be greater than 0")) - - _, err = i.NextN(ctx, -1) - is.True(strings.Contains(err.Error(), "n must be greater than 0")) - }) - - t.Run("subscription termination", func(t *testing.T) { - is := is.New(t) - i := testCDCIterator(ctx, t, pool, table, true) - <-i.sub.Ready() - - _, err := pool.Exec(ctx, fmt.Sprintf(`INSERT INTO %s (id, column1, column2, column3, column4, column5) - VALUES (30, 'test-1', 100, false, 12.3, 14)`, table)) - is.NoErr(err) - - time.Sleep(100 * time.Millisecond) - is.NoErr(i.Teardown(ctx)) - - _, err = i.NextN(ctx, 5) - is.True(err != nil) - is.True(strings.Contains(err.Error(), "logical replication error")) - }) -} - func testCDCIterator(ctx context.Context, t *testing.T, pool *pgxpool.Pool, table string, start bool) *CDCIterator { is := is.New(t) config := CDCConfig{ diff --git a/source/logrepl/handler_test.go b/source/logrepl/handler_test.go index 5956377..19736b0 100644 --- a/source/logrepl/handler_test.go +++ b/source/logrepl/handler_test.go @@ -29,7 +29,7 @@ func TestHandler_Batching_BatchSizeReached(t *testing.T) { is := is.New(t) ch := make(chan []opencdc.Record, 1) - underTest := NewCDCHandler(ctx, nil, nil, ch, false, 5, time.Second) + underTest := NewCDCHandler(ctx, nil, nil, nil, ch, false, 5, time.Second) want := make([]opencdc.Record, 5) for i := 0; i < cap(want); i++ { rec := newTestRecord(i) @@ -51,7 +51,7 @@ func TestHandler_Batching_FlushInterval(t *testing.T) { ch := make(chan []opencdc.Record, 1) flushInterval := time.Second - underTest := NewCDCHandler(ctx, nil, nil, ch, false, 5, flushInterval) + underTest := NewCDCHandler(ctx, nil, nil, nil, ch, false, 5, flushInterval) want := make([]opencdc.Record, 3) for i := 0; i < cap(want); i++ { @@ -74,7 +74,7 @@ func TestHandler_Batching_ContextCancelled(t *testing.T) { is := is.New(t) ch := make(chan []opencdc.Record, 1) - underTest := NewCDCHandler(ctx, nil, nil, ch, false, 5, time.Second) + underTest := NewCDCHandler(ctx, nil, nil, nil, ch, false, 5, time.Second) cancel() <-ctx.Done() underTest.addToBatch(ctx, newTestRecord(0)) diff --git a/source/snapshot/fetch_worker_integration_test.go b/source/snapshot/fetch_worker_integration_test.go index 69ba665..ec85ddf 100644 --- a/source/snapshot/fetch_worker_integration_test.go +++ b/source/snapshot/fetch_worker_integration_test.go @@ -130,7 +130,7 @@ func Test_FetcherValidate(t *testing.T) { is := is.New(t) f := NewFetchWorker( pool, - make(chan<- FetchData), + make(chan<- []FetchData), FetchConfig{ Table: table, Key: "id", @@ -144,7 +144,7 @@ func Test_FetcherValidate(t *testing.T) { is := is.New(t) f := NewFetchWorker( pool, - make(chan<- FetchData), + make(chan<- []FetchData), FetchConfig{ Table: "missing_table", Key: "id", @@ -160,7 +160,7 @@ func Test_FetcherValidate(t *testing.T) { is := is.New(t) f := NewFetchWorker( pool, - make(chan<- FetchData), + make(chan<- []FetchData), FetchConfig{ Table: table, Key: "column3", @@ -176,7 +176,7 @@ func Test_FetcherValidate(t *testing.T) { is := is.New(t) f := NewFetchWorker( pool, - make(chan<- FetchData), + make(chan<- []FetchData), FetchConfig{ Table: table, Key: "column2", @@ -191,7 +191,7 @@ func Test_FetcherValidate(t *testing.T) { is := is.New(t) f := NewFetchWorker( pool, - make(chan<- FetchData), + make(chan<- []FetchData), FetchConfig{ Table: table, Key: "missing_key", From b40a8efc1d95a5d8122e0afba5ad6b45a2ef4eac Mon Sep 17 00:00:00 2001 From: Haris Osmanagic Date: Tue, 17 Jun 2025 17:09:32 +0200 Subject: [PATCH 26/60] fix cdc_test --- source/logrepl/cdc_test.go | 23 ++++++++++++----------- 1 file changed, 12 insertions(+), 11 deletions(-) diff --git a/source/logrepl/cdc_test.go b/source/logrepl/cdc_test.go index 61f82be..48ce764 100644 --- a/source/logrepl/cdc_test.go +++ b/source/logrepl/cdc_test.go @@ -617,8 +617,8 @@ func insertTestRows(ctx context.Context, is *is.I, pool *pgxpool.Pool, table str _, err := pool.Exec( ctx, fmt.Sprintf( - `INSERT INTO %s (id, column1, column2, column3, column4, column5) - VALUES (%d, 'test-%d', %d, false, 12.3, 14)`, table, i+10, i, i*100, + `INSERT INTO %s (id, key, column1, column2, column3, column4, "UppercaseColumn1") + VALUES (%d, '%d', 'test-%d', %d, false, 12.3, %d)`, table, i+10, i+10, i, i*100, i+10, ), ) is.NoErr(err) @@ -626,6 +626,8 @@ func insertTestRows(ctx context.Context, is *is.I, pool *pgxpool.Pool, table str } func verifyOpenCDCRecords(is *is.I, got []opencdc.Record, tableName string, fromID, toID int) { + is.Helper() + // Build the expected records slice var want []opencdc.Record @@ -638,15 +640,14 @@ func verifyOpenCDCRecords(is *is.I, got []opencdc.Record, tableName string, from }, Payload: opencdc.Change{ After: opencdc.StructuredData{ - "id": id, - "key": nil, - "column1": fmt.Sprintf("test-%d", i), - "column2": int32(i) * 100, //nolint:gosec // fine, we know the value is small enough - "column3": false, - "column4": big.NewRat(123, 10), - "column6": nil, - "column7": nil, - "UppercaseColumn1": nil, + "id": id, + "key": []uint8(fmt.Sprintf("%d", id)), + "column1": fmt.Sprintf("test-%d", i), + "column2": int32(i) * 100, //nolint:gosec // fine, we know the value is small enough + "column3": false, + "column4": big.NewRat(123, 10), + // UppercaseColumn1 is a Postgres interger (4 bytes) + "UppercaseColumn1": int32(id), }, }, Metadata: opencdc.Metadata{ From 5aa9fd24965106e8627c06b0fe5a330264783119 Mon Sep 17 00:00:00 2001 From: Haris Osmanagic Date: Tue, 17 Jun 2025 17:59:07 +0200 Subject: [PATCH 27/60] fix source_integration_test --- source_integration_test.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/source_integration_test.go b/source_integration_test.go index f2f795d..8e136e5 100644 --- a/source_integration_test.go +++ b/source_integration_test.go @@ -110,7 +110,7 @@ func createTableWithManyTypes(ctx context.Context, t *testing.T) string { // table names with capital letters table := strings.ToUpper(test.RandomIdentifier(t)) - query := fmt.Sprintf(`CREATE TABLE %s ( + query := fmt.Sprintf(`CREATE TABLE %q ( id bigserial PRIMARY KEY, col_bytea bytea, col_bytea_not_null bytea NOT NULL, @@ -155,7 +155,7 @@ func createTableWithManyTypes(ctx context.Context, t *testing.T) string { is.NoErr(err) t.Cleanup(func() { - query := `DROP TABLE %s` + query := `DROP TABLE %q` query = fmt.Sprintf(query, table) _, err := conn.Exec(context.Background(), query) is.NoErr(err) @@ -172,7 +172,7 @@ func insertRowNotNullColumnsOnly(ctx context.Context, t *testing.T, table string conn := test.ConnectSimple(ctx, t, test.RepmgrConnString) query := fmt.Sprintf( - `INSERT INTO %s ( + `INSERT INTO %q ( col_bytea_not_null, col_varchar_not_null, col_date_not_null, @@ -239,7 +239,7 @@ func insertRowAllColumns(ctx context.Context, t *testing.T, table string, rowNum conn := test.ConnectSimple(ctx, t, test.RepmgrConnString) query := fmt.Sprintf( - `INSERT INTO %s ( + `INSERT INTO %q ( col_bytea, col_bytea_not_null, col_varchar, col_varchar_not_null, col_date, col_date_not_null, From aef7582b11897a50efb9bf2b96d35b427df6bfb0 Mon Sep 17 00:00:00 2001 From: Haris Osmanagic Date: Tue, 17 Jun 2025 18:25:42 +0200 Subject: [PATCH 28/60] fix source_integration_test --- source_integration_test.go | 27 +++++++++++++++++++-------- 1 file changed, 19 insertions(+), 8 deletions(-) diff --git a/source_integration_test.go b/source_integration_test.go index 8e136e5..0b0bba9 100644 --- a/source_integration_test.go +++ b/source_integration_test.go @@ -65,22 +65,33 @@ func TestSource_ReadN_Snapshot_CDC(t *testing.T) { is.NoErr(s.Teardown(ctx)) }) - gotRecords, err := s.ReadN(ctx, 1) + snapshotRecs, err := s.ReadN(ctx, 2) is.NoErr(err) - is.Equal(1, len(gotRecords)) - err = s.Ack(ctx, gotRecords[0].Position) + is.Equal(2, len(snapshotRecs)) + err = s.Ack(ctx, snapshotRecs[0].Position) is.NoErr(err) - assertRecordOK(is, tableName, gotRecords[0]) + err = s.Ack(ctx, snapshotRecs[1].Position) + is.NoErr(err) + + assertRecordOK(is, tableName, snapshotRecs[0]) + assertRecordOK(is, tableName, snapshotRecs[1]) insertRowNotNullColumnsOnly(ctx, t, tableName, 3) insertRowAllColumns(ctx, t, tableName, 4) - gotRecords, err = s.ReadN(ctx, 1) + cdcRecs, err := s.ReadN(ctx, 1) + is.NoErr(err) + is.Equal(1, len(cdcRecs)) + err = s.Ack(ctx, cdcRecs[0].Position) + is.NoErr(err) + assertRecordOK(is, tableName, cdcRecs[0]) + + cdcRecs, err = s.ReadN(ctx, 1) is.NoErr(err) - is.Equal(1, len(gotRecords)) - err = s.Ack(ctx, gotRecords[0].Position) + is.Equal(1, len(cdcRecs)) + err = s.Ack(ctx, cdcRecs[0].Position) is.NoErr(err) - assertRecordOK(is, tableName, gotRecords[0]) + assertRecordOK(is, tableName, cdcRecs[0]) } func assertRecordOK(is *is.I, tableName string, gotRecord opencdc.Record) { From 7bc7d767676807659dab1ae0d91a1cb0824c901f Mon Sep 17 00:00:00 2001 From: Haris Osmanagic Date: Tue, 17 Jun 2025 18:33:06 +0200 Subject: [PATCH 29/60] sync with main --- source/logrepl/cdc_test.go | 22 ---------------------- 1 file changed, 22 deletions(-) diff --git a/source/logrepl/cdc_test.go b/source/logrepl/cdc_test.go index 48ce764..883c66c 100644 --- a/source/logrepl/cdc_test.go +++ b/source/logrepl/cdc_test.go @@ -555,28 +555,6 @@ func TestCDCIterator_NextN(t *testing.T) { _, err = i.NextN(ctx, -1) is.True(strings.Contains(err.Error(), "n must be greater than 0")) }) - - t.Run("subscription termination", func(t *testing.T) { - is := is.New(t) - i := testCDCIterator(ctx, t, pool, table, true) - <-i.sub.Ready() - - _, err := pool.Exec(ctx, fmt.Sprintf(`INSERT INTO %s (id, key, column1, column2, column3, column4, "UppercaseColumn1") - VALUES (30, '30', 'test-1', 100, false, 12.3, 14)`, table)) - is.NoErr(err) - - go func() { - time.Sleep(100 * time.Millisecond) - is.NoErr(i.Teardown(ctx)) - }() - - records, err := i.NextN(ctx, 5) - if err != nil { - is.True(strings.Contains(err.Error(), "logical replication error")) - } else { - is.True(len(records) > 0) - } - }) } func TestCDCIterator_NextN_InternalBatching(t *testing.T) { From 7d701227c58219499cc251ae03c2c7b79e936add Mon Sep 17 00:00:00 2001 From: Haris Osmanagic Date: Tue, 17 Jun 2025 18:59:48 +0200 Subject: [PATCH 30/60] add comment --- source/logrepl/internal/relationset_test.go | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/source/logrepl/internal/relationset_test.go b/source/logrepl/internal/relationset_test.go index e548494..f01df2a 100644 --- a/source/logrepl/internal/relationset_test.go +++ b/source/logrepl/internal/relationset_test.go @@ -109,6 +109,10 @@ func TestRelationSetAllTypes(t *testing.T) { func setupTableAllTypes(ctx context.Context, t *testing.T, conn test.Querier) string { is := is.New(t) table := test.RandomIdentifier(t) + // todo still need to support: + // bit, varbit, box, char(n), cidr, circle, inet, interval, line, lseg, + // macaddr, macaddr8, money, path, pg_lsn, pg_snapshot, point, polygon, + // time, timetz, tsquery, tsvector, xml query := `CREATE TABLE %s ( id bigserial PRIMARY KEY, col_bit bit(8) NOT NULL, From b2c5210bd76d7550fc2f5f3ed04bfe29b038566b Mon Sep 17 00:00:00 2001 From: Haris Osmanagic Date: Tue, 17 Jun 2025 21:13:54 +0200 Subject: [PATCH 31/60] better source_integration_test --- source_integration_test.go | 143 +++++++++++++++++++++++-------------- 1 file changed, 90 insertions(+), 53 deletions(-) diff --git a/source_integration_test.go b/source_integration_test.go index 0b0bba9..38d20e5 100644 --- a/source_integration_test.go +++ b/source_integration_test.go @@ -19,6 +19,7 @@ import ( "fmt" "strings" "testing" + "time" "github.com/conduitio/conduit-commons/config" "github.com/conduitio/conduit-commons/opencdc" @@ -26,6 +27,7 @@ import ( "github.com/conduitio/conduit-connector-postgres/source/logrepl" "github.com/conduitio/conduit-connector-postgres/test" sdk "github.com/conduitio/conduit-connector-sdk" + "github.com/google/uuid" "github.com/matryer/is" ) @@ -73,8 +75,8 @@ func TestSource_ReadN_Snapshot_CDC(t *testing.T) { err = s.Ack(ctx, snapshotRecs[1].Position) is.NoErr(err) - assertRecordOK(is, tableName, snapshotRecs[0]) - assertRecordOK(is, tableName, snapshotRecs[1]) + assertRecordOK(is, tableName, snapshotRecs[0], 1) + assertRecordOK(is, tableName, snapshotRecs[1], 2) insertRowNotNullColumnsOnly(ctx, t, tableName, 3) insertRowAllColumns(ctx, t, tableName, 4) @@ -84,20 +86,29 @@ func TestSource_ReadN_Snapshot_CDC(t *testing.T) { is.Equal(1, len(cdcRecs)) err = s.Ack(ctx, cdcRecs[0].Position) is.NoErr(err) - assertRecordOK(is, tableName, cdcRecs[0]) + assertRecordOK(is, tableName, cdcRecs[0], 3) cdcRecs, err = s.ReadN(ctx, 1) is.NoErr(err) is.Equal(1, len(cdcRecs)) err = s.Ack(ctx, cdcRecs[0].Position) is.NoErr(err) - assertRecordOK(is, tableName, cdcRecs[0]) + assertRecordOK(is, tableName, cdcRecs[0], 4) } -func assertRecordOK(is *is.I, tableName string, gotRecord opencdc.Record) { +func assertRecordOK(is *is.I, tableName string, gotRecord opencdc.Record, rowNum int) { is.True(gotRecord.Key != nil) is.True(gotRecord.Payload.After != nil) + assertSchemaPresent(is, tableName, gotRecord) + assertPayloadOK(is, tableName, gotRecord, rowNum) +} + +func assertPayloadOK(is *is.I, tableName string, record opencdc.Record, rowNum int) { + +} + +func assertSchemaPresent(is *is.I, tableName string, gotRecord opencdc.Record) { payloadSchemaSubject, err := gotRecord.Metadata.GetPayloadSchemaSubject() is.NoErr(err) is.Equal(tableName+"_payload", payloadSchemaSubject) @@ -117,8 +128,8 @@ func createTableWithManyTypes(ctx context.Context, t *testing.T) string { is := is.New(t) conn := test.ConnectSimple(ctx, t, test.RepmgrConnString) - // Be sure primary key discovering works correctly on - // table names with capital letters + // Verify we can discover the primary key even when the table name + // contains capital letters. table := strings.ToUpper(test.RandomIdentifier(t)) query := fmt.Sprintf(`CREATE TABLE %q ( @@ -182,51 +193,60 @@ func insertRowNotNullColumnsOnly(ctx context.Context, t *testing.T, table string is := is.New(t) conn := test.ConnectSimple(ctx, t, test.RepmgrConnString) + tsLayout := "2006-01-02 15:04:05.000000" + + rowTS, err := time.Parse("2006-01-02 15:04:05", "2022-01-21 17:04:05") + is.NoErr(err) + rowTS = rowTS.Add(time.Duration(rowNumber) * time.Hour) + + rowUUID := uuid.NewString() + query := fmt.Sprintf( `INSERT INTO %q ( - col_bytea_not_null, - col_varchar_not_null, - col_date_not_null, - col_float4_not_null, - col_float8_not_null, - col_int2_not_null, - col_int4_not_null, - col_int8_not_null, - col_numeric_not_null, - col_text_not_null, - col_timestamp_not_null, - col_timestamptz_not_null, - col_uuid_not_null, - col_json_not_null, - col_jsonb_not_null, - col_bool_not_null, - col_serial_not_null, - col_smallserial_not_null, - col_bigserial_not_null - ) VALUES ( - '%s'::bytea, -- col_bytea_not_null - 'foo-%v', -- col_varchar_not_null - now(), -- col_date_not_null - %f, -- col_float4_not_null - %f, -- col_float8_not_null - %d, -- col_int2_not_null - %d, -- col_int4_not_null - %d, -- col_int8_not_null - %f, -- col_numeric_not_null - 'bar-%v', -- col_text_not_null - now(), -- col_timestamp_not_null - now(), -- col_timestamptz_not_null - gen_random_uuid(), -- col_uuid_not_null - '{"key": "value-%v"}'::json, -- col_json_not_null - '{"key": "value-%v"}'::jsonb, -- col_jsonb_not_null - %t, -- col_bool_not_null - %d, -- col_serial_not_null - %d, -- col_smallserial_not_null - %d -- col_bigserial_not_null - )`, + col_bytea_not_null, + col_varchar_not_null, + col_date_not_null, + col_float4_not_null, + col_float8_not_null, + col_int2_not_null, + col_int4_not_null, + col_int8_not_null, + col_numeric_not_null, + col_text_not_null, + col_timestamp_not_null, + col_timestamptz_not_null, + col_uuid_not_null, + col_json_not_null, + col_jsonb_not_null, + col_bool_not_null, + col_serial_not_null, + col_smallserial_not_null, + col_bigserial_not_null + ) VALUES ( + '%s'::bytea, + 'foo-%v', + '%s'::date, + %f, + %f, + %d, + %d, + %d, + %f, + 'bar-%v', + '%s'::timestamp, + '%s'::timestamptz, + '%s'::uuid, + '{"key": "value-%v"}'::json, + '{"key": "value-%v"}'::jsonb, + %t, + %d, + %d, + %d + )`, table, fmt.Sprintf("col_bytea_-%v", rowNumber), rowNumber, + rowTS.Format("2006-01-02"), float32(rowNumber)/10, float64(rowNumber)/10, rowNumber%32768, @@ -234,6 +254,9 @@ func insertRowNotNullColumnsOnly(ctx context.Context, t *testing.T, table string rowNumber, float64(100+rowNumber)/10, rowNumber, + rowTS.Format(tsLayout), + rowTS.Format(tsLayout), + rowUUID, rowNumber, rowNumber, rowNumber%2 == 0, @@ -241,7 +264,8 @@ func insertRowNotNullColumnsOnly(ctx context.Context, t *testing.T, table string rowNumber, rowNumber, ) - _, err := conn.Exec(ctx, query) + + _, err = conn.Exec(ctx, query) is.NoErr(err) } @@ -249,6 +273,14 @@ func insertRowAllColumns(ctx context.Context, t *testing.T, table string, rowNum is := is.New(t) conn := test.ConnectSimple(ctx, t, test.RepmgrConnString) + tsLayout := "2006-01-02 15:04:05.000000" + + rowTS, err := time.Parse("2006-01-02 15:04:05", "2022-01-21 17:04:05") + is.NoErr(err) + rowTS = rowTS.Add(time.Duration(rowNumber) * time.Hour) + + rowUUID := uuid.NewString() + query := fmt.Sprintf( `INSERT INTO %q ( col_bytea, col_bytea_not_null, @@ -273,7 +305,7 @@ func insertRowAllColumns(ctx context.Context, t *testing.T, table string, rowNum ) VALUES ( '%s'::bytea, '%s'::bytea, 'foo-%v', 'foo-%v', - now(), now(), + '%s'::date, '%s'::date, %f, %f, %f, %f, %d, %d, @@ -281,9 +313,9 @@ func insertRowAllColumns(ctx context.Context, t *testing.T, table string, rowNum %d, %d, %f, %f, 'bar-%v', 'bar-%v', - now(), now(), - now(), now(), - gen_random_uuid(), gen_random_uuid(), + '%s'::timestamp, '%s'::timestamp, + '%s'::timestamptz, '%s'::timestamptz, + '%s'::uuid, '%s'::uuid, '{"key": "value-%v"}'::json, '{"key": "value-%v"}'::json, '{"key": "value-%v"}'::jsonb, '{"key": "value-%v"}'::jsonb, %t, %t, @@ -294,6 +326,7 @@ func insertRowAllColumns(ctx context.Context, t *testing.T, table string, rowNum table, fmt.Sprintf("col_bytea_-%v", rowNumber), fmt.Sprintf("col_bytea_-%v", rowNumber), rowNumber, rowNumber, + rowTS.Format(tsLayout), rowTS.Format(tsLayout), float32(rowNumber)/10, float32(rowNumber)/10, float64(rowNumber)/10, float64(rowNumber)/10, rowNumber%32768, rowNumber%32768, @@ -301,6 +334,9 @@ func insertRowAllColumns(ctx context.Context, t *testing.T, table string, rowNum rowNumber, rowNumber, float64(100+rowNumber)/10, float64(100+rowNumber)/10, rowNumber, rowNumber, + rowTS.Format(tsLayout), rowTS.Format(tsLayout), + rowTS.Format(tsLayout), rowTS.Format(tsLayout), + rowUUID, rowUUID, rowNumber, rowNumber, rowNumber, rowNumber, rowNumber%2 == 0, rowNumber%2 == 0, @@ -308,7 +344,8 @@ func insertRowAllColumns(ctx context.Context, t *testing.T, table string, rowNum rowNumber, rowNumber, rowNumber, rowNumber, ) - _, err := conn.Exec(ctx, query) + + _, err = conn.Exec(ctx, query) is.NoErr(err) } From 40c747600e7bede4025cfefd9f6d06d84b66be16 Mon Sep 17 00:00:00 2001 From: Haris Osmanagic Date: Tue, 17 Jun 2025 21:49:23 +0200 Subject: [PATCH 32/60] more checks in source_integration_test --- source_integration_test.go | 240 ++++++++++++++++++++++++++----------- 1 file changed, 167 insertions(+), 73 deletions(-) diff --git a/source_integration_test.go b/source_integration_test.go index 38d20e5..7ba318d 100644 --- a/source_integration_test.go +++ b/source_integration_test.go @@ -17,6 +17,7 @@ package postgres import ( "context" "fmt" + "math/big" "strings" "testing" "time" @@ -27,7 +28,8 @@ import ( "github.com/conduitio/conduit-connector-postgres/source/logrepl" "github.com/conduitio/conduit-connector-postgres/test" sdk "github.com/conduitio/conduit-connector-sdk" - "github.com/google/uuid" + "github.com/conduitio/conduit-connector-sdk/schema" + "github.com/google/go-cmp/cmp" "github.com/matryer/is" ) @@ -36,6 +38,9 @@ func TestSource_ReadN_Snapshot_CDC(t *testing.T) { ctx := test.Context(t) tableName := createTableWithManyTypes(ctx, t) + // Snapshot data + insertRowNotNullColumnsOnly(ctx, t, tableName, 1) + insertRowAllColumns(ctx, t, tableName, 2) slotName := "conduitslot1" publicationName := "conduitpub1" @@ -67,45 +72,85 @@ func TestSource_ReadN_Snapshot_CDC(t *testing.T) { is.NoErr(s.Teardown(ctx)) }) + // Read and ack the 2 snapshots records snapshotRecs, err := s.ReadN(ctx, 2) is.NoErr(err) is.Equal(2, len(snapshotRecs)) + err = s.Ack(ctx, snapshotRecs[0].Position) is.NoErr(err) err = s.Ack(ctx, snapshotRecs[1].Position) is.NoErr(err) - assertRecordOK(is, tableName, snapshotRecs[0], 1) - assertRecordOK(is, tableName, snapshotRecs[1], 2) + // Verify snapshot records + assertRecordOK(is, tableName, snapshotRecs[0], 1, true) + assertRecordOK(is, tableName, snapshotRecs[1], 2, false) + // CDC data insertRowNotNullColumnsOnly(ctx, t, tableName, 3) insertRowAllColumns(ctx, t, tableName, 4) + // Read, ack and verify the first CDC record cdcRecs, err := s.ReadN(ctx, 1) is.NoErr(err) is.Equal(1, len(cdcRecs)) err = s.Ack(ctx, cdcRecs[0].Position) is.NoErr(err) - assertRecordOK(is, tableName, cdcRecs[0], 3) + assertRecordOK(is, tableName, cdcRecs[0], 3, true) + // Read, ack and verify the second CDC record cdcRecs, err = s.ReadN(ctx, 1) is.NoErr(err) is.Equal(1, len(cdcRecs)) err = s.Ack(ctx, cdcRecs[0].Position) is.NoErr(err) - assertRecordOK(is, tableName, cdcRecs[0], 4) + assertRecordOK(is, tableName, cdcRecs[0], 4, false) } -func assertRecordOK(is *is.I, tableName string, gotRecord opencdc.Record, rowNum int) { +func assertRecordOK(is *is.I, tableName string, gotRecord opencdc.Record, rowNum int, notNullOnly bool) { + is.Helper() + is.True(gotRecord.Key != nil) is.True(gotRecord.Payload.After != nil) assertSchemaPresent(is, tableName, gotRecord) - assertPayloadOK(is, tableName, gotRecord, rowNum) + assertPayloadOK(is, gotRecord, rowNum, notNullOnly) } -func assertPayloadOK(is *is.I, tableName string, record opencdc.Record, rowNum int) { +func assertPayloadOK(is *is.I, record opencdc.Record, rowNum int, notNullOnly bool) { + is.Helper() + + sch, err := schema.Get( + context.TODO(), + assert(record.Metadata.GetPayloadSchemaSubject()), + assert(record.Metadata.GetPayloadSchemaVersion()), + ) + is.NoErr(err) + + got := opencdc.StructuredData{} + err = sch.Unmarshal(record.Payload.After.Bytes(), &got) + is.NoErr(err) + + want := expectedRecord(rowNum) + + if notNullOnly { + for key := range want { + if !strings.HasSuffix(key, "_not_null") && !strings.HasSuffix(key, "serial") && key != "id" { + want[key] = nil + } + } + } + is.Equal( + "", + cmp.Diff( + want, + got, + cmp.Comparer(func(x, y *big.Rat) bool { + return x.Cmp(y) == 0 + }), + ), + ) // -want, +got } func assertSchemaPresent(is *is.I, tableName string, gotRecord opencdc.Record) { @@ -183,9 +228,6 @@ func createTableWithManyTypes(ctx context.Context, t *testing.T) string { is.NoErr(err) }) - insertRowNotNullColumnsOnly(ctx, t, table, 1) - insertRowAllColumns(ctx, t, table, 2) - return table } @@ -193,16 +235,11 @@ func insertRowNotNullColumnsOnly(ctx context.Context, t *testing.T, table string is := is.New(t) conn := test.ConnectSimple(ctx, t, test.RepmgrConnString) - tsLayout := "2006-01-02 15:04:05.000000" - - rowTS, err := time.Parse("2006-01-02 15:04:05", "2022-01-21 17:04:05") - is.NoErr(err) - rowTS = rowTS.Add(time.Duration(rowNumber) * time.Hour) - - rowUUID := uuid.NewString() + rec := expectedRecord(rowNumber) query := fmt.Sprintf( `INSERT INTO %q ( + id, col_bytea_not_null, col_varchar_not_null, col_date_not_null, @@ -223,8 +260,9 @@ func insertRowNotNullColumnsOnly(ctx context.Context, t *testing.T, table string col_smallserial_not_null, col_bigserial_not_null ) VALUES ( + %d, '%s'::bytea, - 'foo-%v', + '%s', '%s'::date, %f, %f, @@ -232,40 +270,41 @@ func insertRowNotNullColumnsOnly(ctx context.Context, t *testing.T, table string %d, %d, %f, - 'bar-%v', + '%s', '%s'::timestamp, '%s'::timestamptz, '%s'::uuid, - '{"key": "value-%v"}'::json, - '{"key": "value-%v"}'::jsonb, + '%s'::json, + '%s'::jsonb, %t, %d, %d, %d )`, table, - fmt.Sprintf("col_bytea_-%v", rowNumber), - rowNumber, - rowTS.Format("2006-01-02"), - float32(rowNumber)/10, - float64(rowNumber)/10, - rowNumber%32768, - rowNumber, - rowNumber, - float64(100+rowNumber)/10, - rowNumber, - rowTS.Format(tsLayout), - rowTS.Format(tsLayout), - rowUUID, - rowNumber, - rowNumber, - rowNumber%2 == 0, - rowNumber, - rowNumber, rowNumber, + rec["col_bytea_not_null"], + rec["col_varchar_not_null"], + rec["col_date_not_null"], + rec["col_float4_not_null"], + rec["col_float8_not_null"], + rec["col_int2_not_null"], + rec["col_int4_not_null"], + rec["col_int8_not_null"], + rec["col_numeric_not_null"], + rec["col_text_not_null"], + rec["col_timestamp_not_null"], + rec["col_timestamptz_not_null"], + rec["col_uuid_not_null"], + rec["col_json_not_null"], + rec["col_jsonb_not_null"], + rec["col_bool_not_null"], + rec["col_serial_not_null"], + rec["col_smallserial_not_null"], + rec["col_bigserial_not_null"], ) - _, err = conn.Exec(ctx, query) + _, err := conn.Exec(ctx, query) is.NoErr(err) } @@ -273,16 +312,11 @@ func insertRowAllColumns(ctx context.Context, t *testing.T, table string, rowNum is := is.New(t) conn := test.ConnectSimple(ctx, t, test.RepmgrConnString) - tsLayout := "2006-01-02 15:04:05.000000" - - rowTS, err := time.Parse("2006-01-02 15:04:05", "2022-01-21 17:04:05") - is.NoErr(err) - rowTS = rowTS.Add(time.Duration(rowNumber) * time.Hour) - - rowUUID := uuid.NewString() + rec := expectedRecord(rowNumber) query := fmt.Sprintf( `INSERT INTO %q ( + id, col_bytea, col_bytea_not_null, col_varchar, col_varchar_not_null, col_date, col_date_not_null, @@ -303,52 +337,105 @@ func insertRowAllColumns(ctx context.Context, t *testing.T, table string, rowNum col_smallserial, col_smallserial_not_null, col_bigserial, col_bigserial_not_null ) VALUES ( + %d, '%s'::bytea, '%s'::bytea, - 'foo-%v', 'foo-%v', - '%s'::date, '%s'::date, + '%s', '%s', + '%s'::date, '%s'::date, %f, %f, %f, %f, %d, %d, %d, %d, %d, %d, %f, %f, - 'bar-%v', 'bar-%v', + '%s', '%s', '%s'::timestamp, '%s'::timestamp, '%s'::timestamptz, '%s'::timestamptz, '%s'::uuid, '%s'::uuid, - '{"key": "value-%v"}'::json, '{"key": "value-%v"}'::json, - '{"key": "value-%v"}'::jsonb, '{"key": "value-%v"}'::jsonb, + '%s'::json, '%s'::json, + '%s'::jsonb, '%s'::jsonb, %t, %t, %d, %d, %d, %d, %d, %d )`, table, - fmt.Sprintf("col_bytea_-%v", rowNumber), fmt.Sprintf("col_bytea_-%v", rowNumber), - rowNumber, rowNumber, - rowTS.Format(tsLayout), rowTS.Format(tsLayout), - float32(rowNumber)/10, float32(rowNumber)/10, - float64(rowNumber)/10, float64(rowNumber)/10, - rowNumber%32768, rowNumber%32768, - rowNumber, rowNumber, - rowNumber, rowNumber, - float64(100+rowNumber)/10, float64(100+rowNumber)/10, - rowNumber, rowNumber, - rowTS.Format(tsLayout), rowTS.Format(tsLayout), - rowTS.Format(tsLayout), rowTS.Format(tsLayout), - rowUUID, rowUUID, - rowNumber, rowNumber, - rowNumber, rowNumber, - rowNumber%2 == 0, rowNumber%2 == 0, - rowNumber, rowNumber, - rowNumber, rowNumber, - rowNumber, rowNumber, + rowNumber, + rec["col_bytea"], rec["col_bytea_not_null"], + rec["col_varchar"], rec["col_varchar_not_null"], + rec["col_date"], rec["col_date_not_null"], + rec["col_float4"], rec["col_float4_not_null"], + rec["col_float8"], rec["col_float8_not_null"], + rec["col_int2"], rec["col_int2_not_null"], + rec["col_int4"], rec["col_int4_not_null"], + rec["col_int8"], rec["col_int8_not_null"], + rec["col_numeric"], rec["col_numeric_not_null"], + rec["col_text"], rec["col_text_not_null"], + rec["col_timestamp"], rec["col_timestamp_not_null"], + rec["col_timestamptz"], rec["col_timestamptz_not_null"], + rec["col_uuid"], rec["col_uuid_not_null"], + rec["col_json"], rec["col_json_not_null"], + rec["col_jsonb"], rec["col_jsonb_not_null"], + rec["col_bool"], rec["col_bool_not_null"], + rec["col_serial"], rec["col_serial_not_null"], + rec["col_smallserial"], rec["col_smallserial_not_null"], + rec["col_bigserial"], rec["col_bigserial_not_null"], ) - _, err = conn.Exec(ctx, query) + _, err := conn.Exec(ctx, query) is.NoErr(err) } +func expectedRecord(rowNumber int) opencdc.StructuredData { + tsLayout := "2006-01-02 15:04:05.000000" + + rowTS, _ := time.Parse("2006-01-02 15:04:05", "2022-01-21 17:04:05") + rowTS = rowTS.Add(time.Duration(rowNumber) * time.Hour) + + rowUUID := fmt.Sprintf("a74a9875-978e-4832-b1b8-6b0f8793a%03d", rowNumber) + + return opencdc.StructuredData{ + "id": rowNumber, + "col_bytea": fmt.Sprintf("col_bytea_-%v", rowNumber), + "col_bytea_not_null": fmt.Sprintf("col_bytea_-%v", rowNumber), + "col_varchar": fmt.Sprintf("foo-%v", rowNumber), + "col_varchar_not_null": fmt.Sprintf("foo-%v", rowNumber), + "col_date": rowTS.Format("2006-01-02"), + "col_date_not_null": rowTS.Format("2006-01-02"), + "col_float4": float32(rowNumber) / 10, + "col_float4_not_null": float32(rowNumber) / 10, + "col_float8": float64(rowNumber) / 10, + "col_float8_not_null": float64(rowNumber) / 10, + "col_int2": rowNumber % 32768, + "col_int2_not_null": rowNumber % 32768, + "col_int4": rowNumber, + "col_int4_not_null": rowNumber, + "col_int8": rowNumber, + "col_int8_not_null": rowNumber, + "col_numeric": float64(100+rowNumber) / 10, + "col_numeric_not_null": float64(100+rowNumber) / 10, + "col_text": fmt.Sprintf("bar-%v", rowNumber), + "col_text_not_null": fmt.Sprintf("bar-%v", rowNumber), + "col_timestamp": rowTS.Format(tsLayout), + "col_timestamp_not_null": rowTS.Format(tsLayout), + "col_timestamptz": rowTS.Format(tsLayout), + "col_timestamptz_not_null": rowTS.Format(tsLayout), + "col_uuid": rowUUID, + "col_uuid_not_null": rowUUID, + "col_json": fmt.Sprintf(`{"key": "value-%v"}`, rowNumber), + "col_json_not_null": fmt.Sprintf(`{"key": "value-%v"}`, rowNumber), + "col_jsonb": fmt.Sprintf(`{"key": "value-%v"}`, rowNumber), + "col_jsonb_not_null": fmt.Sprintf(`{"key": "value-%v"}`, rowNumber), + "col_bool": rowNumber%2 == 0, + "col_bool_not_null": rowNumber%2 == 0, + "col_serial": rowNumber, + "col_serial_not_null": rowNumber, + "col_smallserial": rowNumber, + "col_smallserial_not_null": rowNumber, + "col_bigserial": rowNumber, + "col_bigserial_not_null": rowNumber, + } +} + func TestSource_ParseConfig(t *testing.T) { testCases := []struct { name string @@ -391,3 +478,10 @@ func TestSource_ParseConfig(t *testing.T) { }) } } + +func assert[T any](val T, err error) T { + if err != nil { + panic(err) + } + return val +} From 06f41141b9c8cf56ccba8b96381f013552205c90 Mon Sep 17 00:00:00 2001 From: Haris Osmanagic Date: Tue, 17 Jun 2025 21:56:59 +0200 Subject: [PATCH 33/60] more checks in source_integration_test --- source_integration_test.go | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/source_integration_test.go b/source_integration_test.go index 7ba318d..04ec910 100644 --- a/source_integration_test.go +++ b/source_integration_test.go @@ -393,10 +393,11 @@ func expectedRecord(rowNumber int) opencdc.StructuredData { rowUUID := fmt.Sprintf("a74a9875-978e-4832-b1b8-6b0f8793a%03d", rowNumber) + id := int64(rowNumber) return opencdc.StructuredData{ - "id": rowNumber, - "col_bytea": fmt.Sprintf("col_bytea_-%v", rowNumber), - "col_bytea_not_null": fmt.Sprintf("col_bytea_-%v", rowNumber), + "id": id, + "col_bytea": fmt.Sprintf("col_bytea_%v", rowNumber), + "col_bytea_not_null": fmt.Sprintf("col_bytea_%v", rowNumber), "col_varchar": fmt.Sprintf("foo-%v", rowNumber), "col_varchar_not_null": fmt.Sprintf("foo-%v", rowNumber), "col_date": rowTS.Format("2006-01-02"), @@ -409,8 +410,8 @@ func expectedRecord(rowNumber int) opencdc.StructuredData { "col_int2_not_null": rowNumber % 32768, "col_int4": rowNumber, "col_int4_not_null": rowNumber, - "col_int8": rowNumber, - "col_int8_not_null": rowNumber, + "col_int8": id, + "col_int8_not_null": id, "col_numeric": float64(100+rowNumber) / 10, "col_numeric_not_null": float64(100+rowNumber) / 10, "col_text": fmt.Sprintf("bar-%v", rowNumber), @@ -431,8 +432,8 @@ func expectedRecord(rowNumber int) opencdc.StructuredData { "col_serial_not_null": rowNumber, "col_smallserial": rowNumber, "col_smallserial_not_null": rowNumber, - "col_bigserial": rowNumber, - "col_bigserial_not_null": rowNumber, + "col_bigserial": id, + "col_bigserial_not_null": id, } } From d372fe7b41e94d2e5afcf4024ce54b5acc8fe804 Mon Sep 17 00:00:00 2001 From: Haris Osmanagic Date: Wed, 18 Jun 2025 12:37:14 +0200 Subject: [PATCH 34/60] fix source_integration_test --- destination_integration_test.go | 4 +- source/logrepl/cdc_test.go | 8 +- source/logrepl/combined_test.go | 4 +- source/logrepl/internal/relationset_test.go | 12 +-- .../snapshot/fetch_worker_integration_test.go | 8 +- source/types/types.go | 4 + source_integration_test.go | 79 +++++++++++++------ test/helper.go | 6 ++ 8 files changed, 74 insertions(+), 51 deletions(-) diff --git a/destination_integration_test.go b/destination_integration_test.go index fe58045..bdc3bb4 100644 --- a/destination_integration_test.go +++ b/destination_integration_test.go @@ -184,9 +184,7 @@ func TestDestination_Write(t *testing.T) { cmp.Diff( tt.record.Payload.After, got, - cmp.Comparer(func(x, y *big.Rat) bool { - return x.Cmp(y) == 0 - }), + test.BigRatComparer, ), ) // -want, +got case opencdc.OperationDelete: diff --git a/source/logrepl/cdc_test.go b/source/logrepl/cdc_test.go index 883c66c..1addc46 100644 --- a/source/logrepl/cdc_test.go +++ b/source/logrepl/cdc_test.go @@ -342,9 +342,7 @@ func TestCDCIterator_Operation_NextN(t *testing.T) { tt.want, got, cmpopts.IgnoreUnexported(opencdc.Record{}), - cmp.Comparer(func(x, y *big.Rat) bool { - return x.Cmp(y) == 0 - }), + test.BigRatComparer, )) is.NoErr(i.Ack(ctx, got.Position)) }) @@ -639,9 +637,7 @@ func verifyOpenCDCRecords(is *is.I, got []opencdc.Record, tableName string, from cmpOpts := []cmp.Option{ cmpopts.IgnoreUnexported(opencdc.Record{}), cmpopts.IgnoreFields(opencdc.Record{}, "Position", "Metadata"), - cmp.Comparer(func(x, y *big.Rat) bool { - return x.Cmp(y) == 0 - }), + test.BigRatComparer, } is.Equal("", cmp.Diff(want, got, cmpOpts...)) // mismatch (-want +got) } diff --git a/source/logrepl/combined_test.go b/source/logrepl/combined_test.go index ad5b07b..82bc5bd 100644 --- a/source/logrepl/combined_test.go +++ b/source/logrepl/combined_test.go @@ -320,9 +320,7 @@ func TestCombinedIterator_NextN(t *testing.T) { is.Equal("", cmp.Diff( expectedRecords[6], records[0].Payload.After.(opencdc.StructuredData), - cmp.Comparer(func(x, y *big.Rat) bool { - return x.Cmp(y) == 0 - }), + test.BigRatComparer, )) is.NoErr(i.Ack(ctx, records[0].Position)) diff --git a/source/logrepl/internal/relationset_test.go b/source/logrepl/internal/relationset_test.go index f01df2a..c08ad56 100644 --- a/source/logrepl/internal/relationset_test.go +++ b/source/logrepl/internal/relationset_test.go @@ -109,10 +109,6 @@ func TestRelationSetAllTypes(t *testing.T) { func setupTableAllTypes(ctx context.Context, t *testing.T, conn test.Querier) string { is := is.New(t) table := test.RandomIdentifier(t) - // todo still need to support: - // bit, varbit, box, char(n), cidr, circle, inet, interval, line, lseg, - // macaddr, macaddr8, money, path, pg_lsn, pg_snapshot, point, polygon, - // time, timetz, tsquery, tsvector, xml query := `CREATE TABLE %s ( id bigserial PRIMARY KEY, col_bit bit(8) NOT NULL, @@ -360,9 +356,7 @@ func isValuesAllTypes(is *is.I, got map[string]any) { cmp.Comparer(func(x, y netip.Prefix) bool { return x.String() == y.String() }), - cmp.Comparer(func(x, y *big.Rat) bool { - return x.Cmp(y) == 0 - }), + test.BigRatComparer, )) } @@ -460,8 +454,6 @@ func isValuesAllTypesStandalone(is *is.I, got map[string]any) { cmp.Comparer(func(x, y netip.Prefix) bool { return x.String() == y.String() }), - cmp.Comparer(func(x, y *big.Rat) bool { - return x.Cmp(y) == 0 - }), + test.BigRatComparer, )) } diff --git a/source/snapshot/fetch_worker_integration_test.go b/source/snapshot/fetch_worker_integration_test.go index ec85ddf..4824739 100644 --- a/source/snapshot/fetch_worker_integration_test.go +++ b/source/snapshot/fetch_worker_integration_test.go @@ -289,9 +289,7 @@ func Test_FetcherRun_Initial(t *testing.T) { is.Equal("", cmp.Diff( expectedMatch[i], got.Payload, - cmp.Comparer(func(x, y *big.Rat) bool { - return x.Cmp(y) == 0 - }), + test.BigRatComparer, )) is.Equal(got.Position, position.SnapshotPosition{ @@ -360,9 +358,7 @@ func Test_FetcherRun_Resume(t *testing.T) { "column4": big.NewRat(836, 25), "UppercaseColumn1": int32(3), }, - cmp.Comparer(func(x, y *big.Rat) bool { - return x.Cmp(y) == 0 - }), + test.BigRatComparer, ), ) diff --git a/source/types/types.go b/source/types/types.go index 31ce561..51e1c7c 100644 --- a/source/types/types.go +++ b/source/types/types.go @@ -26,6 +26,10 @@ var ( UUID = UUIDFormatter{} ) +// todo still need to support: +// bit, varbit, box, char(n), cidr, circle, inet, interval, line, lseg, +// macaddr, macaddr8, money, path, pg_lsn, pg_snapshot, point, polygon, +// time, timetz, tsquery, tsvector, xml func Format(oid uint32, v any, isNotNull bool) (any, error) { val, err := format(oid, v) if err != nil { diff --git a/source_integration_test.go b/source_integration_test.go index 04ec910..d2f8bb8 100644 --- a/source_integration_test.go +++ b/source_integration_test.go @@ -131,26 +131,9 @@ func assertPayloadOK(is *is.I, record opencdc.Record, rowNum int, notNullOnly bo err = sch.Unmarshal(record.Payload.After.Bytes(), &got) is.NoErr(err) - want := expectedRecord(rowNum) + want := expectedRecord(rowNum, notNullOnly) - if notNullOnly { - for key := range want { - if !strings.HasSuffix(key, "_not_null") && !strings.HasSuffix(key, "serial") && key != "id" { - want[key] = nil - } - } - } - - is.Equal( - "", - cmp.Diff( - want, - got, - cmp.Comparer(func(x, y *big.Rat) bool { - return x.Cmp(y) == 0 - }), - ), - ) // -want, +got + is.Equal("", cmp.Diff(want, got, test.BigRatComparer)) // -want, +got } func assertSchemaPresent(is *is.I, tableName string, gotRecord opencdc.Record) { @@ -235,7 +218,7 @@ func insertRowNotNullColumnsOnly(ctx context.Context, t *testing.T, table string is := is.New(t) conn := test.ConnectSimple(ctx, t, test.RepmgrConnString) - rec := expectedRecord(rowNumber) + rec := generateRecord(rowNumber, true) query := fmt.Sprintf( `INSERT INTO %q ( @@ -312,7 +295,7 @@ func insertRowAllColumns(ctx context.Context, t *testing.T, table string, rowNum is := is.New(t) conn := test.ConnectSimple(ctx, t, test.RepmgrConnString) - rec := expectedRecord(rowNumber) + rec := generateRecord(rowNumber, false) query := fmt.Sprintf( `INSERT INTO %q ( @@ -385,7 +368,47 @@ func insertRowAllColumns(ctx context.Context, t *testing.T, table string, rowNum is.NoErr(err) } -func expectedRecord(rowNumber int) opencdc.StructuredData { +func expectedRecord(rowNumber int, notNullOnly bool) opencdc.StructuredData { + rec := generateRecord(rowNumber, notNullOnly) + + for key, value := range rec { + if value != nil { + rec[key] = normalizeNotNullValue(key, value) + } else { + rec[key] = normalizeNullValue(key, value) + } + } + + return rec +} + +func normalizeNullValue(key string, value interface{}) interface{} { + normalized := value + switch { + case strings.Contains(key, "_uuid"): + normalized = "" + } + + return normalized +} + +func normalizeNotNullValue(key string, value interface{}) interface{} { + normalized := value + switch { + case strings.Contains(key, "_bytea"), + strings.Contains(key, "_json"), + strings.Contains(key, "_jsonb"): + normalized = []uint8(value.(string)) + case strings.Contains(key, "_numeric"): + val := new(big.Rat) + val.SetString(fmt.Sprintf("%v", value)) + normalized = val + } + + return normalized +} + +func generateRecord(rowNumber int, notNullOnly bool) opencdc.StructuredData { tsLayout := "2006-01-02 15:04:05.000000" rowTS, _ := time.Parse("2006-01-02 15:04:05", "2022-01-21 17:04:05") @@ -394,7 +417,7 @@ func expectedRecord(rowNumber int) opencdc.StructuredData { rowUUID := fmt.Sprintf("a74a9875-978e-4832-b1b8-6b0f8793a%03d", rowNumber) id := int64(rowNumber) - return opencdc.StructuredData{ + rec := opencdc.StructuredData{ "id": id, "col_bytea": fmt.Sprintf("col_bytea_%v", rowNumber), "col_bytea_not_null": fmt.Sprintf("col_bytea_%v", rowNumber), @@ -435,6 +458,16 @@ func expectedRecord(rowNumber int) opencdc.StructuredData { "col_bigserial": id, "col_bigserial_not_null": id, } + + if notNullOnly { + for key := range rec { + if !strings.HasSuffix(key, "_not_null") && !strings.HasSuffix(key, "serial") && key != "id" { + rec[key] = nil + } + } + } + + return rec } func TestSource_ParseConfig(t *testing.T) { diff --git a/test/helper.go b/test/helper.go index 616fa78..37d240f 100644 --- a/test/helper.go +++ b/test/helper.go @@ -18,6 +18,7 @@ import ( "context" "errors" "fmt" + "math/big" "strconv" "strings" "testing" @@ -25,6 +26,7 @@ import ( "github.com/conduitio/conduit-commons/csync" "github.com/conduitio/conduit-connector-postgres/source/cpool" + "github.com/google/go-cmp/cmp" "github.com/jackc/pgx/v5" "github.com/jackc/pgx/v5/pgconn" "github.com/jackc/pgx/v5/pgxpool" @@ -126,6 +128,10 @@ const testTableCreateQuery = ` "UppercaseColumn1" integer not null )` +var BigRatComparer = cmp.Comparer(func(x, y *big.Rat) bool { + return x.Cmp(y) == 0 +}) + type Querier interface { Exec(ctx context.Context, sql string, arguments ...any) (pgconn.CommandTag, error) Query(ctx context.Context, sql string, args ...any) (pgx.Rows, error) From 664701706401e4f6d62465bc01593b30d971e4de Mon Sep 17 00:00:00 2001 From: Haris Osmanagic Date: Wed, 18 Jun 2025 12:50:42 +0200 Subject: [PATCH 35/60] format, fixes --- destination.go | 14 -------------- source/logrepl/internal/subscription_test.go | 18 ++++++++++++------ source/types/types_test.go | 6 +++--- 3 files changed, 15 insertions(+), 23 deletions(-) diff --git a/destination.go b/destination.go index bc6ff58..52f6098 100644 --- a/destination.go +++ b/destination.go @@ -275,20 +275,6 @@ func (d *Destination) formatColumnsAndValues( return colArgs, valArgs, nil } -// getKeyColumnName will return the name of the first item in the key or the -// connector-configured default name of the key column name. -func (d *Destination) getKeyColumnName(key opencdc.StructuredData, defaultKeyName string) string { - if len(key) > 1 { - // Go maps aren't order preserving, so anything over len 1 will have - // non deterministic results until we handle composite keys. - panic("composite keys not yet supported") - } - for k := range key { - return k - } - return defaultKeyName -} - func (d *Destination) hasKey(e opencdc.Record) bool { structuredKey, ok := e.Key.(opencdc.StructuredData) if !ok { diff --git a/source/logrepl/internal/subscription_test.go b/source/logrepl/internal/subscription_test.go index ad6ae99..3365afb 100644 --- a/source/logrepl/internal/subscription_test.go +++ b/source/logrepl/internal/subscription_test.go @@ -64,8 +64,10 @@ func TestSubscription_WithRepmgr(t *testing.T) { t.Run("first insert table1", func(t *testing.T) { is := is.New(t) - query := fmt.Sprintf(`INSERT INTO %s (id, key, column1, column2, column3, column4, "UppercaseColumn1") - VALUES (6, '6', 'bizz', 456, false, 12.3, 61)`, table1) + query := fmt.Sprintf( + `INSERT INTO %s (id, key, column1, column2, column3, column4, "UppercaseColumn1") + VALUES (6, '6', 'bizz', 456, false, 12.3, 61)`, + table1) _, err := pool.Exec(ctx, query) is.NoErr(err) @@ -82,8 +84,10 @@ func TestSubscription_WithRepmgr(t *testing.T) { t.Run("second insert table1", func(t *testing.T) { is := is.New(t) - query := fmt.Sprintf(`INSERT INTO %s (id, key, column1, column2, column3, column4, "UppercaseColumn1") - VALUES (7, '7', 'bizz', 456, false, 12.3, 61)`, table1) + query := fmt.Sprintf( + `INSERT INTO %s (id, key, column1, column2, column3, column4, "UppercaseColumn1") + VALUES (7, '7', 'bizz', 456, false, 12.3, 61)`, + table1) _, err := pool.Exec(ctx, query) is.NoErr(err) @@ -162,8 +166,10 @@ func TestSubscription_ClosedContext(t *testing.T) { sub, messages := setupSubscription(ctx, t, pool, table) // insert to get new messages into publication - query := fmt.Sprintf(`INSERT INTO %s (id, key, column1, column2, column3, column4, "UppercaseColumn1") - VALUES (6, '6', 'bizz', 456, false, 12.3, 61)`, table) + query := fmt.Sprintf( + `INSERT INTO %s (id, key, column1, column2, column3, column4, "UppercaseColumn1") + VALUES (6, '6', 'bizz', 456, false, 12.3, 61)`, + table) _, err := pool.Exec(ctx, query) is.NoErr(err) diff --git a/source/types/types_test.go b/source/types/types_test.go index 0253f6d..cb206ca 100644 --- a/source/types/types_test.go +++ b/source/types/types_test.go @@ -37,16 +37,16 @@ func Test_Format(t *testing.T) { { name: "int float string bool", input: []any{ - 1021, + 1021, 199.2, "foo", true, }, inputOID: []uint32{ 0, 0, 0, 0, }, expect: []any{ - 1021, + 1021, 199.2, "foo", true, }, expectNullable: []any{ - lang.Ptr(1021), + lang.Ptr(1021), lang.Ptr(199.2), lang.Ptr("foo"), lang.Ptr(true), }, }, { From 9c0cc4791226d94f569604c86354af8e3e5d5b73 Mon Sep 17 00:00:00 2001 From: Haris Osmanagic Date: Wed, 18 Jun 2025 12:54:08 +0200 Subject: [PATCH 36/60] move method --- source/logrepl/cdc_test.go | 66 +++++++++++++++++++------------------- 1 file changed, 33 insertions(+), 33 deletions(-) diff --git a/source/logrepl/cdc_test.go b/source/logrepl/cdc_test.go index 1addc46..e93557b 100644 --- a/source/logrepl/cdc_test.go +++ b/source/logrepl/cdc_test.go @@ -445,6 +445,39 @@ func TestCDCIterator_Ack(t *testing.T) { } } +func TestCDCIterator_NextN_InternalBatching(t *testing.T) { + ctx := test.Context(t) + pool := test.ConnectPool(ctx, t, test.RepmgrConnString) + table := test.SetupEmptyTable(ctx, t, pool) + + is := is.New(t) + underTest := testCDCIterator(ctx, t, pool, table, true) + <-underTest.sub.Ready() + + insertTestRows(ctx, is, pool, table, 1, 1) + // wait until the CDCHandler flushes this one record + // so that we force the CDCIterator to wait for another batch + time.Sleep(time.Second * 2) + insertTestRows(ctx, is, pool, table, 2, 5) + + // we request 2 records, expect records 1 and 2 + got, err := underTest.NextN(ctx, 2) + is.NoErr(err) + verifyOpenCDCRecords(is, got, table, 1, 2) + time.Sleep(200 * time.Millisecond) + + // we request 2 records, expect records 3 and 4 + got, err = underTest.NextN(ctx, 2) + is.NoErr(err) + verifyOpenCDCRecords(is, got, table, 3, 4) + time.Sleep(200 * time.Millisecond) + + // we request 2 records, expect record 5 + got, err = underTest.NextN(ctx, 2) + is.NoErr(err) + verifyOpenCDCRecords(is, got, table, 5, 5) +} + func TestCDCIterator_NextN(t *testing.T) { ctx := test.Context(t) pool := test.ConnectPool(ctx, t, test.RepmgrConnString) @@ -555,39 +588,6 @@ func TestCDCIterator_NextN(t *testing.T) { }) } -func TestCDCIterator_NextN_InternalBatching(t *testing.T) { - ctx := test.Context(t) - pool := test.ConnectPool(ctx, t, test.RepmgrConnString) - table := test.SetupEmptyTable(ctx, t, pool) - - is := is.New(t) - underTest := testCDCIterator(ctx, t, pool, table, true) - <-underTest.sub.Ready() - - insertTestRows(ctx, is, pool, table, 1, 1) - // wait until the CDCHandler flushes this one record - // so that we force the CDCIterator to wait for another batch - time.Sleep(time.Second * 2) - insertTestRows(ctx, is, pool, table, 2, 5) - - // we request 2 records, expect records 1 and 2 - got, err := underTest.NextN(ctx, 2) - is.NoErr(err) - verifyOpenCDCRecords(is, got, table, 1, 2) - time.Sleep(200 * time.Millisecond) - - // we request 2 records, expect records 3 and 4 - got, err = underTest.NextN(ctx, 2) - is.NoErr(err) - verifyOpenCDCRecords(is, got, table, 3, 4) - time.Sleep(200 * time.Millisecond) - - // we request 2 records, expect record 5 - got, err = underTest.NextN(ctx, 2) - is.NoErr(err) - verifyOpenCDCRecords(is, got, table, 5, 5) -} - func insertTestRows(ctx context.Context, is *is.I, pool *pgxpool.Pool, table string, from int, to int) { for i := from; i <= to; i++ { _, err := pool.Exec( From 1ce91bfb6f0e8676cc5ba77d185b0fe3dbf48a3e Mon Sep 17 00:00:00 2001 From: Haris Osmanagic Date: Wed, 18 Jun 2025 12:55:02 +0200 Subject: [PATCH 37/60] move method --- source/logrepl/cdc_test.go | 26 +++++++++++++------------- 1 file changed, 13 insertions(+), 13 deletions(-) diff --git a/source/logrepl/cdc_test.go b/source/logrepl/cdc_test.go index e93557b..c8c6ebe 100644 --- a/source/logrepl/cdc_test.go +++ b/source/logrepl/cdc_test.go @@ -478,6 +478,19 @@ func TestCDCIterator_NextN_InternalBatching(t *testing.T) { verifyOpenCDCRecords(is, got, table, 5, 5) } +func insertTestRows(ctx context.Context, is *is.I, pool *pgxpool.Pool, table string, from int, to int) { + for i := from; i <= to; i++ { + _, err := pool.Exec( + ctx, + fmt.Sprintf( + `INSERT INTO %s (id, key, column1, column2, column3, column4, "UppercaseColumn1") + VALUES (%d, '%d', 'test-%d', %d, false, 12.3, %d)`, table, i+10, i+10, i, i*100, i+10, + ), + ) + is.NoErr(err) + } +} + func TestCDCIterator_NextN(t *testing.T) { ctx := test.Context(t) pool := test.ConnectPool(ctx, t, test.RepmgrConnString) @@ -588,19 +601,6 @@ func TestCDCIterator_NextN(t *testing.T) { }) } -func insertTestRows(ctx context.Context, is *is.I, pool *pgxpool.Pool, table string, from int, to int) { - for i := from; i <= to; i++ { - _, err := pool.Exec( - ctx, - fmt.Sprintf( - `INSERT INTO %s (id, key, column1, column2, column3, column4, "UppercaseColumn1") - VALUES (%d, '%d', 'test-%d', %d, false, 12.3, %d)`, table, i+10, i+10, i, i*100, i+10, - ), - ) - is.NoErr(err) - } -} - func verifyOpenCDCRecords(is *is.I, got []opencdc.Record, tableName string, fromID, toID int) { is.Helper() From 5ea78abb3862da0d7b351fac22a1c921d9baaca6 Mon Sep 17 00:00:00 2001 From: Haris Osmanagic Date: Wed, 18 Jun 2025 12:55:56 +0200 Subject: [PATCH 38/60] move method --- source/logrepl/cdc_test.go | 82 +++++++++++++++++++------------------- 1 file changed, 41 insertions(+), 41 deletions(-) diff --git a/source/logrepl/cdc_test.go b/source/logrepl/cdc_test.go index c8c6ebe..4788b7e 100644 --- a/source/logrepl/cdc_test.go +++ b/source/logrepl/cdc_test.go @@ -491,6 +491,47 @@ func insertTestRows(ctx context.Context, is *is.I, pool *pgxpool.Pool, table str } } +func verifyOpenCDCRecords(is *is.I, got []opencdc.Record, tableName string, fromID, toID int) { + is.Helper() + + // Build the expected records slice + var want []opencdc.Record + + for i := fromID; i <= toID; i++ { + id := int64(i + 10) + record := opencdc.Record{ + Operation: opencdc.OperationCreate, + Key: opencdc.StructuredData{ + "id": id, + }, + Payload: opencdc.Change{ + After: opencdc.StructuredData{ + "id": id, + "key": []uint8(fmt.Sprintf("%d", id)), + "column1": fmt.Sprintf("test-%d", i), + "column2": int32(i) * 100, //nolint:gosec // fine, we know the value is small enough + "column3": false, + "column4": big.NewRat(123, 10), + // UppercaseColumn1 is a Postgres interger (4 bytes) + "UppercaseColumn1": int32(id), + }, + }, + Metadata: opencdc.Metadata{ + opencdc.MetadataCollection: tableName, + }, + } + + want = append(want, record) + } + + cmpOpts := []cmp.Option{ + cmpopts.IgnoreUnexported(opencdc.Record{}), + cmpopts.IgnoreFields(opencdc.Record{}, "Position", "Metadata"), + test.BigRatComparer, + } + is.Equal("", cmp.Diff(want, got, cmpOpts...)) // mismatch (-want +got) +} + func TestCDCIterator_NextN(t *testing.T) { ctx := test.Context(t) pool := test.ConnectPool(ctx, t, test.RepmgrConnString) @@ -601,47 +642,6 @@ func TestCDCIterator_NextN(t *testing.T) { }) } -func verifyOpenCDCRecords(is *is.I, got []opencdc.Record, tableName string, fromID, toID int) { - is.Helper() - - // Build the expected records slice - var want []opencdc.Record - - for i := fromID; i <= toID; i++ { - id := int64(i + 10) - record := opencdc.Record{ - Operation: opencdc.OperationCreate, - Key: opencdc.StructuredData{ - "id": id, - }, - Payload: opencdc.Change{ - After: opencdc.StructuredData{ - "id": id, - "key": []uint8(fmt.Sprintf("%d", id)), - "column1": fmt.Sprintf("test-%d", i), - "column2": int32(i) * 100, //nolint:gosec // fine, we know the value is small enough - "column3": false, - "column4": big.NewRat(123, 10), - // UppercaseColumn1 is a Postgres interger (4 bytes) - "UppercaseColumn1": int32(id), - }, - }, - Metadata: opencdc.Metadata{ - opencdc.MetadataCollection: tableName, - }, - } - - want = append(want, record) - } - - cmpOpts := []cmp.Option{ - cmpopts.IgnoreUnexported(opencdc.Record{}), - cmpopts.IgnoreFields(opencdc.Record{}, "Position", "Metadata"), - test.BigRatComparer, - } - is.Equal("", cmp.Diff(want, got, cmpOpts...)) // mismatch (-want +got) -} - func testCDCIterator(ctx context.Context, t *testing.T, pool *pgxpool.Pool, table string, start bool) *CDCIterator { is := is.New(t) config := CDCConfig{ From f1f44b6d7264d65198467405cbe54fee562292b9 Mon Sep 17 00:00:00 2001 From: Haris Osmanagic Date: Wed, 18 Jun 2025 13:05:34 +0200 Subject: [PATCH 39/60] destination from main --- destination.go | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/destination.go b/destination.go index 52f6098..0e6e4a3 100644 --- a/destination.go +++ b/destination.go @@ -187,6 +187,12 @@ func (d *Destination) remove(ctx context.Context, r opencdc.Record, b *pgx.Batch return fmt.Errorf("error formatting delete query: %w", err) } + sdk.Logger(ctx).Trace(). + Str("table", tableName). + Str("query", query). + Any("key", key). + Msg("deleting record") + b.Queue(query, args...) return nil } From 0bbc863d005c55ac551516ec4a3d86d0f4e51208 Mon Sep 17 00:00:00 2001 From: Haris Osmanagic Date: Wed, 18 Jun 2025 13:31:29 +0200 Subject: [PATCH 40/60] fix date --- source_integration_test.go | 3 +++ 1 file changed, 3 insertions(+) diff --git a/source_integration_test.go b/source_integration_test.go index d2f8bb8..01708d2 100644 --- a/source_integration_test.go +++ b/source_integration_test.go @@ -403,6 +403,9 @@ func normalizeNotNullValue(key string, value interface{}) interface{} { val := new(big.Rat) val.SetString(fmt.Sprintf("%v", value)) normalized = val + case strings.Contains(key, "_date"): + val := assert(time.Parse("2006-01-02", value.(string))) + normalized = val } return normalized From 2447721605276018f2a0cfe7733ceccd13743238 Mon Sep 17 00:00:00 2001 From: Haris Osmanagic Date: Wed, 18 Jun 2025 13:41:36 +0200 Subject: [PATCH 41/60] lint --- source/logrepl/cdc_test.go | 3 ++- source_integration_test.go | 3 +-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/source/logrepl/cdc_test.go b/source/logrepl/cdc_test.go index 4788b7e..bbb9a33 100644 --- a/source/logrepl/cdc_test.go +++ b/source/logrepl/cdc_test.go @@ -512,7 +512,8 @@ func verifyOpenCDCRecords(is *is.I, got []opencdc.Record, tableName string, from "column2": int32(i) * 100, //nolint:gosec // fine, we know the value is small enough "column3": false, "column4": big.NewRat(123, 10), - // UppercaseColumn1 is a Postgres interger (4 bytes) + // UppercaseColumn1 is a Postgres integer (4 bytes) + //nolint:gosec // integer overflow not possible, id is a small value always "UppercaseColumn1": int32(id), }, }, diff --git a/source_integration_test.go b/source_integration_test.go index 01708d2..e92edda 100644 --- a/source_integration_test.go +++ b/source_integration_test.go @@ -384,8 +384,7 @@ func expectedRecord(rowNumber int, notNullOnly bool) opencdc.StructuredData { func normalizeNullValue(key string, value interface{}) interface{} { normalized := value - switch { - case strings.Contains(key, "_uuid"): + if strings.Contains(key, "_uuid") { normalized = "" } From 8327705902f5d20cc3d6e87f97d0ed82d8eb3672 Mon Sep 17 00:00:00 2001 From: Haris Osmanagic Date: Wed, 18 Jun 2025 14:38:15 +0200 Subject: [PATCH 42/60] finally done with source_integration_test --- source_integration_test.go | 133 ++++++++++++++++--------------------- 1 file changed, 58 insertions(+), 75 deletions(-) diff --git a/source_integration_test.go b/source_integration_test.go index e92edda..a047aef 100644 --- a/source_integration_test.go +++ b/source_integration_test.go @@ -39,8 +39,8 @@ func TestSource_ReadN_Snapshot_CDC(t *testing.T) { tableName := createTableWithManyTypes(ctx, t) // Snapshot data - insertRowNotNullColumnsOnly(ctx, t, tableName, 1) - insertRowAllColumns(ctx, t, tableName, 2) + insertRowAllColumns(ctx, t, tableName, 1) + insertRowNotNullColumnsOnly(ctx, t, tableName, 2) slotName := "conduitslot1" publicationName := "conduitpub1" @@ -83,8 +83,8 @@ func TestSource_ReadN_Snapshot_CDC(t *testing.T) { is.NoErr(err) // Verify snapshot records - assertRecordOK(is, tableName, snapshotRecs[0], 1, true) - assertRecordOK(is, tableName, snapshotRecs[1], 2, false) + assertRecordOK(is, tableName, snapshotRecs[0], 1, false) + assertRecordOK(is, tableName, snapshotRecs[1], 2, true) // CDC data insertRowNotNullColumnsOnly(ctx, t, tableName, 3) @@ -161,7 +161,7 @@ func createTableWithManyTypes(ctx context.Context, t *testing.T) string { table := strings.ToUpper(test.RandomIdentifier(t)) query := fmt.Sprintf(`CREATE TABLE %q ( - id bigserial PRIMARY KEY, + id integer PRIMARY KEY, col_bytea bytea, col_bytea_not_null bytea NOT NULL, col_varchar varchar(10), @@ -218,7 +218,7 @@ func insertRowNotNullColumnsOnly(ctx context.Context, t *testing.T, table string is := is.New(t) conn := test.ConnectSimple(ctx, t, test.RepmgrConnString) - rec := generateRecord(rowNumber, true) + rec := generatePayloadData(rowNumber, true) query := fmt.Sprintf( `INSERT INTO %q ( @@ -238,10 +238,7 @@ func insertRowNotNullColumnsOnly(ctx context.Context, t *testing.T, table string col_uuid_not_null, col_json_not_null, col_jsonb_not_null, - col_bool_not_null, - col_serial_not_null, - col_smallserial_not_null, - col_bigserial_not_null + col_bool_not_null ) VALUES ( %d, '%s'::bytea, @@ -259,10 +256,7 @@ func insertRowNotNullColumnsOnly(ctx context.Context, t *testing.T, table string '%s'::uuid, '%s'::json, '%s'::jsonb, - %t, - %d, - %d, - %d + %t )`, table, rowNumber, @@ -282,9 +276,6 @@ func insertRowNotNullColumnsOnly(ctx context.Context, t *testing.T, table string rec["col_json_not_null"], rec["col_jsonb_not_null"], rec["col_bool_not_null"], - rec["col_serial_not_null"], - rec["col_smallserial_not_null"], - rec["col_bigserial_not_null"], ) _, err := conn.Exec(ctx, query) @@ -295,7 +286,7 @@ func insertRowAllColumns(ctx context.Context, t *testing.T, table string, rowNum is := is.New(t) conn := test.ConnectSimple(ctx, t, test.RepmgrConnString) - rec := generateRecord(rowNumber, false) + rec := generatePayloadData(rowNumber, false) query := fmt.Sprintf( `INSERT INTO %q ( @@ -315,10 +306,7 @@ func insertRowAllColumns(ctx context.Context, t *testing.T, table string, rowNum col_uuid, col_uuid_not_null, col_json, col_json_not_null, col_jsonb, col_jsonb_not_null, - col_bool, col_bool_not_null, - col_serial, col_serial_not_null, - col_smallserial, col_smallserial_not_null, - col_bigserial, col_bigserial_not_null + col_bool, col_bool_not_null ) VALUES ( %d, '%s'::bytea, '%s'::bytea, @@ -336,10 +324,7 @@ func insertRowAllColumns(ctx context.Context, t *testing.T, table string, rowNum '%s'::uuid, '%s'::uuid, '%s'::json, '%s'::json, '%s'::jsonb, '%s'::jsonb, - %t, %t, - %d, %d, - %d, %d, - %d, %d + %t, %t )`, table, rowNumber, @@ -359,9 +344,6 @@ func insertRowAllColumns(ctx context.Context, t *testing.T, table string, rowNum rec["col_json"], rec["col_json_not_null"], rec["col_jsonb"], rec["col_jsonb_not_null"], rec["col_bool"], rec["col_bool_not_null"], - rec["col_serial"], rec["col_serial_not_null"], - rec["col_smallserial"], rec["col_smallserial_not_null"], - rec["col_bigserial"], rec["col_bigserial_not_null"], ) _, err := conn.Exec(ctx, query) @@ -369,7 +351,7 @@ func insertRowAllColumns(ctx context.Context, t *testing.T, table string, rowNum } func expectedRecord(rowNumber int, notNullOnly bool) opencdc.StructuredData { - rec := generateRecord(rowNumber, notNullOnly) + rec := generatePayloadData(rowNumber, notNullOnly) for key, value := range rec { if value != nil { @@ -394,71 +376,72 @@ func normalizeNullValue(key string, value interface{}) interface{} { func normalizeNotNullValue(key string, value interface{}) interface{} { normalized := value switch { - case strings.Contains(key, "_bytea"), - strings.Contains(key, "_json"), - strings.Contains(key, "_jsonb"): + case strings.HasPrefix(key, "col_bytea"), + strings.HasPrefix(key, "col_json"), + strings.HasPrefix(key, "col_jsonb"): normalized = []uint8(value.(string)) - case strings.Contains(key, "_numeric"): + case strings.HasPrefix(key, "col_numeric"): val := new(big.Rat) val.SetString(fmt.Sprintf("%v", value)) normalized = val - case strings.Contains(key, "_date"): + case strings.HasPrefix(key, "col_date"): val := assert(time.Parse("2006-01-02", value.(string))) normalized = val + case strings.HasPrefix(key, "col_timestamp"): + val := assert(time.Parse(time.RFC3339, value.(string))) + normalized = val } return normalized } -func generateRecord(rowNumber int, notNullOnly bool) opencdc.StructuredData { - tsLayout := "2006-01-02 15:04:05.000000" - +func generatePayloadData(id int, notNullOnly bool) opencdc.StructuredData { rowTS, _ := time.Parse("2006-01-02 15:04:05", "2022-01-21 17:04:05") - rowTS = rowTS.Add(time.Duration(rowNumber) * time.Hour) + rowTS = rowTS.Add(time.Duration(id) * time.Hour) - rowUUID := fmt.Sprintf("a74a9875-978e-4832-b1b8-6b0f8793a%03d", rowNumber) + rowUUID := fmt.Sprintf("a74a9875-978e-4832-b1b8-6b0f8793a%03d", id) - id := int64(rowNumber) + idInt64 := int64(id) rec := opencdc.StructuredData{ "id": id, - "col_bytea": fmt.Sprintf("col_bytea_%v", rowNumber), - "col_bytea_not_null": fmt.Sprintf("col_bytea_%v", rowNumber), - "col_varchar": fmt.Sprintf("foo-%v", rowNumber), - "col_varchar_not_null": fmt.Sprintf("foo-%v", rowNumber), + "col_bytea": fmt.Sprintf("col_bytea_%v", id), + "col_bytea_not_null": fmt.Sprintf("col_bytea_%v", id), + "col_varchar": fmt.Sprintf("foo-%v", id), + "col_varchar_not_null": fmt.Sprintf("foo-%v", id), "col_date": rowTS.Format("2006-01-02"), "col_date_not_null": rowTS.Format("2006-01-02"), - "col_float4": float32(rowNumber) / 10, - "col_float4_not_null": float32(rowNumber) / 10, - "col_float8": float64(rowNumber) / 10, - "col_float8_not_null": float64(rowNumber) / 10, - "col_int2": rowNumber % 32768, - "col_int2_not_null": rowNumber % 32768, - "col_int4": rowNumber, - "col_int4_not_null": rowNumber, - "col_int8": id, - "col_int8_not_null": id, - "col_numeric": float64(100+rowNumber) / 10, - "col_numeric_not_null": float64(100+rowNumber) / 10, - "col_text": fmt.Sprintf("bar-%v", rowNumber), - "col_text_not_null": fmt.Sprintf("bar-%v", rowNumber), - "col_timestamp": rowTS.Format(tsLayout), - "col_timestamp_not_null": rowTS.Format(tsLayout), - "col_timestamptz": rowTS.Format(tsLayout), - "col_timestamptz_not_null": rowTS.Format(tsLayout), + "col_float4": float32(id) / 10, + "col_float4_not_null": float32(id) / 10, + "col_float8": float64(id) / 10, + "col_float8_not_null": float64(id) / 10, + "col_int2": id % 32768, + "col_int2_not_null": id % 32768, + "col_int4": id, + "col_int4_not_null": id, + "col_int8": idInt64, + "col_int8_not_null": idInt64, + "col_numeric": float64(100+id) / 10, + "col_numeric_not_null": float64(100+id) / 10, + "col_text": fmt.Sprintf("bar-%v", id), + "col_text_not_null": fmt.Sprintf("bar-%v", id), + "col_timestamp": rowTS.Format(time.RFC3339), + "col_timestamp_not_null": rowTS.Format(time.RFC3339), + "col_timestamptz": rowTS.Format(time.RFC3339), + "col_timestamptz_not_null": rowTS.Format(time.RFC3339), "col_uuid": rowUUID, "col_uuid_not_null": rowUUID, - "col_json": fmt.Sprintf(`{"key": "value-%v"}`, rowNumber), - "col_json_not_null": fmt.Sprintf(`{"key": "value-%v"}`, rowNumber), - "col_jsonb": fmt.Sprintf(`{"key": "value-%v"}`, rowNumber), - "col_jsonb_not_null": fmt.Sprintf(`{"key": "value-%v"}`, rowNumber), - "col_bool": rowNumber%2 == 0, - "col_bool_not_null": rowNumber%2 == 0, - "col_serial": rowNumber, - "col_serial_not_null": rowNumber, - "col_smallserial": rowNumber, - "col_smallserial_not_null": rowNumber, - "col_bigserial": id, - "col_bigserial_not_null": id, + "col_json": fmt.Sprintf(`{"key": "value-%v"}`, id), + "col_json_not_null": fmt.Sprintf(`{"key": "value-%v"}`, id), + "col_jsonb": fmt.Sprintf(`{"key": "value-%v"}`, id), + "col_jsonb_not_null": fmt.Sprintf(`{"key": "value-%v"}`, id), + "col_bool": id%2 == 0, + "col_bool_not_null": id%2 == 0, + "col_serial": id, + "col_serial_not_null": id, + "col_smallserial": id, + "col_smallserial_not_null": id, + "col_bigserial": idInt64, + "col_bigserial_not_null": idInt64, } if notNullOnly { From aa732dd950d18b023eb414c70cd6e735c6498117 Mon Sep 17 00:00:00 2001 From: Haris Osmanagic Date: Wed, 18 Jun 2025 14:50:20 +0200 Subject: [PATCH 43/60] simplify method --- source_integration_test.go | 33 +++++++++++++++++++-------------- 1 file changed, 19 insertions(+), 14 deletions(-) diff --git a/source_integration_test.go b/source_integration_test.go index a047aef..6dda103 100644 --- a/source_integration_test.go +++ b/source_integration_test.go @@ -107,21 +107,25 @@ func TestSource_ReadN_Snapshot_CDC(t *testing.T) { assertRecordOK(is, tableName, cdcRecs[0], 4, false) } -func assertRecordOK(is *is.I, tableName string, gotRecord opencdc.Record, rowNum int, notNullOnly bool) { +// assertRecordOK asserts that the input record has a schema and that its payload +// is what we expect (based on the ID and what columns are included). +func assertRecordOK(is *is.I, tableName string, gotRecord opencdc.Record, id int, notNullOnly bool) { is.Helper() is.True(gotRecord.Key != nil) is.True(gotRecord.Payload.After != nil) assertSchemaPresent(is, tableName, gotRecord) - assertPayloadOK(is, gotRecord, rowNum, notNullOnly) + assertPayloadOK(is, gotRecord, id, notNullOnly) } +// assertPayloadOK decodes the record's payload and asserts that the payload +// is what we expect (based on the ID and what columns are included). func assertPayloadOK(is *is.I, record opencdc.Record, rowNum int, notNullOnly bool) { is.Helper() sch, err := schema.Get( - context.TODO(), + context.Background(), assert(record.Metadata.GetPayloadSchemaSubject()), assert(record.Metadata.GetPayloadSchemaVersion()), ) @@ -131,7 +135,7 @@ func assertPayloadOK(is *is.I, record opencdc.Record, rowNum int, notNullOnly bo err = sch.Unmarshal(record.Payload.After.Bytes(), &got) is.NoErr(err) - want := expectedRecord(rowNum, notNullOnly) + want := expectedData(rowNum, notNullOnly) is.Equal("", cmp.Diff(want, got, test.BigRatComparer)) // -want, +got } @@ -350,8 +354,13 @@ func insertRowAllColumns(ctx context.Context, t *testing.T, table string, rowNum is.NoErr(err) } -func expectedRecord(rowNumber int, notNullOnly bool) opencdc.StructuredData { - rec := generatePayloadData(rowNumber, notNullOnly) +// expectedData creates an opencdc.StructuredData with expected keys and values +// based on the ID and the columns (NOT NULL columns only or all columns). +func expectedData(id int, notNullOnly bool) opencdc.StructuredData { + // We start with the data that was used to insert a test row. + // Then we normalize the data (e.g., JSON values are converted from strings + // to []uint8. + rec := generatePayloadData(id, notNullOnly) for key, value := range rec { if value != nil { @@ -376,10 +385,6 @@ func normalizeNullValue(key string, value interface{}) interface{} { func normalizeNotNullValue(key string, value interface{}) interface{} { normalized := value switch { - case strings.HasPrefix(key, "col_bytea"), - strings.HasPrefix(key, "col_json"), - strings.HasPrefix(key, "col_jsonb"): - normalized = []uint8(value.(string)) case strings.HasPrefix(key, "col_numeric"): val := new(big.Rat) val.SetString(fmt.Sprintf("%v", value)) @@ -430,10 +435,10 @@ func generatePayloadData(id int, notNullOnly bool) opencdc.StructuredData { "col_timestamptz_not_null": rowTS.Format(time.RFC3339), "col_uuid": rowUUID, "col_uuid_not_null": rowUUID, - "col_json": fmt.Sprintf(`{"key": "value-%v"}`, id), - "col_json_not_null": fmt.Sprintf(`{"key": "value-%v"}`, id), - "col_jsonb": fmt.Sprintf(`{"key": "value-%v"}`, id), - "col_jsonb_not_null": fmt.Sprintf(`{"key": "value-%v"}`, id), + "col_json": []uint8(fmt.Sprintf(`{"key": "json-value-%v"}`, id)), + "col_json_not_null": []uint8(fmt.Sprintf(`{"key": "json-not-value-%v"}`, id)), + "col_jsonb": []uint8(fmt.Sprintf(`{"key": "jsonb-value-%v"}`, id)), + "col_jsonb_not_null": []uint8(fmt.Sprintf(`{"key": "jsonb-not-value-%v"}`, id)), "col_bool": id%2 == 0, "col_bool_not_null": id%2 == 0, "col_serial": id, From e00e218cd45f85770da47a00c933e22cdef03b4e Mon Sep 17 00:00:00 2001 From: Haris Osmanagic Date: Wed, 18 Jun 2025 14:55:15 +0200 Subject: [PATCH 44/60] col_bytea --- source_integration_test.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/source_integration_test.go b/source_integration_test.go index 6dda103..1d2643c 100644 --- a/source_integration_test.go +++ b/source_integration_test.go @@ -409,8 +409,8 @@ func generatePayloadData(id int, notNullOnly bool) opencdc.StructuredData { idInt64 := int64(id) rec := opencdc.StructuredData{ "id": id, - "col_bytea": fmt.Sprintf("col_bytea_%v", id), - "col_bytea_not_null": fmt.Sprintf("col_bytea_%v", id), + "col_bytea": []uint8(fmt.Sprintf("col_bytea_%v", id)), + "col_bytea_not_null": []uint8(fmt.Sprintf("col_bytea_not_null_%v", id)), "col_varchar": fmt.Sprintf("foo-%v", id), "col_varchar_not_null": fmt.Sprintf("foo-%v", id), "col_date": rowTS.Format("2006-01-02"), From edbfe3c9c6308b13373dca8bfd125da2bb1176f5 Mon Sep 17 00:00:00 2001 From: Haris Osmanagic Date: Wed, 18 Jun 2025 15:05:15 +0200 Subject: [PATCH 45/60] simplify method --- source_integration_test.go | 23 +++++++++++++---------- 1 file changed, 13 insertions(+), 10 deletions(-) diff --git a/source_integration_test.go b/source_integration_test.go index 1d2643c..9a4869e 100644 --- a/source_integration_test.go +++ b/source_integration_test.go @@ -31,6 +31,7 @@ import ( "github.com/conduitio/conduit-connector-sdk/schema" "github.com/google/go-cmp/cmp" "github.com/matryer/is" + "github.com/shopspring/decimal" ) func TestSource_ReadN_Snapshot_CDC(t *testing.T) { @@ -253,7 +254,7 @@ func insertRowNotNullColumnsOnly(ctx context.Context, t *testing.T, table string %d, %d, %d, - %f, + %s, '%s', '%s'::timestamp, '%s'::timestamptz, @@ -272,7 +273,7 @@ func insertRowNotNullColumnsOnly(ctx context.Context, t *testing.T, table string rec["col_int2_not_null"], rec["col_int4_not_null"], rec["col_int8_not_null"], - rec["col_numeric_not_null"], + decimalString(rec["col_numeric_not_null"]), rec["col_text_not_null"], rec["col_timestamp_not_null"], rec["col_timestamptz_not_null"], @@ -321,7 +322,7 @@ func insertRowAllColumns(ctx context.Context, t *testing.T, table string, rowNum %d, %d, %d, %d, %d, %d, - %f, %f, + %s, %s, '%s', '%s', '%s'::timestamp, '%s'::timestamp, '%s'::timestamptz, '%s'::timestamptz, @@ -340,7 +341,7 @@ func insertRowAllColumns(ctx context.Context, t *testing.T, table string, rowNum rec["col_int2"], rec["col_int2_not_null"], rec["col_int4"], rec["col_int4_not_null"], rec["col_int8"], rec["col_int8_not_null"], - rec["col_numeric"], rec["col_numeric_not_null"], + decimalString(rec["col_numeric"]), decimalString(rec["col_numeric_not_null"]), rec["col_text"], rec["col_text_not_null"], rec["col_timestamp"], rec["col_timestamp_not_null"], rec["col_timestamptz"], rec["col_timestamptz_not_null"], @@ -354,6 +355,10 @@ func insertRowAllColumns(ctx context.Context, t *testing.T, table string, rowNum is.NoErr(err) } +func decimalString(v interface{}) string { + return decimal.NewFromBigRat(v.(*big.Rat), 2).String() +} + // expectedData creates an opencdc.StructuredData with expected keys and values // based on the ID and the columns (NOT NULL columns only or all columns). func expectedData(id int, notNullOnly bool) opencdc.StructuredData { @@ -385,10 +390,6 @@ func normalizeNullValue(key string, value interface{}) interface{} { func normalizeNotNullValue(key string, value interface{}) interface{} { normalized := value switch { - case strings.HasPrefix(key, "col_numeric"): - val := new(big.Rat) - val.SetString(fmt.Sprintf("%v", value)) - normalized = val case strings.HasPrefix(key, "col_date"): val := assert(time.Parse("2006-01-02", value.(string))) normalized = val @@ -407,6 +408,8 @@ func generatePayloadData(id int, notNullOnly bool) opencdc.StructuredData { rowUUID := fmt.Sprintf("a74a9875-978e-4832-b1b8-6b0f8793a%03d", id) idInt64 := int64(id) + numericVal := big.NewRat(int64(100+id), 10) + rec := opencdc.StructuredData{ "id": id, "col_bytea": []uint8(fmt.Sprintf("col_bytea_%v", id)), @@ -425,8 +428,8 @@ func generatePayloadData(id int, notNullOnly bool) opencdc.StructuredData { "col_int4_not_null": id, "col_int8": idInt64, "col_int8_not_null": idInt64, - "col_numeric": float64(100+id) / 10, - "col_numeric_not_null": float64(100+id) / 10, + "col_numeric": numericVal, + "col_numeric_not_null": numericVal, "col_text": fmt.Sprintf("bar-%v", id), "col_text_not_null": fmt.Sprintf("bar-%v", id), "col_timestamp": rowTS.Format(time.RFC3339), From ee8f150bd88454ecb573dd108ab46602fa59b179 Mon Sep 17 00:00:00 2001 From: Haris Osmanagic Date: Wed, 18 Jun 2025 15:21:42 +0200 Subject: [PATCH 46/60] simplify col_date --- source_integration_test.go | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/source_integration_test.go b/source_integration_test.go index 9a4869e..d5d2a0f 100644 --- a/source_integration_test.go +++ b/source_integration_test.go @@ -267,7 +267,7 @@ func insertRowNotNullColumnsOnly(ctx context.Context, t *testing.T, table string rowNumber, rec["col_bytea_not_null"], rec["col_varchar_not_null"], - rec["col_date_not_null"], + rec["col_date_not_null"].(time.Time).Format(time.DateOnly), rec["col_float4_not_null"], rec["col_float8_not_null"], rec["col_int2_not_null"], @@ -335,7 +335,7 @@ func insertRowAllColumns(ctx context.Context, t *testing.T, table string, rowNum rowNumber, rec["col_bytea"], rec["col_bytea_not_null"], rec["col_varchar"], rec["col_varchar_not_null"], - rec["col_date"], rec["col_date_not_null"], + rec["col_date"].(time.Time).Format(time.DateOnly), rec["col_date_not_null"].(time.Time).Format(time.DateOnly), rec["col_float4"], rec["col_float4_not_null"], rec["col_float8"], rec["col_float8_not_null"], rec["col_int2"], rec["col_int2_not_null"], @@ -390,9 +390,9 @@ func normalizeNullValue(key string, value interface{}) interface{} { func normalizeNotNullValue(key string, value interface{}) interface{} { normalized := value switch { - case strings.HasPrefix(key, "col_date"): - val := assert(time.Parse("2006-01-02", value.(string))) - normalized = val + // case strings.HasPrefix(key, "col_date"): + // val := assert(time.Parse("2006-01-02", value.(string))) + // normalized = val case strings.HasPrefix(key, "col_timestamp"): val := assert(time.Parse(time.RFC3339, value.(string))) normalized = val @@ -416,8 +416,8 @@ func generatePayloadData(id int, notNullOnly bool) opencdc.StructuredData { "col_bytea_not_null": []uint8(fmt.Sprintf("col_bytea_not_null_%v", id)), "col_varchar": fmt.Sprintf("foo-%v", id), "col_varchar_not_null": fmt.Sprintf("foo-%v", id), - "col_date": rowTS.Format("2006-01-02"), - "col_date_not_null": rowTS.Format("2006-01-02"), + "col_date": rowTS.Truncate(24 * time.Hour), + "col_date_not_null": rowTS.Truncate(24 * time.Hour), "col_float4": float32(id) / 10, "col_float4_not_null": float32(id) / 10, "col_float8": float64(id) / 10, From 49c99edb8f068b5fc12fe82d8b61b06c1646a2c2 Mon Sep 17 00:00:00 2001 From: Haris Osmanagic Date: Wed, 18 Jun 2025 15:29:58 +0200 Subject: [PATCH 47/60] no normalization needed --- source_integration_test.go | 111 ++++++++++++++++--------------------- 1 file changed, 47 insertions(+), 64 deletions(-) diff --git a/source_integration_test.go b/source_integration_test.go index d5d2a0f..ebeafe8 100644 --- a/source_integration_test.go +++ b/source_integration_test.go @@ -275,8 +275,8 @@ func insertRowNotNullColumnsOnly(ctx context.Context, t *testing.T, table string rec["col_int8_not_null"], decimalString(rec["col_numeric_not_null"]), rec["col_text_not_null"], - rec["col_timestamp_not_null"], - rec["col_timestamptz_not_null"], + rec["col_timestamp_not_null"].(time.Time).Format(time.RFC3339), + rec["col_timestamptz_not_null"].(time.Time).Format(time.RFC3339), rec["col_uuid_not_null"], rec["col_json_not_null"], rec["col_jsonb_not_null"], @@ -343,8 +343,8 @@ func insertRowAllColumns(ctx context.Context, t *testing.T, table string, rowNum rec["col_int8"], rec["col_int8_not_null"], decimalString(rec["col_numeric"]), decimalString(rec["col_numeric_not_null"]), rec["col_text"], rec["col_text_not_null"], - rec["col_timestamp"], rec["col_timestamp_not_null"], - rec["col_timestamptz"], rec["col_timestamptz_not_null"], + rec["col_timestamp"].(time.Time).Format(time.RFC3339), rec["col_timestamp_not_null"].(time.Time).Format(time.RFC3339), + rec["col_timestamptz"].(time.Time).Format(time.RFC3339), rec["col_timestamptz_not_null"].(time.Time).Format(time.RFC3339), rec["col_uuid"], rec["col_uuid_not_null"], rec["col_json"], rec["col_json_not_null"], rec["col_jsonb"], rec["col_jsonb_not_null"], @@ -368,39 +368,14 @@ func expectedData(id int, notNullOnly bool) opencdc.StructuredData { rec := generatePayloadData(id, notNullOnly) for key, value := range rec { - if value != nil { - rec[key] = normalizeNotNullValue(key, value) - } else { - rec[key] = normalizeNullValue(key, value) + if value == nil && strings.Contains(key, "_uuid") { + rec[key] = "" } } return rec } -func normalizeNullValue(key string, value interface{}) interface{} { - normalized := value - if strings.Contains(key, "_uuid") { - normalized = "" - } - - return normalized -} - -func normalizeNotNullValue(key string, value interface{}) interface{} { - normalized := value - switch { - // case strings.HasPrefix(key, "col_date"): - // val := assert(time.Parse("2006-01-02", value.(string))) - // normalized = val - case strings.HasPrefix(key, "col_timestamp"): - val := assert(time.Parse(time.RFC3339, value.(string))) - normalized = val - } - - return normalized -} - func generatePayloadData(id int, notNullOnly bool) opencdc.StructuredData { rowTS, _ := time.Parse("2006-01-02 15:04:05", "2022-01-21 17:04:05") rowTS = rowTS.Add(time.Duration(id) * time.Hour) @@ -411,45 +386,53 @@ func generatePayloadData(id int, notNullOnly bool) opencdc.StructuredData { numericVal := big.NewRat(int64(100+id), 10) rec := opencdc.StructuredData{ - "id": id, - "col_bytea": []uint8(fmt.Sprintf("col_bytea_%v", id)), - "col_bytea_not_null": []uint8(fmt.Sprintf("col_bytea_not_null_%v", id)), - "col_varchar": fmt.Sprintf("foo-%v", id), - "col_varchar_not_null": fmt.Sprintf("foo-%v", id), - "col_date": rowTS.Truncate(24 * time.Hour), - "col_date_not_null": rowTS.Truncate(24 * time.Hour), - "col_float4": float32(id) / 10, - "col_float4_not_null": float32(id) / 10, - "col_float8": float64(id) / 10, - "col_float8_not_null": float64(id) / 10, - "col_int2": id % 32768, - "col_int2_not_null": id % 32768, - "col_int4": id, - "col_int4_not_null": id, - "col_int8": idInt64, - "col_int8_not_null": idInt64, - "col_numeric": numericVal, - "col_numeric_not_null": numericVal, - "col_text": fmt.Sprintf("bar-%v", id), - "col_text_not_null": fmt.Sprintf("bar-%v", id), - "col_timestamp": rowTS.Format(time.RFC3339), - "col_timestamp_not_null": rowTS.Format(time.RFC3339), - "col_timestamptz": rowTS.Format(time.RFC3339), - "col_timestamptz_not_null": rowTS.Format(time.RFC3339), - "col_uuid": rowUUID, - "col_uuid_not_null": rowUUID, - "col_json": []uint8(fmt.Sprintf(`{"key": "json-value-%v"}`, id)), - "col_json_not_null": []uint8(fmt.Sprintf(`{"key": "json-not-value-%v"}`, id)), - "col_jsonb": []uint8(fmt.Sprintf(`{"key": "jsonb-value-%v"}`, id)), - "col_jsonb_not_null": []uint8(fmt.Sprintf(`{"key": "jsonb-not-value-%v"}`, id)), - "col_bool": id%2 == 0, - "col_bool_not_null": id%2 == 0, + "id": id, + + "col_bytea": []uint8(fmt.Sprintf("col_bytea_%v", id)), + "col_bytea_not_null": []uint8(fmt.Sprintf("col_bytea_not_null_%v", id)), + "col_varchar": fmt.Sprintf("foo-%v", id), + "col_varchar_not_null": fmt.Sprintf("foo-%v", id), + "col_text": fmt.Sprintf("bar-%v", id), + "col_text_not_null": fmt.Sprintf("bar-%v", id), + + "col_uuid": rowUUID, + "col_uuid_not_null": rowUUID, + + "col_json": []uint8(fmt.Sprintf(`{"key": "json-value-%v"}`, id)), + "col_json_not_null": []uint8(fmt.Sprintf(`{"key": "json-not-value-%v"}`, id)), + "col_jsonb": []uint8(fmt.Sprintf(`{"key": "jsonb-value-%v"}`, id)), + "col_jsonb_not_null": []uint8(fmt.Sprintf(`{"key": "jsonb-not-value-%v"}`, id)), + + "col_float4": float32(id) / 10, + "col_float4_not_null": float32(id) / 10, + "col_float8": float64(id) / 10, + "col_float8_not_null": float64(id) / 10, + "col_int2": id % 32768, + "col_int2_not_null": id % 32768, + "col_int4": id, + "col_int4_not_null": id, + "col_int8": idInt64, + "col_int8_not_null": idInt64, + + "col_numeric": numericVal, + "col_numeric_not_null": numericVal, + "col_serial": id, "col_serial_not_null": id, "col_smallserial": id, "col_smallserial_not_null": id, "col_bigserial": idInt64, "col_bigserial_not_null": idInt64, + + "col_date": rowTS.Truncate(24 * time.Hour), + "col_date_not_null": rowTS.Truncate(24 * time.Hour), + "col_timestamp": rowTS, + "col_timestamp_not_null": rowTS, + "col_timestamptz": rowTS, + "col_timestamptz_not_null": rowTS, + + "col_bool": id%2 == 0, + "col_bool_not_null": id%2 == 0, } if notNullOnly { From 19cabbaf70a0366d0371508b8f4c8454ee337421 Mon Sep 17 00:00:00 2001 From: Haris Osmanagic Date: Wed, 18 Jun 2025 15:47:46 +0200 Subject: [PATCH 48/60] less type casting --- source_integration_test.go | 104 ++++++++++++++++--------------------- 1 file changed, 46 insertions(+), 58 deletions(-) diff --git a/source_integration_test.go b/source_integration_test.go index ebeafe8..d0ff20b 100644 --- a/source_integration_test.go +++ b/source_integration_test.go @@ -22,8 +22,10 @@ import ( "testing" "time" + "github.com/Masterminds/squirrel" "github.com/conduitio/conduit-commons/config" "github.com/conduitio/conduit-commons/opencdc" + "github.com/conduitio/conduit-connector-postgres/internal" "github.com/conduitio/conduit-connector-postgres/source" "github.com/conduitio/conduit-connector-postgres/source/logrepl" "github.com/conduitio/conduit-connector-postgres/test" @@ -293,65 +295,51 @@ func insertRowAllColumns(ctx context.Context, t *testing.T, table string, rowNum rec := generatePayloadData(rowNumber, false) - query := fmt.Sprintf( - `INSERT INTO %q ( - id, - col_bytea, col_bytea_not_null, - col_varchar, col_varchar_not_null, - col_date, col_date_not_null, - col_float4, col_float4_not_null, - col_float8, col_float8_not_null, - col_int2, col_int2_not_null, - col_int4, col_int4_not_null, - col_int8, col_int8_not_null, - col_numeric, col_numeric_not_null, - col_text, col_text_not_null, - col_timestamp, col_timestamp_not_null, - col_timestamptz, col_timestamptz_not_null, - col_uuid, col_uuid_not_null, - col_json, col_json_not_null, - col_jsonb, col_jsonb_not_null, - col_bool, col_bool_not_null - ) VALUES ( - %d, - '%s'::bytea, '%s'::bytea, - '%s', '%s', - '%s'::date, '%s'::date, - %f, %f, - %f, %f, - %d, %d, - %d, %d, - %d, %d, - %s, %s, - '%s', '%s', - '%s'::timestamp, '%s'::timestamp, - '%s'::timestamptz, '%s'::timestamptz, - '%s'::uuid, '%s'::uuid, - '%s'::json, '%s'::json, - '%s'::jsonb, '%s'::jsonb, - %t, %t - )`, - table, - rowNumber, - rec["col_bytea"], rec["col_bytea_not_null"], - rec["col_varchar"], rec["col_varchar_not_null"], - rec["col_date"].(time.Time).Format(time.DateOnly), rec["col_date_not_null"].(time.Time).Format(time.DateOnly), - rec["col_float4"], rec["col_float4_not_null"], - rec["col_float8"], rec["col_float8_not_null"], - rec["col_int2"], rec["col_int2_not_null"], - rec["col_int4"], rec["col_int4_not_null"], - rec["col_int8"], rec["col_int8_not_null"], - decimalString(rec["col_numeric"]), decimalString(rec["col_numeric_not_null"]), - rec["col_text"], rec["col_text_not_null"], - rec["col_timestamp"].(time.Time).Format(time.RFC3339), rec["col_timestamp_not_null"].(time.Time).Format(time.RFC3339), - rec["col_timestamptz"].(time.Time).Format(time.RFC3339), rec["col_timestamptz_not_null"].(time.Time).Format(time.RFC3339), - rec["col_uuid"], rec["col_uuid_not_null"], - rec["col_json"], rec["col_json_not_null"], - rec["col_jsonb"], rec["col_jsonb_not_null"], - rec["col_bool"], rec["col_bool_not_null"], - ) + query, args, err := squirrel.Insert(internal.WrapSQLIdent(table)). + Columns( + "id", + "col_bytea", "col_bytea_not_null", + "col_varchar", "col_varchar_not_null", + "col_date", "col_date_not_null", + "col_float4", "col_float4_not_null", + "col_float8", "col_float8_not_null", + "col_int2", "col_int2_not_null", + "col_int4", "col_int4_not_null", + "col_int8", "col_int8_not_null", + "col_numeric", "col_numeric_not_null", + "col_text", "col_text_not_null", + "col_timestamp", "col_timestamp_not_null", + "col_timestamptz", "col_timestamptz_not_null", + "col_uuid", "col_uuid_not_null", + "col_json", "col_json_not_null", + "col_jsonb", "col_jsonb_not_null", + "col_bool", "col_bool_not_null", + ). + Values( + rowNumber, + rec["col_bytea"], rec["col_bytea_not_null"], + rec["col_varchar"], rec["col_varchar_not_null"], + rec["col_date"], rec["col_date_not_null"], + rec["col_float4"], rec["col_float4_not_null"], + rec["col_float8"], rec["col_float8_not_null"], + rec["col_int2"], rec["col_int2_not_null"], + rec["col_int4"], rec["col_int4_not_null"], + rec["col_int8"], rec["col_int8_not_null"], + squirrel.Expr("?", decimalString(rec["col_numeric"])), squirrel.Expr("?", decimalString(rec["col_numeric_not_null"])), + rec["col_text"], rec["col_text_not_null"], + rec["col_timestamp"], rec["col_timestamp_not_null"], + rec["col_timestamptz"], rec["col_timestamptz_not_null"], + squirrel.Expr("?::uuid", rec["col_uuid"]), squirrel.Expr("?::uuid", rec["col_uuid_not_null"]), + rec["col_json"], rec["col_json_not_null"], + rec["col_jsonb"], rec["col_jsonb_not_null"], + rec["col_bool"], rec["col_bool_not_null"], + ). + PlaceholderFormat(squirrel.Dollar). + ToSql() - _, err := conn.Exec(ctx, query) + is.NoErr(err) + + _, err = conn.Exec(ctx, query, args...) is.NoErr(err) } From 66b66d3f6ff1c93ddabbc891b2895bef1a556cb0 Mon Sep 17 00:00:00 2001 From: Haris Osmanagic Date: Wed, 18 Jun 2025 16:10:01 +0200 Subject: [PATCH 49/60] simplify test --- source_integration_test.go | 290 ++++++++++++++----------------------- 1 file changed, 108 insertions(+), 182 deletions(-) diff --git a/source_integration_test.go b/source_integration_test.go index d0ff20b..c45ddee 100644 --- a/source_integration_test.go +++ b/source_integration_test.go @@ -32,6 +32,7 @@ import ( sdk "github.com/conduitio/conduit-connector-sdk" "github.com/conduitio/conduit-connector-sdk/schema" "github.com/google/go-cmp/cmp" + "github.com/google/uuid" "github.com/matryer/is" "github.com/shopspring/decimal" ) @@ -42,8 +43,8 @@ func TestSource_ReadN_Snapshot_CDC(t *testing.T) { tableName := createTableWithManyTypes(ctx, t) // Snapshot data - insertRowAllColumns(ctx, t, tableName, 1) - insertRowNotNullColumnsOnly(ctx, t, tableName, 2) + insertRow(ctx, t, tableName, 1, false) + insertRow(ctx, t, tableName, 2, true) slotName := "conduitslot1" publicationName := "conduitpub1" @@ -90,8 +91,8 @@ func TestSource_ReadN_Snapshot_CDC(t *testing.T) { assertRecordOK(is, tableName, snapshotRecs[1], 2, true) // CDC data - insertRowNotNullColumnsOnly(ctx, t, tableName, 3) - insertRowAllColumns(ctx, t, tableName, 4) + insertRow(ctx, t, tableName, 3, true) + insertRow(ctx, t, tableName, 4, false) // Read, ack and verify the first CDC record cdcRecs, err := s.ReadN(ctx, 1) @@ -110,55 +111,6 @@ func TestSource_ReadN_Snapshot_CDC(t *testing.T) { assertRecordOK(is, tableName, cdcRecs[0], 4, false) } -// assertRecordOK asserts that the input record has a schema and that its payload -// is what we expect (based on the ID and what columns are included). -func assertRecordOK(is *is.I, tableName string, gotRecord opencdc.Record, id int, notNullOnly bool) { - is.Helper() - - is.True(gotRecord.Key != nil) - is.True(gotRecord.Payload.After != nil) - - assertSchemaPresent(is, tableName, gotRecord) - assertPayloadOK(is, gotRecord, id, notNullOnly) -} - -// assertPayloadOK decodes the record's payload and asserts that the payload -// is what we expect (based on the ID and what columns are included). -func assertPayloadOK(is *is.I, record opencdc.Record, rowNum int, notNullOnly bool) { - is.Helper() - - sch, err := schema.Get( - context.Background(), - assert(record.Metadata.GetPayloadSchemaSubject()), - assert(record.Metadata.GetPayloadSchemaVersion()), - ) - is.NoErr(err) - - got := opencdc.StructuredData{} - err = sch.Unmarshal(record.Payload.After.Bytes(), &got) - is.NoErr(err) - - want := expectedData(rowNum, notNullOnly) - - is.Equal("", cmp.Diff(want, got, test.BigRatComparer)) // -want, +got -} - -func assertSchemaPresent(is *is.I, tableName string, gotRecord opencdc.Record) { - payloadSchemaSubject, err := gotRecord.Metadata.GetPayloadSchemaSubject() - is.NoErr(err) - is.Equal(tableName+"_payload", payloadSchemaSubject) - payloadSchemaVersion, err := gotRecord.Metadata.GetPayloadSchemaVersion() - is.NoErr(err) - is.Equal(1, payloadSchemaVersion) - - keySchemaSubject, err := gotRecord.Metadata.GetKeySchemaSubject() - is.NoErr(err) - is.Equal(tableName+"_key", keySchemaSubject) - keySchemaVersion, err := gotRecord.Metadata.GetKeySchemaVersion() - is.NoErr(err) - is.Equal(1, keySchemaVersion) -} - func createTableWithManyTypes(ctx context.Context, t *testing.T) string { is := is.New(t) @@ -221,119 +173,38 @@ func createTableWithManyTypes(ctx context.Context, t *testing.T) string { return table } -func insertRowNotNullColumnsOnly(ctx context.Context, t *testing.T, table string, rowNumber int) { +// insertRow inserts a row using the values provided by generatePayloadData. +// if notNullOnly is true, only NOT NULL columns are inserted. +func insertRow(ctx context.Context, t *testing.T, table string, rowNumber int, notNullOnly bool) { is := is.New(t) conn := test.ConnectSimple(ctx, t, test.RepmgrConnString) - rec := generatePayloadData(rowNumber, true) - - query := fmt.Sprintf( - `INSERT INTO %q ( - id, - col_bytea_not_null, - col_varchar_not_null, - col_date_not_null, - col_float4_not_null, - col_float8_not_null, - col_int2_not_null, - col_int4_not_null, - col_int8_not_null, - col_numeric_not_null, - col_text_not_null, - col_timestamp_not_null, - col_timestamptz_not_null, - col_uuid_not_null, - col_json_not_null, - col_jsonb_not_null, - col_bool_not_null - ) VALUES ( - %d, - '%s'::bytea, - '%s', - '%s'::date, - %f, - %f, - %d, - %d, - %d, - %s, - '%s', - '%s'::timestamp, - '%s'::timestamptz, - '%s'::uuid, - '%s'::json, - '%s'::jsonb, - %t - )`, - table, - rowNumber, - rec["col_bytea_not_null"], - rec["col_varchar_not_null"], - rec["col_date_not_null"].(time.Time).Format(time.DateOnly), - rec["col_float4_not_null"], - rec["col_float8_not_null"], - rec["col_int2_not_null"], - rec["col_int4_not_null"], - rec["col_int8_not_null"], - decimalString(rec["col_numeric_not_null"]), - rec["col_text_not_null"], - rec["col_timestamp_not_null"].(time.Time).Format(time.RFC3339), - rec["col_timestamptz_not_null"].(time.Time).Format(time.RFC3339), - rec["col_uuid_not_null"], - rec["col_json_not_null"], - rec["col_jsonb_not_null"], - rec["col_bool_not_null"], - ) - - _, err := conn.Exec(ctx, query) - is.NoErr(err) -} + rec := generatePayloadData(rowNumber, false) -func insertRowAllColumns(ctx context.Context, t *testing.T, table string, rowNumber int) { - is := is.New(t) - conn := test.ConnectSimple(ctx, t, test.RepmgrConnString) + var columns []string + var values []interface{} + for key, value := range rec { + // the database generates serial values + if strings.Contains(key, "serial") { + continue + } + // check if only NOT NULL columns are needed (names ends in _not_null) + // id is an exception + if notNullOnly && !strings.HasSuffix(key, "_not_null") && key != "id" { + continue + } - rec := generatePayloadData(rowNumber, false) + columns = append(columns, key) + if strings.HasPrefix(key, "col_numeric") { + values = append(values, decimalString(value)) + } else { + values = append(values, value) + } + } query, args, err := squirrel.Insert(internal.WrapSQLIdent(table)). - Columns( - "id", - "col_bytea", "col_bytea_not_null", - "col_varchar", "col_varchar_not_null", - "col_date", "col_date_not_null", - "col_float4", "col_float4_not_null", - "col_float8", "col_float8_not_null", - "col_int2", "col_int2_not_null", - "col_int4", "col_int4_not_null", - "col_int8", "col_int8_not_null", - "col_numeric", "col_numeric_not_null", - "col_text", "col_text_not_null", - "col_timestamp", "col_timestamp_not_null", - "col_timestamptz", "col_timestamptz_not_null", - "col_uuid", "col_uuid_not_null", - "col_json", "col_json_not_null", - "col_jsonb", "col_jsonb_not_null", - "col_bool", "col_bool_not_null", - ). - Values( - rowNumber, - rec["col_bytea"], rec["col_bytea_not_null"], - rec["col_varchar"], rec["col_varchar_not_null"], - rec["col_date"], rec["col_date_not_null"], - rec["col_float4"], rec["col_float4_not_null"], - rec["col_float8"], rec["col_float8_not_null"], - rec["col_int2"], rec["col_int2_not_null"], - rec["col_int4"], rec["col_int4_not_null"], - rec["col_int8"], rec["col_int8_not_null"], - squirrel.Expr("?", decimalString(rec["col_numeric"])), squirrel.Expr("?", decimalString(rec["col_numeric_not_null"])), - rec["col_text"], rec["col_text_not_null"], - rec["col_timestamp"], rec["col_timestamp_not_null"], - rec["col_timestamptz"], rec["col_timestamptz_not_null"], - squirrel.Expr("?::uuid", rec["col_uuid"]), squirrel.Expr("?::uuid", rec["col_uuid_not_null"]), - rec["col_json"], rec["col_json_not_null"], - rec["col_jsonb"], rec["col_jsonb_not_null"], - rec["col_bool"], rec["col_bool_not_null"], - ). + Columns(columns...). + Values(values...). PlaceholderFormat(squirrel.Dollar). ToSql() @@ -343,33 +214,11 @@ func insertRowAllColumns(ctx context.Context, t *testing.T, table string, rowNum is.NoErr(err) } -func decimalString(v interface{}) string { - return decimal.NewFromBigRat(v.(*big.Rat), 2).String() -} - -// expectedData creates an opencdc.StructuredData with expected keys and values -// based on the ID and the columns (NOT NULL columns only or all columns). -func expectedData(id int, notNullOnly bool) opencdc.StructuredData { - // We start with the data that was used to insert a test row. - // Then we normalize the data (e.g., JSON values are converted from strings - // to []uint8. - rec := generatePayloadData(id, notNullOnly) - - for key, value := range rec { - if value == nil && strings.Contains(key, "_uuid") { - rec[key] = "" - } - } - - return rec -} - func generatePayloadData(id int, notNullOnly bool) opencdc.StructuredData { rowTS, _ := time.Parse("2006-01-02 15:04:05", "2022-01-21 17:04:05") rowTS = rowTS.Add(time.Duration(id) * time.Hour) - rowUUID := fmt.Sprintf("a74a9875-978e-4832-b1b8-6b0f8793a%03d", id) - + rowUUID := assert(uuid.Parse(fmt.Sprintf("a74a9875-978e-4832-b1b8-6b0f8793a%03d", id))) idInt64 := int64(id) numericVal := big.NewRat(int64(100+id), 10) @@ -405,6 +254,9 @@ func generatePayloadData(id int, notNullOnly bool) opencdc.StructuredData { "col_numeric": numericVal, "col_numeric_not_null": numericVal, + // NB: these values are not used in insert queries, but we assume + // the test rows will always be inserted in order, i.e., + // test row 1, then test row 2, etc. "col_serial": id, "col_serial_not_null": id, "col_smallserial": id, @@ -434,6 +286,80 @@ func generatePayloadData(id int, notNullOnly bool) opencdc.StructuredData { return rec } +func decimalString(v interface{}) string { + return decimal.NewFromBigRat(v.(*big.Rat), 2).String() +} + +// assertRecordOK asserts that the input record has a schema and that its payload +// is what we expect (based on the ID and what columns are included). +func assertRecordOK(is *is.I, tableName string, gotRecord opencdc.Record, id int, notNullOnly bool) { + is.Helper() + + is.True(gotRecord.Key != nil) + is.True(gotRecord.Payload.After != nil) + + assertSchemaPresent(is, tableName, gotRecord) + assertPayloadOK(is, gotRecord, id, notNullOnly) +} + +func assertSchemaPresent(is *is.I, tableName string, gotRecord opencdc.Record) { + payloadSchemaSubject, err := gotRecord.Metadata.GetPayloadSchemaSubject() + is.NoErr(err) + is.Equal(tableName+"_payload", payloadSchemaSubject) + payloadSchemaVersion, err := gotRecord.Metadata.GetPayloadSchemaVersion() + is.NoErr(err) + is.Equal(1, payloadSchemaVersion) + + keySchemaSubject, err := gotRecord.Metadata.GetKeySchemaSubject() + is.NoErr(err) + is.Equal(tableName+"_key", keySchemaSubject) + keySchemaVersion, err := gotRecord.Metadata.GetKeySchemaVersion() + is.NoErr(err) + is.Equal(1, keySchemaVersion) +} + +// assertPayloadOK decodes the record's payload and asserts that the payload +// is what we expect (based on the ID and what columns are included). +func assertPayloadOK(is *is.I, record opencdc.Record, rowNum int, notNullOnly bool) { + is.Helper() + + sch, err := schema.Get( + context.Background(), + assert(record.Metadata.GetPayloadSchemaSubject()), + assert(record.Metadata.GetPayloadSchemaVersion()), + ) + is.NoErr(err) + + got := opencdc.StructuredData{} + err = sch.Unmarshal(record.Payload.After.Bytes(), &got) + is.NoErr(err) + + want := expectedData(rowNum, notNullOnly) + + is.Equal("", cmp.Diff(want, got, test.BigRatComparer)) // -want, +got +} + +// expectedData creates an opencdc.StructuredData with expected keys and values +// based on the ID and the columns (NOT NULL columns only or all columns). +// It also converts values that are written into the test table as one type +// but read as another (e.g., we use UUID objects when inserting test data, +// but they are read as strings). +func expectedData(id int, notNullOnly bool) opencdc.StructuredData { + rec := generatePayloadData(id, notNullOnly) + + for key, value := range rec { + if strings.HasPrefix(key, "col_uuid") { + if value == nil { + rec[key] = "" + } else { + rec[key] = value.(uuid.UUID).String() + } + } + } + + return rec +} + func TestSource_ParseConfig(t *testing.T) { testCases := []struct { name string From 99f00301a5bd6ec04bc22907379b78180bcd78f1 Mon Sep 17 00:00:00 2001 From: Haris Osmanagic Date: Wed, 18 Jun 2025 16:12:12 +0200 Subject: [PATCH 50/60] add comment --- source/types/types.go | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/source/types/types.go b/source/types/types.go index 51e1c7c..98e5faf 100644 --- a/source/types/types.go +++ b/source/types/types.go @@ -26,6 +26,10 @@ var ( UUID = UUIDFormatter{} ) +// Format formats the input value v with the corresponding Postgres OID +// into an appropriate Go value (that can later be serialized with Avro). +// If the input value is nullable (i.e. isNotNull is false), then the method +// returns a pointer. // todo still need to support: // bit, varbit, box, char(n), cidr, circle, inet, interval, line, lseg, // macaddr, macaddr8, money, path, pg_lsn, pg_snapshot, point, polygon, From 42ecfd5b91ccd74a6ad94e45a099bbfc6c203e42 Mon Sep 17 00:00:00 2001 From: Haris Osmanagic Date: Wed, 18 Jun 2025 17:09:36 +0200 Subject: [PATCH 51/60] add time zone offsets --- source_integration_test.go | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/source_integration_test.go b/source_integration_test.go index c45ddee..2e50c86 100644 --- a/source_integration_test.go +++ b/source_integration_test.go @@ -215,7 +215,8 @@ func insertRow(ctx context.Context, t *testing.T, table string, rowNumber int, n } func generatePayloadData(id int, notNullOnly bool) opencdc.StructuredData { - rowTS, _ := time.Parse("2006-01-02 15:04:05", "2022-01-21 17:04:05") + // Add a time zone offset + rowTS := assert(time.Parse(time.RFC3339, fmt.Sprintf("2022-01-21T17:04:05+%02d:00", id))) rowTS = rowTS.Add(time.Duration(id) * time.Hour) rowUUID := assert(uuid.Parse(fmt.Sprintf("a74a9875-978e-4832-b1b8-6b0f8793a%03d", id))) @@ -266,8 +267,8 @@ func generatePayloadData(id int, notNullOnly bool) opencdc.StructuredData { "col_date": rowTS.Truncate(24 * time.Hour), "col_date_not_null": rowTS.Truncate(24 * time.Hour), - "col_timestamp": rowTS, - "col_timestamp_not_null": rowTS, + "col_timestamp": rowTS.UTC(), + "col_timestamp_not_null": rowTS.UTC(), "col_timestamptz": rowTS, "col_timestamptz_not_null": rowTS, From fa095eb8b653eaa765f3af4bfe25dea3d81f48bc Mon Sep 17 00:00:00 2001 From: Haris Osmanagic Date: Wed, 18 Jun 2025 17:30:57 +0200 Subject: [PATCH 52/60] comments --- source_integration_test.go | 66 +++++++++++++++++++------------------- 1 file changed, 33 insertions(+), 33 deletions(-) diff --git a/source_integration_test.go b/source_integration_test.go index 2e50c86..b112741 100644 --- a/source_integration_test.go +++ b/source_integration_test.go @@ -42,7 +42,7 @@ func TestSource_ReadN_Snapshot_CDC(t *testing.T) { ctx := test.Context(t) tableName := createTableWithManyTypes(ctx, t) - // Snapshot data + // Write the snapshot data insertRow(ctx, t, tableName, 1, false) insertRow(ctx, t, tableName, 2, true) @@ -77,38 +77,36 @@ func TestSource_ReadN_Snapshot_CDC(t *testing.T) { }) // Read and ack the 2 snapshots records - snapshotRecs, err := s.ReadN(ctx, 2) + recs, err := s.ReadN(ctx, 2) is.NoErr(err) - is.Equal(2, len(snapshotRecs)) + is.Equal(2, len(recs)) - err = s.Ack(ctx, snapshotRecs[0].Position) + // Verify snapshot record no. 1 + err = s.Ack(ctx, recs[0].Position) is.NoErr(err) - err = s.Ack(ctx, snapshotRecs[1].Position) + assertRecordOK(is, tableName, recs[0], 1, false) + // Verify snapshot record no. 2 + err = s.Ack(ctx, recs[1].Position) is.NoErr(err) + assertRecordOK(is, tableName, recs[1], 2, true) - // Verify snapshot records - assertRecordOK(is, tableName, snapshotRecs[0], 1, false) - assertRecordOK(is, tableName, snapshotRecs[1], 2, true) - - // CDC data + // Write the CDC data insertRow(ctx, t, tableName, 3, true) insertRow(ctx, t, tableName, 4, false) - // Read, ack and verify the first CDC record - cdcRecs, err := s.ReadN(ctx, 1) - is.NoErr(err) - is.Equal(1, len(cdcRecs)) - err = s.Ack(ctx, cdcRecs[0].Position) + // Read, ack, and verify CDC record no. 1 + recs, err = s.ReadN(ctx, 1) is.NoErr(err) - assertRecordOK(is, tableName, cdcRecs[0], 3, true) + is.Equal(1, len(recs)) + // record no. 3 has the NOT NULL columns only + assertRecordOK(is, tableName, recs[0], 3, true) - // Read, ack and verify the second CDC record - cdcRecs, err = s.ReadN(ctx, 1) - is.NoErr(err) - is.Equal(1, len(cdcRecs)) - err = s.Ack(ctx, cdcRecs[0].Position) + // Read, ack, and verify CDC record no. 1 + recs, err = s.ReadN(ctx, 1) is.NoErr(err) - assertRecordOK(is, tableName, cdcRecs[0], 4, false) + is.Equal(1, len(recs)) + // record no. 3 has the NOT NULL columns only + assertRecordOK(is, tableName, recs[0], 4, false) } func createTableWithManyTypes(ctx context.Context, t *testing.T) string { @@ -123,8 +121,8 @@ func createTableWithManyTypes(ctx context.Context, t *testing.T) string { id integer PRIMARY KEY, col_bytea bytea, col_bytea_not_null bytea NOT NULL, - col_varchar varchar(10), - col_varchar_not_null varchar(10) NOT NULL, + col_varchar varchar(30), + col_varchar_not_null varchar(30) NOT NULL, col_date date, col_date_not_null date NOT NULL, col_float4 float4, @@ -195,6 +193,7 @@ func insertRow(ctx context.Context, t *testing.T, table string, rowNumber int, n } columns = append(columns, key) + // col_numeric is a big.Rat, so we convert it to a string if strings.HasPrefix(key, "col_numeric") { values = append(values, decimalString(value)) } else { @@ -228,10 +227,10 @@ func generatePayloadData(id int, notNullOnly bool) opencdc.StructuredData { "col_bytea": []uint8(fmt.Sprintf("col_bytea_%v", id)), "col_bytea_not_null": []uint8(fmt.Sprintf("col_bytea_not_null_%v", id)), - "col_varchar": fmt.Sprintf("foo-%v", id), - "col_varchar_not_null": fmt.Sprintf("foo-%v", id), - "col_text": fmt.Sprintf("bar-%v", id), - "col_text_not_null": fmt.Sprintf("bar-%v", id), + "col_varchar": fmt.Sprintf("col_varchar_%v", id), + "col_varchar_not_null": fmt.Sprintf("col_varchar_not_null_%v", id), + "col_text": fmt.Sprintf("col_text_%v", id), + "col_text_not_null": fmt.Sprintf("col_text_not_null_%v", id), "col_uuid": rowUUID, "col_uuid_not_null": rowUUID, @@ -273,7 +272,7 @@ func generatePayloadData(id int, notNullOnly bool) opencdc.StructuredData { "col_timestamptz_not_null": rowTS, "col_bool": id%2 == 0, - "col_bool_not_null": id%2 == 0, + "col_bool_not_null": id%2 == 1, } if notNullOnly { @@ -337,18 +336,19 @@ func assertPayloadOK(is *is.I, record opencdc.Record, rowNum int, notNullOnly bo want := expectedData(rowNum, notNullOnly) - is.Equal("", cmp.Diff(want, got, test.BigRatComparer)) // -want, +got + is.Equal("", cmp.Diff(want, got, test.BigRatComparer)) // expected different payload (-want, +got) } // expectedData creates an opencdc.StructuredData with expected keys and values // based on the ID and the columns (NOT NULL columns only or all columns). -// It also converts values that are written into the test table as one type -// but read as another (e.g., we use UUID objects when inserting test data, -// but they are read as strings). +// Its output is different generatePayloadData, because certain values are written +// into the test table as one type, but read as another (e.g., we use UUID objects +// when inserting test data, but they are read as strings). func expectedData(id int, notNullOnly bool) opencdc.StructuredData { rec := generatePayloadData(id, notNullOnly) for key, value := range rec { + // UUID are written as byte arrays but read as strings. if strings.HasPrefix(key, "col_uuid") { if value == nil { rec[key] = "" From 0eb72c599cf4c5835956aa09b1e52a8ddd0f7dd2 Mon Sep 17 00:00:00 2001 From: Haris Osmanagic Date: Wed, 18 Jun 2025 18:59:21 +0200 Subject: [PATCH 53/60] move table_info --- {source/common => internal}/table_info.go | 5 ++--- source/logrepl/cdc.go | 4 ++-- source/logrepl/handler.go | 6 +++--- source/logrepl/internal/relationset.go | 6 +++--- source/logrepl/internal/relationset_test.go | 4 ++-- source/schema/avro.go | 6 +++--- source/schema/avro_integration_test.go | 4 ++-- source/snapshot/fetch_worker.go | 7 +++---- source/types/types.go | 3 ++- 9 files changed, 22 insertions(+), 23 deletions(-) rename {source/common => internal}/table_info.go (94%) diff --git a/source/common/table_info.go b/internal/table_info.go similarity index 94% rename from source/common/table_info.go rename to internal/table_info.go index ce4d413..d1a191c 100644 --- a/source/common/table_info.go +++ b/internal/table_info.go @@ -12,13 +12,12 @@ // See the License for the specific language governing permissions and // limitations under the License. -package common +package internal import ( "context" "fmt" - "github.com/conduitio/conduit-connector-postgres/internal" sdk "github.com/conduitio/conduit-connector-sdk" "github.com/jackc/pgx/v5/pgxpool" ) @@ -73,7 +72,7 @@ func (i TableInfoFetcher) Refresh(ctx context.Context, tableName string) error { ORDER BY a.attnum; ` - rows, err := tx.Query(context.Background(), query, internal.WrapSQLIdent(tableName)) + rows, err := tx.Query(ctx, query, WrapSQLIdent(tableName)) if err != nil { sdk.Logger(ctx). Err(err). diff --git a/source/logrepl/cdc.go b/source/logrepl/cdc.go index b55062a..c3306c5 100644 --- a/source/logrepl/cdc.go +++ b/source/logrepl/cdc.go @@ -21,7 +21,7 @@ import ( "time" "github.com/conduitio/conduit-commons/opencdc" - "github.com/conduitio/conduit-connector-postgres/source/common" + internal2 "github.com/conduitio/conduit-connector-postgres/internal" "github.com/conduitio/conduit-connector-postgres/source/logrepl/internal" "github.com/conduitio/conduit-connector-postgres/source/position" sdk "github.com/conduitio/conduit-connector-sdk" @@ -88,7 +88,7 @@ func NewCDCIterator(ctx context.Context, pool *pgxpool.Pool, c CDCConfig) (*CDCI handler := NewCDCHandler( ctx, internal.NewRelationSet(), - common.NewTableInfoFetcher(pool), + internal2.NewTableInfoFetcher(pool), c.TableKeys, batchesCh, c.WithAvroSchema, diff --git a/source/logrepl/handler.go b/source/logrepl/handler.go index 042c070..6754ea5 100644 --- a/source/logrepl/handler.go +++ b/source/logrepl/handler.go @@ -22,7 +22,7 @@ import ( "github.com/conduitio/conduit-commons/opencdc" cschema "github.com/conduitio/conduit-commons/schema" - "github.com/conduitio/conduit-connector-postgres/source/common" + internal2 "github.com/conduitio/conduit-connector-postgres/internal" "github.com/conduitio/conduit-connector-postgres/source/logrepl/internal" "github.com/conduitio/conduit-connector-postgres/source/position" "github.com/conduitio/conduit-connector-postgres/source/schema" @@ -37,7 +37,7 @@ type CDCHandler struct { tableKeys map[string]string relationSet *internal.RelationSet - tableInfo *common.TableInfoFetcher + tableInfo *internal2.TableInfoFetcher // batchSize is the largest number of records this handler will send at once. batchSize int @@ -58,7 +58,7 @@ type CDCHandler struct { func NewCDCHandler( ctx context.Context, rs *internal.RelationSet, - tableInfo *common.TableInfoFetcher, + tableInfo *internal2.TableInfoFetcher, tableKeys map[string]string, out chan<- []opencdc.Record, withAvroSchema bool, diff --git a/source/logrepl/internal/relationset.go b/source/logrepl/internal/relationset.go index bfe9ad4..ed0cffb 100644 --- a/source/logrepl/internal/relationset.go +++ b/source/logrepl/internal/relationset.go @@ -18,7 +18,7 @@ import ( "errors" "fmt" - "github.com/conduitio/conduit-connector-postgres/source/common" + "github.com/conduitio/conduit-connector-postgres/internal" "github.com/conduitio/conduit-connector-postgres/source/types" "github.com/jackc/pglogrepl" "github.com/jackc/pgx/v5/pgtype" @@ -51,7 +51,7 @@ func (rs *RelationSet) Get(id uint32) (*pglogrepl.RelationMessage, error) { return msg, nil } -func (rs *RelationSet) Values(id uint32, row *pglogrepl.TupleData, tableInfo *common.TableInfo) (map[string]any, error) { +func (rs *RelationSet) Values(id uint32, row *pglogrepl.TupleData, tableInfo *internal.TableInfo) (map[string]any, error) { if row == nil { return nil, errors.New("no tuple data") } @@ -85,7 +85,7 @@ func (rs *RelationSet) oidToCodec(id uint32) pgtype.Codec { return dt.Codec } -func (rs *RelationSet) decodeValue(col *pglogrepl.RelationMessageColumn, colInfo *common.ColumnInfo, data []byte) (any, error) { +func (rs *RelationSet) decodeValue(col *pglogrepl.RelationMessageColumn, colInfo *internal.ColumnInfo, data []byte) (any, error) { decoder := rs.oidToCodec(col.DataType) // This workaround is due to an issue in pgx v5.7.1. // Namely, that version introduces an XML codec diff --git a/source/logrepl/internal/relationset_test.go b/source/logrepl/internal/relationset_test.go index c08ad56..b26a582 100644 --- a/source/logrepl/internal/relationset_test.go +++ b/source/logrepl/internal/relationset_test.go @@ -23,7 +23,7 @@ import ( "testing" "time" - "github.com/conduitio/conduit-connector-postgres/source/common" + "github.com/conduitio/conduit-connector-postgres/internal" "github.com/conduitio/conduit-connector-postgres/test" "github.com/google/go-cmp/cmp" "github.com/jackc/pglogrepl" @@ -77,7 +77,7 @@ func TestRelationSetAllTypes(t *testing.T) { break } - tableInfo := common.NewTableInfoFetcher(pool) + tableInfo := internal.NewTableInfoFetcher(pool) err := tableInfo.Refresh(ctx, table) is.NoErr(err) diff --git a/source/schema/avro.go b/source/schema/avro.go index 7cd3b28..71c06a4 100644 --- a/source/schema/avro.go +++ b/source/schema/avro.go @@ -19,7 +19,7 @@ import ( "fmt" "slices" - "github.com/conduitio/conduit-connector-postgres/source/common" + "github.com/conduitio/conduit-connector-postgres/internal" "github.com/hamba/avro/v2" "github.com/jackc/pglogrepl" "github.com/jackc/pgx/v5/pgconn" @@ -66,7 +66,7 @@ type avroExtractor struct { // ExtractLogrepl extracts an Avro schema from the given pglogrepl.RelationMessage. // If `fieldNames` are specified, then only the given fields will be included in the schema. -func (a *avroExtractor) ExtractLogrepl(schemaName string, rel *pglogrepl.RelationMessage, tableInfo *common.TableInfo, fieldNames ...string) (*avro.RecordSchema, error) { +func (a *avroExtractor) ExtractLogrepl(schemaName string, rel *pglogrepl.RelationMessage, tableInfo *internal.TableInfo, fieldNames ...string) (*avro.RecordSchema, error) { var fields []pgconn.FieldDescription for i := range rel.Columns { @@ -82,7 +82,7 @@ func (a *avroExtractor) ExtractLogrepl(schemaName string, rel *pglogrepl.Relatio // Extract extracts an Avro schema from the given Postgres field descriptions. // If `fieldNames` are specified, then only the given fields will be included in the schema. -func (a *avroExtractor) Extract(schemaName string, tableInfo *common.TableInfo, fields []pgconn.FieldDescription, fieldNames ...string) (*avro.RecordSchema, error) { +func (a *avroExtractor) Extract(schemaName string, tableInfo *internal.TableInfo, fields []pgconn.FieldDescription, fieldNames ...string) (*avro.RecordSchema, error) { var avroFields []*avro.Field for _, f := range fields { diff --git a/source/schema/avro_integration_test.go b/source/schema/avro_integration_test.go index 582bb58..e21fb8f 100644 --- a/source/schema/avro_integration_test.go +++ b/source/schema/avro_integration_test.go @@ -25,7 +25,7 @@ import ( "testing" "time" - "github.com/conduitio/conduit-connector-postgres/source/common" + "github.com/conduitio/conduit-connector-postgres/internal" "github.com/conduitio/conduit-connector-postgres/source/cpool" "github.com/conduitio/conduit-connector-postgres/source/types" "github.com/conduitio/conduit-connector-postgres/test" @@ -43,7 +43,7 @@ func Test_AvroExtract(t *testing.T) { is.NoErr(err) table := setupAvroTestTable(ctx, t, c) - tableInfoFetcher := common.NewTableInfoFetcher(connPool) + tableInfoFetcher := internal.NewTableInfoFetcher(connPool) err = tableInfoFetcher.Refresh(ctx, table) is.NoErr(err) diff --git a/source/snapshot/fetch_worker.go b/source/snapshot/fetch_worker.go index 6e5da57..2a46162 100644 --- a/source/snapshot/fetch_worker.go +++ b/source/snapshot/fetch_worker.go @@ -25,7 +25,6 @@ import ( "github.com/conduitio/conduit-commons/opencdc" cschema "github.com/conduitio/conduit-commons/schema" "github.com/conduitio/conduit-connector-postgres/internal" - "github.com/conduitio/conduit-connector-postgres/source/common" "github.com/conduitio/conduit-connector-postgres/source/position" "github.com/conduitio/conduit-connector-postgres/source/schema" "github.com/conduitio/conduit-connector-postgres/source/types" @@ -96,7 +95,7 @@ type FetchWorker struct { out chan<- []FetchData // notNullMap maps column names to if the column is NOT NULL. - tableInfoFetcher *common.TableInfoFetcher + tableInfoFetcher *internal.TableInfoFetcher keySchema *cschema.Schema payloadSchema *cschema.Schema @@ -110,7 +109,7 @@ func NewFetchWorker(db *pgxpool.Pool, out chan<- []FetchData, c FetchConfig) *Fe conf: c, db: db, out: out, - tableInfoFetcher: common.NewTableInfoFetcher(db), + tableInfoFetcher: internal.NewTableInfoFetcher(db), cursorName: "fetcher_" + strings.ReplaceAll(uuid.NewString(), "-", ""), } @@ -549,6 +548,6 @@ func (f *FetchWorker) extractSchemas(ctx context.Context, fields []pgconn.FieldD return nil } -func (f *FetchWorker) getTableInfo() *common.TableInfo { +func (f *FetchWorker) getTableInfo() *internal.TableInfo { return f.tableInfoFetcher.GetTable(f.conf.Table) } diff --git a/source/types/types.go b/source/types/types.go index 98e5faf..02e8464 100644 --- a/source/types/types.go +++ b/source/types/types.go @@ -30,7 +30,8 @@ var ( // into an appropriate Go value (that can later be serialized with Avro). // If the input value is nullable (i.e. isNotNull is false), then the method // returns a pointer. -// todo still need to support: +// +// The following types are currently not supported: // bit, varbit, box, char(n), cidr, circle, inet, interval, line, lseg, // macaddr, macaddr8, money, path, pg_lsn, pg_snapshot, point, polygon, // time, timetz, tsquery, tsvector, xml From 0fb14acee44c6cb35784b80639a09598baddd8d0 Mon Sep 17 00:00:00 2001 From: Haris Osmanagic Date: Wed, 18 Jun 2025 19:42:19 +0200 Subject: [PATCH 54/60] fix uuid test --- source/types/types.go | 8 ++++---- source/types/types_test.go | 7 ++++--- source/types/uuid.go | 4 ++-- source_integration_test.go | 8 ++------ 4 files changed, 12 insertions(+), 15 deletions(-) diff --git a/source/types/types.go b/source/types/types.go index 02e8464..d9de8ba 100644 --- a/source/types/types.go +++ b/source/types/types.go @@ -36,15 +36,15 @@ var ( // macaddr, macaddr8, money, path, pg_lsn, pg_snapshot, point, polygon, // time, timetz, tsquery, tsvector, xml func Format(oid uint32, v any, isNotNull bool) (any, error) { + if v == nil { + return nil, nil + } + val, err := format(oid, v) if err != nil { return nil, err } - if val == nil { - return nil, nil - } - if reflect.TypeOf(val).Kind() != reflect.Ptr && !isNotNull { return GetPointer(val), nil } diff --git a/source/types/types_test.go b/source/types/types_test.go index cb206ca..d7d2520 100644 --- a/source/types/types_test.go +++ b/source/types/types_test.go @@ -82,16 +82,17 @@ func Test_Format(t *testing.T) { { name: "uuid", input: []any{ - [16]uint8{0xbd, 0x94, 0xee, 0x0b, 0x56, 0x4f, 0x40, 0x88, 0xbf, 0x4e, 0x8d, 0x5e, 0x62, 0x6c, 0xaf, 0x66}, nil, + [16]uint8{0xbd, 0x94, 0xee, 0x0b, 0x56, 0x4f, 0x40, 0x88, 0xbf, 0x4e, 0x8d, 0x5e, 0x62, 0x6c, 0xaf, 0x66}, + nil, }, inputOID: []uint32{ pgtype.UUIDOID, pgtype.UUIDOID, }, expect: []any{ - "bd94ee0b-564f-4088-bf4e-8d5e626caf66", "", + "bd94ee0b-564f-4088-bf4e-8d5e626caf66", nil, }, expectNullable: []any{ - lang.Ptr("bd94ee0b-564f-4088-bf4e-8d5e626caf66"), lang.Ptr(""), + lang.Ptr("bd94ee0b-564f-4088-bf4e-8d5e626caf66"), nil, }, }, } diff --git a/source/types/uuid.go b/source/types/uuid.go index c2f8093..c941e00 100644 --- a/source/types/uuid.go +++ b/source/types/uuid.go @@ -22,8 +22,8 @@ import ( type UUIDFormatter struct{} -// Format takes a slice of bytes and returns a UUID in string format -// Returns error when byte array cannot be parsed. +// Format transforms a byte array into a UUID in string format. +// Returns an error if the byte array cannot be parsed. func (UUIDFormatter) Format(v any) (string, error) { if v == nil { return "", nil diff --git a/source_integration_test.go b/source_integration_test.go index b112741..2d26095 100644 --- a/source_integration_test.go +++ b/source_integration_test.go @@ -349,12 +349,8 @@ func expectedData(id int, notNullOnly bool) opencdc.StructuredData { for key, value := range rec { // UUID are written as byte arrays but read as strings. - if strings.HasPrefix(key, "col_uuid") { - if value == nil { - rec[key] = "" - } else { - rec[key] = value.(uuid.UUID).String() - } + if strings.HasPrefix(key, "col_uuid") && value != nil { + rec[key] = value.(uuid.UUID).String() } } From 0db44e42a564fbf4d7dbe04d8f00bff6a91ad0c0 Mon Sep 17 00:00:00 2001 From: Haris Osmanagic Date: Wed, 18 Jun 2025 20:43:43 +0200 Subject: [PATCH 55/60] remove comment --- source/schema/avro_integration_test.go | 10 ---------- 1 file changed, 10 deletions(-) diff --git a/source/schema/avro_integration_test.go b/source/schema/avro_integration_test.go index e21fb8f..36a8003 100644 --- a/source/schema/avro_integration_test.go +++ b/source/schema/avro_integration_test.go @@ -546,16 +546,6 @@ func avrolizeMap(fields []pgconn.FieldDescription, values []any) map[string]any strings.HasSuffix(f.Name, "_not_null") row[f.Name] = assert(types.Format(f.DataTypeOID, values[i], isNotNull)) - // switch f.DataTypeOID { - // case pgtype.NumericOID: - // n := new(big.Rat) - // n.SetString(fmt.Sprint(types.Format(0, values[i], true))) - // row[f.Name] = n - // case pgtype.UUIDOID: - // row[f.Name] = fmt.Sprint(values[i]) - // default: - // row[f.Name] = values[i] - // } } return row From 4406396d3c285fc192576d3362f19b1af9ee6e81 Mon Sep 17 00:00:00 2001 From: Haris Osmanagic Date: Fri, 20 Jun 2025 13:35:44 +0200 Subject: [PATCH 56/60] add test for delete --- source_integration_test.go | 65 ++++++++++++++++++++------------------ 1 file changed, 35 insertions(+), 30 deletions(-) diff --git a/source_integration_test.go b/source_integration_test.go index 2d26095..1b23231 100644 --- a/source_integration_test.go +++ b/source_integration_test.go @@ -33,18 +33,21 @@ import ( "github.com/conduitio/conduit-connector-sdk/schema" "github.com/google/go-cmp/cmp" "github.com/google/uuid" + "github.com/jackc/pgx/v5" "github.com/matryer/is" "github.com/shopspring/decimal" ) +// todo test updates and deletes, for not null and nullable values func TestSource_ReadN_Snapshot_CDC(t *testing.T) { is := is.New(t) ctx := test.Context(t) + notNullOnly := false + conn := test.ConnectSimple(ctx, t, test.RepmgrConnString) tableName := createTableWithManyTypes(ctx, t) // Write the snapshot data - insertRow(ctx, t, tableName, 1, false) - insertRow(ctx, t, tableName, 2, true) + insertRow(ctx, is, conn, tableName, 1, notNullOnly) slotName := "conduitslot1" publicationName := "conduitpub1" @@ -76,37 +79,31 @@ func TestSource_ReadN_Snapshot_CDC(t *testing.T) { is.NoErr(s.Teardown(ctx)) }) - // Read and ack the 2 snapshots records - recs, err := s.ReadN(ctx, 2) - is.NoErr(err) - is.Equal(2, len(recs)) - - // Verify snapshot record no. 1 - err = s.Ack(ctx, recs[0].Position) - is.NoErr(err) - assertRecordOK(is, tableName, recs[0], 1, false) - // Verify snapshot record no. 2 - err = s.Ack(ctx, recs[1].Position) - is.NoErr(err) - assertRecordOK(is, tableName, recs[1], 2, true) + // Read, ack, and assert the snapshot record is OK + rec := readAndAck(ctx, is, s) + assertRecordOK(is, tableName, rec, 1, notNullOnly) // Write the CDC data - insertRow(ctx, t, tableName, 3, true) - insertRow(ctx, t, tableName, 4, false) + insertRow(ctx, is, conn, tableName, 2, notNullOnly) - // Read, ack, and verify CDC record no. 1 - recs, err = s.ReadN(ctx, 1) + // Read, ack, and verify the CDC record + rec = readAndAck(ctx, is, s) + assertRecordOK(is, tableName, rec, 2, notNullOnly) + + deleteRow(ctx, is, conn, tableName, 2) + rec = readAndAck(ctx, is, s) + is.Equal(opencdc.OperationDelete, rec.Operation) +} + +func readAndAck(ctx context.Context, is *is.I, s sdk.Source) opencdc.Record { + recs, err := s.ReadN(ctx, 1) is.NoErr(err) is.Equal(1, len(recs)) - // record no. 3 has the NOT NULL columns only - assertRecordOK(is, tableName, recs[0], 3, true) - // Read, ack, and verify CDC record no. 1 - recs, err = s.ReadN(ctx, 1) + err = s.Ack(ctx, recs[0].Position) is.NoErr(err) - is.Equal(1, len(recs)) - // record no. 3 has the NOT NULL columns only - assertRecordOK(is, tableName, recs[0], 4, false) + + return recs[0] } func createTableWithManyTypes(ctx context.Context, t *testing.T) string { @@ -173,10 +170,7 @@ func createTableWithManyTypes(ctx context.Context, t *testing.T) string { // insertRow inserts a row using the values provided by generatePayloadData. // if notNullOnly is true, only NOT NULL columns are inserted. -func insertRow(ctx context.Context, t *testing.T, table string, rowNumber int, notNullOnly bool) { - is := is.New(t) - conn := test.ConnectSimple(ctx, t, test.RepmgrConnString) - +func insertRow(ctx context.Context, is *is.I, conn *pgx.Conn, table string, rowNumber int, notNullOnly bool) { rec := generatePayloadData(rowNumber, false) var columns []string @@ -213,6 +207,17 @@ func insertRow(ctx context.Context, t *testing.T, table string, rowNumber int, n is.NoErr(err) } +func deleteRow(ctx context.Context, is *is.I, conn *pgx.Conn, table string, rowNumber int) { + query, args, err := squirrel.Delete(internal.WrapSQLIdent(table)). + Where(squirrel.Eq{"id": rowNumber}). + PlaceholderFormat(squirrel.Dollar). + ToSql() + is.NoErr(err) + + _, err = conn.Exec(ctx, query, args...) + is.NoErr(err) +} + func generatePayloadData(id int, notNullOnly bool) opencdc.StructuredData { // Add a time zone offset rowTS := assert(time.Parse(time.RFC3339, fmt.Sprintf("2022-01-21T17:04:05+%02d:00", id))) From 59a0f5d26aa48d34bc2ae45e73fbc0ddbab3424a Mon Sep 17 00:00:00 2001 From: Haris Osmanagic Date: Fri, 20 Jun 2025 14:59:33 +0200 Subject: [PATCH 57/60] refactor integration test, add note about delete test --- source_integration_test.go | 91 +++++++++++++++++++++++++++++++------- 1 file changed, 75 insertions(+), 16 deletions(-) diff --git a/source_integration_test.go b/source_integration_test.go index 1b23231..fabb89a 100644 --- a/source_integration_test.go +++ b/source_integration_test.go @@ -38,16 +38,68 @@ import ( "github.com/shopspring/decimal" ) -// todo test updates and deletes, for not null and nullable values -func TestSource_ReadN_Snapshot_CDC(t *testing.T) { +type readTestCase struct { + name string + notNullOnly bool + snapshot bool + cdc bool + opDelete bool +} + +func TestSource_ReadN(t *testing.T) { + testCases := []readTestCase{ + { + name: "snapshot not only only", + notNullOnly: true, + snapshot: true, + }, + { + name: "snapshot with nullable values", + notNullOnly: false, + snapshot: true, + }, + { + name: "cdc not only only", + notNullOnly: true, + cdc: true, + }, + { + name: "cdc with nullable values", + notNullOnly: false, + cdc: true, + }, + + { + name: "delete cdc data not only only", + notNullOnly: true, + cdc: true, + opDelete: true, + }, + { + name: "delete cdc data with nullable values", + notNullOnly: false, + cdc: true, + opDelete: true, + }, + } + + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + runReadTest(t, tc) + }) + } +} + +func runReadTest(t *testing.T, tc readTestCase) { is := is.New(t) ctx := test.Context(t) - notNullOnly := false conn := test.ConnectSimple(ctx, t, test.RepmgrConnString) tableName := createTableWithManyTypes(ctx, t) - // Write the snapshot data - insertRow(ctx, is, conn, tableName, 1, notNullOnly) + + if tc.snapshot { + insertRow(ctx, is, conn, tableName, 1, tc.notNullOnly) + } slotName := "conduitslot1" publicationName := "conduitpub1" @@ -79,20 +131,27 @@ func TestSource_ReadN_Snapshot_CDC(t *testing.T) { is.NoErr(s.Teardown(ctx)) }) - // Read, ack, and assert the snapshot record is OK - rec := readAndAck(ctx, is, s) - assertRecordOK(is, tableName, rec, 1, notNullOnly) + if tc.snapshot { + // Read, ack, and assert the snapshot record is OK + rec := readAndAck(ctx, is, s) + assertRecordOK(is, tableName, rec, 1, tc.notNullOnly) + } - // Write the CDC data - insertRow(ctx, is, conn, tableName, 2, notNullOnly) + if tc.cdc { + insertRow(ctx, is, conn, tableName, 1, tc.notNullOnly) - // Read, ack, and verify the CDC record - rec = readAndAck(ctx, is, s) - assertRecordOK(is, tableName, rec, 2, notNullOnly) + // Read, ack, and verify the CDC record + rec := readAndAck(ctx, is, s) + assertRecordOK(is, tableName, rec, 1, tc.notNullOnly) + } - deleteRow(ctx, is, conn, tableName, 2) - rec = readAndAck(ctx, is, s) - is.Equal(opencdc.OperationDelete, rec.Operation) + if tc.opDelete { + // https://github.com/ConduitIO/conduit-connector-postgres/issues/301 + t.Skip("Skipping delete test, see GitHub issue ") + deleteRow(ctx, is, conn, tableName, 1) + rec := readAndAck(ctx, is, s) + is.Equal(opencdc.OperationDelete, rec.Operation) + } } func readAndAck(ctx context.Context, is *is.I, s sdk.Source) opencdc.Record { From dc9f97d1ceee2a326f78f11678559451be76c647 Mon Sep 17 00:00:00 2001 From: Haris Osmanagic Date: Fri, 20 Jun 2025 15:02:42 +0200 Subject: [PATCH 58/60] refactor integration test, add note about delete test --- source_integration_test.go | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/source_integration_test.go b/source_integration_test.go index fabb89a..0900fc1 100644 --- a/source_integration_test.go +++ b/source_integration_test.go @@ -70,13 +70,13 @@ func TestSource_ReadN(t *testing.T) { }, { - name: "delete cdc data not only only", + name: "delete cdc not only only", notNullOnly: true, cdc: true, opDelete: true, }, { - name: "delete cdc data with nullable values", + name: "delete cdc nullable", notNullOnly: false, cdc: true, opDelete: true, @@ -91,6 +91,10 @@ func TestSource_ReadN(t *testing.T) { } func runReadTest(t *testing.T, tc readTestCase) { + if tc.opDelete { + t.Skip("Skipping delete test, see https://github.com/ConduitIO/conduit-connector-postgres/issues/301") + } + is := is.New(t) ctx := test.Context(t) conn := test.ConnectSimple(ctx, t, test.RepmgrConnString) @@ -146,8 +150,6 @@ func runReadTest(t *testing.T, tc readTestCase) { } if tc.opDelete { - // https://github.com/ConduitIO/conduit-connector-postgres/issues/301 - t.Skip("Skipping delete test, see GitHub issue ") deleteRow(ctx, is, conn, tableName, 1) rec := readAndAck(ctx, is, s) is.Equal(opencdc.OperationDelete, rec.Operation) From 09ac652a57a2d0913ff469bbd69aabef4abd86f1 Mon Sep 17 00:00:00 2001 From: Haris Osmanagic Date: Fri, 20 Jun 2025 15:19:55 +0200 Subject: [PATCH 59/60] refactor tests --- source_integration_test.go | 182 ++++++++++++++++++++++--------------- 1 file changed, 110 insertions(+), 72 deletions(-) diff --git a/source_integration_test.go b/source_integration_test.go index 0900fc1..4f98034 100644 --- a/source_integration_test.go +++ b/source_integration_test.go @@ -38,76 +38,140 @@ import ( "github.com/shopspring/decimal" ) -type readTestCase struct { - name string - notNullOnly bool - snapshot bool - cdc bool - opDelete bool -} +var ( + slotName = "conduitslot1" + publicationName = "conduitpub1" +) -func TestSource_ReadN(t *testing.T) { - testCases := []readTestCase{ - { - name: "snapshot not only only", - notNullOnly: true, - snapshot: true, - }, +func TestSource_ReadN_Snapshot(t *testing.T) { + testCases := []struct { + name string + notNullOnly bool + }{ { - name: "snapshot with nullable values", + name: "with null columns", notNullOnly: false, - snapshot: true, }, { - name: "cdc not only only", + name: "not only only", notNullOnly: true, - cdc: true, }, + } + + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + is := is.New(t) + ctx := test.Context(t) + conn := test.ConnectSimple(ctx, t, test.RepmgrConnString) + + tableName := createTableWithManyTypes(ctx, t) + insertRow(ctx, is, conn, tableName, 1, tc.notNullOnly) + + s := openSource(ctx, is, tableName) + t.Cleanup(func() { + is.NoErr(logrepl.Cleanup(context.Background(), logrepl.CleanupConfig{ + URL: test.RepmgrConnString, + SlotName: slotName, + PublicationName: publicationName, + })) + is.NoErr(s.Teardown(ctx)) + }) + + // Read, ack, and assert the snapshot record is OK + rec := readAndAck(ctx, is, s) + assertRecordOK(is, tableName, rec, 1, tc.notNullOnly) + }) + } +} + +func TestSource_ReadN_CDC(t *testing.T) { + testCases := []struct { + name string + notNullOnly bool + }{ { - name: "cdc with nullable values", + name: "with null columns", notNullOnly: false, - cdc: true, }, - { - name: "delete cdc not only only", + name: "not only only", notNullOnly: true, - cdc: true, - opDelete: true, - }, - { - name: "delete cdc nullable", - notNullOnly: false, - cdc: true, - opDelete: true, }, } for _, tc := range testCases { t.Run(tc.name, func(t *testing.T) { - runReadTest(t, tc) + is := is.New(t) + ctx := test.Context(t) + conn := test.ConnectSimple(ctx, t, test.RepmgrConnString) + + tableName := createTableWithManyTypes(ctx, t) + + s := openSource(ctx, is, tableName) + t.Cleanup(func() { + is.NoErr(logrepl.Cleanup(context.Background(), logrepl.CleanupConfig{ + URL: test.RepmgrConnString, + SlotName: slotName, + PublicationName: publicationName, + })) + is.NoErr(s.Teardown(ctx)) + }) + + insertRow(ctx, is, conn, tableName, 1, tc.notNullOnly) + // Read, ack, and assert the CDC record is OK + rec := readAndAck(ctx, is, s) + assertRecordOK(is, tableName, rec, 1, tc.notNullOnly) }) } } -func runReadTest(t *testing.T, tc readTestCase) { - if tc.opDelete { - t.Skip("Skipping delete test, see https://github.com/ConduitIO/conduit-connector-postgres/issues/301") +func TestSource_ReadN_Delete(t *testing.T) { + t.Skip("Skipping until this issue is resolved: https://github.com/ConduitIO/conduit-connector-postgres/issues/301") + testCases := []struct { + name string + notNullOnly bool + }{ + { + name: "with null columns", + notNullOnly: false, + }, + { + name: "not only only", + notNullOnly: true, + }, } - is := is.New(t) - ctx := test.Context(t) - conn := test.ConnectSimple(ctx, t, test.RepmgrConnString) - - tableName := createTableWithManyTypes(ctx, t) - - if tc.snapshot { - insertRow(ctx, is, conn, tableName, 1, tc.notNullOnly) + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + is := is.New(t) + ctx := test.Context(t) + conn := test.ConnectSimple(ctx, t, test.RepmgrConnString) + + tableName := createTableWithManyTypes(ctx, t) + + s := openSource(ctx, is, tableName) + t.Cleanup(func() { + is.NoErr(logrepl.Cleanup(context.Background(), logrepl.CleanupConfig{ + URL: test.RepmgrConnString, + SlotName: slotName, + PublicationName: publicationName, + })) + is.NoErr(s.Teardown(ctx)) + }) + + insertRow(ctx, is, conn, tableName, 1, tc.notNullOnly) + // Read, ack, and assert the CDC record is OK + cdcRec := readAndAck(ctx, is, s) + assertRecordOK(is, tableName, cdcRec, 1, tc.notNullOnly) + + deleteRow(ctx, is, conn, tableName, 1) + deleteRec := readAndAck(ctx, is, s) + is.Equal(opencdc.OperationDelete, deleteRec.Operation) + }) } +} - slotName := "conduitslot1" - publicationName := "conduitpub1" - +func openSource(ctx context.Context, is *is.I, tableName string) sdk.Source { s := NewSource() err := sdk.Util.ParseConfig( ctx, @@ -126,34 +190,8 @@ func runReadTest(t *testing.T, tc readTestCase) { err = s.Open(ctx, nil) is.NoErr(err) - t.Cleanup(func() { - is.NoErr(logrepl.Cleanup(context.Background(), logrepl.CleanupConfig{ - URL: test.RepmgrConnString, - SlotName: slotName, - PublicationName: publicationName, - })) - is.NoErr(s.Teardown(ctx)) - }) - if tc.snapshot { - // Read, ack, and assert the snapshot record is OK - rec := readAndAck(ctx, is, s) - assertRecordOK(is, tableName, rec, 1, tc.notNullOnly) - } - - if tc.cdc { - insertRow(ctx, is, conn, tableName, 1, tc.notNullOnly) - - // Read, ack, and verify the CDC record - rec := readAndAck(ctx, is, s) - assertRecordOK(is, tableName, rec, 1, tc.notNullOnly) - } - - if tc.opDelete { - deleteRow(ctx, is, conn, tableName, 1) - rec := readAndAck(ctx, is, s) - is.Equal(opencdc.OperationDelete, rec.Operation) - } + return s } func readAndAck(ctx context.Context, is *is.I, s sdk.Source) opencdc.Record { From 2826c9163da8d48963ca20569cf7a6e161af1b6a Mon Sep 17 00:00:00 2001 From: Haris Osmanagic Date: Fri, 20 Jun 2025 15:28:32 +0200 Subject: [PATCH 60/60] rename file --- destination.go | 4 +-- .../{db_info.go => numeric_scale_info.go} | 26 +++++++++---------- 2 files changed, 15 insertions(+), 15 deletions(-) rename internal/{db_info.go => numeric_scale_info.go} (75%) diff --git a/destination.go b/destination.go index 0e6e4a3..02adbe9 100644 --- a/destination.go +++ b/destination.go @@ -39,7 +39,7 @@ type Destination struct { getTableName destination.TableFn conn *pgx.Conn - dbInfo *internal.DbInfo + dbInfo *internal.NumericScaleInfo stmtBuilder sq.StatementBuilderType } @@ -347,7 +347,7 @@ func (d *Destination) formatBigRat(ctx context.Context, table string, column str // we need to get the scale of the column so we that we can properly // round the result of dividing the input big.Rat's numerator and denominator. - scale, err := d.dbInfo.GetNumericColumnScale(ctx, table, column) + scale, err := d.dbInfo.Get(ctx, table, column) if err != nil { return "", fmt.Errorf("failed getting scale of numeric column: %w", err) } diff --git a/internal/db_info.go b/internal/numeric_scale_info.go similarity index 75% rename from internal/db_info.go rename to internal/numeric_scale_info.go index 30394bd..d7f12fd 100644 --- a/internal/db_info.go +++ b/internal/numeric_scale_info.go @@ -22,28 +22,28 @@ import ( "github.com/jackc/pgx/v5" ) -// DbInfo provides information about tables in a database. -type DbInfo struct { +// NumericScaleInfo provides information about the scale of numeric columns +// in a database. +type NumericScaleInfo struct { conn *pgx.Conn cache map[string]*tableCache } // tableCache stores information about a table. -// The information is cached and refreshed every 'cacheExpiration'. type tableCache struct { columns map[string]int } -func NewDbInfo(conn *pgx.Conn) *DbInfo { - return &DbInfo{ +func NewDbInfo(conn *pgx.Conn) *NumericScaleInfo { + return &NumericScaleInfo{ conn: conn, cache: map[string]*tableCache{}, } } -func (d *DbInfo) GetNumericColumnScale(ctx context.Context, table string, column string) (int, error) { - // Check if table exists in cache and is not expired - tableInfo, ok := d.cache[table] +func (i *NumericScaleInfo) Get(ctx context.Context, table string, column string) (int, error) { + // Check if table exists in cache + tableInfo, ok := i.cache[table] if ok { scale, ok := tableInfo.columns[column] if ok { @@ -51,23 +51,23 @@ func (d *DbInfo) GetNumericColumnScale(ctx context.Context, table string, column } } else { // Table info has expired, refresh the cache - d.cache[table] = &tableCache{ + i.cache[table] = &tableCache{ columns: map[string]int{}, } } // Fetch scale from database - scale, err := d.numericScaleFromDb(ctx, table, column) + scale, err := i.fetchFromDB(ctx, table, column) if err != nil { return 0, err } - d.cache[table].columns[column] = scale + i.cache[table].columns[column] = scale return scale, nil } -func (d *DbInfo) numericScaleFromDb(ctx context.Context, table string, column string) (int, error) { +func (i *NumericScaleInfo) fetchFromDB(ctx context.Context, table string, column string) (int, error) { // Query to get the column type and numeric scale query := ` SELECT @@ -83,7 +83,7 @@ func (d *DbInfo) numericScaleFromDb(ctx context.Context, table string, column st var dataType string var numericScale *int - err := d.conn.QueryRow(ctx, query, table, column).Scan(&dataType, &numericScale) + err := i.conn.QueryRow(ctx, query, table, column).Scan(&dataType, &numericScale) if err != nil { if errors.Is(err, pgx.ErrNoRows) { return 0, fmt.Errorf("column %s not found in table %s", column, table)