Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

vstreamclient: framework for robust + simple usage #17222

Open
wants to merge 2 commits into
base: main
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
158 changes: 158 additions & 0 deletions go/test/endtoend/vreplication/vstreamclient_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,158 @@
package vreplication

import (
"context"
"fmt"
"reflect"
"slices"
"testing"
"time"

"github.com/stretchr/testify/require"

"vitess.io/vitess/go/vt/log"
binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata"
"vitess.io/vitess/go/vt/vstreamclient"
"vitess.io/vitess/go/vt/vtgate/vtgateconn"
)

// Customer is the concrete type that will be built from the stream
type Customer struct {
ID int64 `vstream:"customer_id"`
Email string `vstream:"email"`
DeletedAt time.Time `vstream:"-"`
}

// To run the tests, this currently expects the local example to be running
// ./101_initial_cluster.sh; mysql < ../common/insert_commerce_data.sql; ./201_customer_tablets.sh; ./202_move_tables.sh; ./203_switch_reads.sh; ./204_switch_writes.sh; ./205_clean_commerce.sh; ./301_customer_sharded.sh; ./302_new_shards.sh; ./303_reshard.sh; ./304_switch_reads.sh; ./305_switch_writes.sh; ./306_down_shard_0.sh; ./307_delete_shard_0.sh
func TestVStreamClient(t *testing.T) {
vc = NewVitessCluster(t, nil)
defer vc.TearDown()

require.NotNil(t, vc)
defaultReplicas = 2
defaultRdonly = 0

defaultCell := vc.Cells[vc.CellNames[0]]
vc.AddKeyspace(t, []*Cell{defaultCell}, "product", "0", initialProductVSchema, initialProductSchema, defaultReplicas, defaultRdonly, 100, nil)
verifyClusterHealth(t, vc)
insertInitialData(t)

ctx := context.Background()
conn, err := vtgateconn.Dial(ctx, fmt.Sprintf("%s:%d", vc.ClusterConfig.hostname, vc.ClusterConfig.vtgateGrpcPort))
if err != nil {
log.Fatal(err)
}
defer conn.Close()

flushCount := 0
gotCustomers := make([]*Customer, 0)

tables := []vstreamclient.TableConfig{{
Keyspace: "customer",
Table: "customer",
MaxRowsPerFlush: 7,
DataType: &Customer{},
FlushFn: func(ctx context.Context, rows []vstreamclient.Row, meta vstreamclient.FlushMeta) error {
flushCount++

fmt.Printf("upserting %d customers\n", len(rows))
for i, row := range rows {
switch {
// delete event
case row.RowChange.After == nil:
customer := row.Data.(*Customer)
customer.DeletedAt = time.Now()

gotCustomers = append(gotCustomers, customer)
fmt.Printf("deleting customer %d: %v\n", i, row)

// insert event
case row.RowChange.Before == nil:
gotCustomers = append(gotCustomers, row.Data.(*Customer))
fmt.Printf("inserting customer %d: %v\n", i, row)

// update event
case row.RowChange.Before != nil:
gotCustomers = append(gotCustomers, row.Data.(*Customer))
fmt.Printf("updating customer %d: %v\n", i, row)
}
}

// a real implementation would do something more meaningful here. For a data warehouse type workload,
// it would probably look like streaming rows into the data warehouse, or for more complex versions,
// write newline delimited json or a parquet file to object storage, then trigger a load job.
return nil
},
}}

t.Run("first vstream run, should succeed", func(t *testing.T) {
ctx, cancel := context.WithTimeout(context.Background(), 2*time.Second)
defer cancel()

vstreamClient, err := vstreamclient.New(ctx, "bob", conn, tables,
vstreamclient.WithMinFlushDuration(500*time.Millisecond),
vstreamclient.WithHeartbeatSeconds(1),
vstreamclient.WithStateTable("commerce", "vstreams"),
vstreamclient.WithEventFunc(func(ctx context.Context, ev *binlogdatapb.VEvent) error {
fmt.Printf("** FIELD EVENT: %v\n", ev)
return nil
}, binlogdatapb.VEventType_FIELD),
)
if err != nil {
t.Fatalf("failed to create VStreamClient: %v", err)
}

err = vstreamClient.Run(ctx)
if err != nil && ctx.Err() == nil {
t.Fatalf("failed to run vstreamclient: %v", err)
}

slices.SortFunc(gotCustomers, func(a, b *Customer) int {
return int(a.ID - b.ID)
})

wantCustomers := []*Customer{
{ID: 1, Email: "[email protected]"},
{ID: 2, Email: "[email protected]"},
{ID: 3, Email: "[email protected]"},
{ID: 4, Email: "[email protected]"},
{ID: 5, Email: "[email protected]"},
}

fmt.Printf("got %d customers | flushed %d times\n", len(gotCustomers), flushCount)
if !reflect.DeepEqual(gotCustomers, wantCustomers) {
t.Fatalf("got %d customers, want %d", len(gotCustomers), len(wantCustomers))
}
})

// this should fail because we're going to restart the stream, but with an additional table
t.Run("second vstream run, should fail", func(t *testing.T) {
withAdditionalTable := append(tables, vstreamclient.TableConfig{
Keyspace: "customer",
Table: "corder",
DataType: &Customer{},
})

ctx, cancel := context.WithTimeout(context.Background(), 2*time.Second)
defer cancel()

_, err := vstreamclient.New(ctx, "bob", conn, withAdditionalTable,
vstreamclient.WithStateTable("commerce", "vstreams"),
)
if err == nil {
t.Fatalf("expected VStreamClient error, got nil")
} else if err.Error() != "vstreamclient: provided tables do not match stored tables" {
t.Fatalf("expected error 'vstreamclient: provided tables do not match stored tables', got '%v'", err)
}
})
}

func getConn(t *testing.T, ctx context.Context) *vtgateconn.VTGateConn {
t.Helper()
conn, err := vtgateconn.Dial(ctx, "localhost:15991")
if err != nil {
t.Fatal(err)
}
return conn
}
84 changes: 84 additions & 0 deletions go/vt/vstreamclient/convert.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,84 @@
package vstreamclient

import (
"fmt"
"reflect"
"time"

"vitess.io/vitess/go/sqltypes"
binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata"
querypb "vitess.io/vitess/go/vt/proto/query"
)

// VStreamScanner allows for custom scan implementations
type VStreamScanner interface {
VStreamScan(fields []*querypb.Field, row []sqltypes.Value, rowEvent *binlogdatapb.RowEvent, rowChange *binlogdatapb.RowChange) error
}

// copyRowToStruct builds a customer from a row event
// TODO: this is very rudimentary mapping that only works for top-level fields
func copyRowToStruct(shard shardConfig, row []sqltypes.Value, vPtr reflect.Value) error {
for fieldName, m := range shard.fieldMap {
structField := reflect.Indirect(vPtr).FieldByIndex(m.structIndex)

switch m.kind {
case reflect.Bool:
rowVal, err := row[m.rowIndex].ToBool()
if err != nil {
return fmt.Errorf("error converting row value to bool for field %s: %w", fieldName, err)
}
structField.SetBool(rowVal)

case reflect.Int, reflect.Int8, reflect.Int16, reflect.Int32, reflect.Int64:
rowVal, err := row[m.rowIndex].ToInt64()
if err != nil {
return fmt.Errorf("error converting row value to int64 for field %s: %w", fieldName, err)
}
structField.SetInt(rowVal)

case reflect.Uint, reflect.Uint8, reflect.Uint16, reflect.Uint32, reflect.Uint64:
rowVal, err := row[m.rowIndex].ToUint64()
if err != nil {
return fmt.Errorf("error converting row value to uint64 for field %s: %w", fieldName, err)
}
structField.SetUint(rowVal)

case reflect.Float32, reflect.Float64:
rowVal, err := row[m.rowIndex].ToFloat64()
if err != nil {
return fmt.Errorf("error converting row value to float64 for field %s: %w", fieldName, err)
}
structField.SetFloat(rowVal)

case reflect.String:
rowVal := row[m.rowIndex].ToString()
structField.SetString(rowVal)

case reflect.Struct:
switch m.structType.(type) {
case time.Time, *time.Time:
rowVal, err := row[m.rowIndex].ToTime()
if err != nil {
return fmt.Errorf("error converting row value to time.Time for field %s: %w", fieldName, err)
}
structField.Set(reflect.ValueOf(rowVal))
}

case reflect.Pointer,
reflect.Slice,
reflect.Array,
reflect.Invalid,
reflect.Uintptr,
reflect.Complex64,
reflect.Complex128,
reflect.Chan,
reflect.Func,
reflect.Interface,
reflect.Map,
reflect.UnsafePointer:
return fmt.Errorf("vstreamclient: unsupported field type: %s", m.kind.String())
}
}

return nil
}
115 changes: 115 additions & 0 deletions go/vt/vstreamclient/options.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,115 @@
package vstreamclient

import (
"fmt"
"time"

"vitess.io/vitess/go/sqlescape"
binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata"
vtgatepb "vitess.io/vitess/go/vt/proto/vtgate"
)

var (
// DefaultMinFlushDuration is the default minimum duration between flushes, used if not explicitly
// set using WithMinFlushDuration. This can be safely modified if needed before calling New.
DefaultMinFlushDuration = 5 * time.Second

// DefaultMaxRowsPerFlush is the default number of rows to buffer per table, used if not explicitly
// set in the table configuration. This same number is also used to chunk rows when calling flush.
// This can be safely modified if needed before calling New.
DefaultMaxRowsPerFlush = 1000
)

// Option is a function that can be used to configure a VStreamClient
type Option func(v *VStreamClient) error

// WithMinFlushDuration sets the minimum duration between flushes. This is useful for ensuring that data
// isn't flushed too often, which can be inefficient. The default is 30 seconds.
func WithMinFlushDuration(d time.Duration) Option {
return func(v *VStreamClient) error {
if d <= 0 {
return fmt.Errorf("vstreamclient: minimum flush duration must be positive, got %s", d.String())
}

v.minFlushDuration = d
return nil
}
}

func WithHeartbeatSeconds(seconds int) Option {
return func(v *VStreamClient) error {
if seconds <= 0 {
return fmt.Errorf("vstreamclient: heartbeat seconds must be positive, got %d", seconds)
}

v.heartbeatSeconds = seconds
return nil
}
}

func WithStateTable(keyspace, table string) Option {
return func(v *VStreamClient) error {
shards, ok := v.shardsByKeyspace[keyspace]
if !ok {
return fmt.Errorf("vstreamclient: keyspace %s not found", keyspace)
}

// this could allow for shard pinning, but we can support that if it becomes useful
if len(shards) > 1 {
return fmt.Errorf("vstreamclient: keyspace %s is sharded, only unsharded keyspaces are supported", keyspace)
}

v.vgtidStateKeyspace = sqlescape.EscapeID(keyspace)
v.vgtidStateTable = sqlescape.EscapeID(table)
return nil
}
}

// DefaultFlags returns a default set of flags for a VStreamClient, safe to use in most cases, but can be customized
func DefaultFlags() *vtgatepb.VStreamFlags {
return &vtgatepb.VStreamFlags{
HeartbeatInterval: 1,
}
}

// WithFlags lets you manually control all the flag options, instead of using helper functions
func WithFlags(flags *vtgatepb.VStreamFlags) Option {
return func(v *VStreamClient) error {
v.flags = flags
return nil
}
}

// WithEventFunc provides for custom event handling functions for specific event types. Only one function
// can be registered per event type, and it is called before the default event handling function. Returning
// an error from the custom function will exit the stream before the default function is called.
func WithEventFunc(fn EventFunc, eventTypes ...binlogdatapb.VEventType) Option {
return func(v *VStreamClient) error {
if len(eventTypes) == 0 {
return fmt.Errorf("vstreamclient: no event types provided")
}

if v.eventFuncs == nil {
v.eventFuncs = make(map[binlogdatapb.VEventType]EventFunc)
}

for _, eventType := range eventTypes {
if _, ok := v.eventFuncs[eventType]; ok {
return fmt.Errorf("vstreamclient: event type %s already has a function", eventType.String())
}

v.eventFuncs[eventType] = fn
}

return nil
}
}

// WithStartingVGtid sets the starting VGtid for the VStreamClient. This is useful for resuming a stream from a
// specific point, vs what might be stored in the state table.
func WithStartingVGtid(vgtid *binlogdatapb.VGtid) Option {
return func(v *VStreamClient) error {
v.latestVgtid = vgtid
return nil
}
}
Loading
Loading