From d5efe8eed32db0ae6516cb01fa4d61f5df54bafb Mon Sep 17 00:00:00 2001 From: Matt Lord Date: Sun, 1 Oct 2023 17:21:41 -0400 Subject: [PATCH] VReplication: Ensure that RowStreamer uses optimal index when possible (#13893) Signed-off-by: Matt Lord --- go/vt/mysqlctl/fakemysqldaemon.go | 164 +++++---- go/vt/mysqlctl/mysql_daemon.go | 2 +- go/vt/mysqlctl/schema.go | 39 +- go/vt/proto/binlogdata/binlogdata.pb.go | 339 +++++++++--------- .../proto/binlogdata/binlogdata_vtproto.pb.go | 46 ++- .../tabletmanager/vreplication/vreplicator.go | 8 +- .../vreplication/vreplicator_test.go | 61 ++-- .../vttablet/tabletserver/vstreamer/engine.go | 5 +- .../tabletserver/vstreamer/rowstreamer.go | 34 +- .../vstreamer/rowstreamer_test.go | 42 ++- .../vstreamer/uvstreamer_flaky_test.go | 2 +- proto/binlogdata.proto | 4 + web/vtadmin/src/proto/vtadmin.d.ts | 6 + web/vtadmin/src/proto/vtadmin.js | 26 +- 14 files changed, 464 insertions(+), 314 deletions(-) diff --git a/go/vt/mysqlctl/fakemysqldaemon.go b/go/vt/mysqlctl/fakemysqldaemon.go index 39ecca84156..de16e100fa8 100644 --- a/go/vt/mysqlctl/fakemysqldaemon.go +++ b/go/vt/mysqlctl/fakemysqldaemon.go @@ -47,101 +47,108 @@ type FakeMysqlDaemon struct { // appPool is set if db is set. appPool *dbconnpool.ConnectionPool - // Running is used by Start / Shutdown + // Running is used by Start / Shutdown. Running bool - // StartupTime is used to simulate mysqlds that take some time to respond - // to a "start" command. It is used by Start. + // StartupTime is used to simulate mysqlds that take some time to + // respond to a "start" command. It is used by Start. StartupTime time.Duration - // ShutdownTime is used to simulate mysqlds that take some time to respond - // to a "stop" request (i.e. a wedged systemd unit). It is used by Shutdown. + // ShutdownTime is used to simulate mysqlds that take some time + // to respond to a "stop" request (i.e. a wedged systemd unit). + // It is used by Shutdown. ShutdownTime time.Duration // MysqlPort will be returned by GetMysqlPort(). Set to -1 to // return an error. MysqlPort atomic.Int32 - // Replicating is updated when calling StartReplication / StopReplication - // (it is not used at all when calling ReplicationStatus, it is the - // test owner responsibility to have these two match) + // Replicating is updated when calling StartReplication / + // StopReplication (it is not used at all when calling + // ReplicationStatus, it is the test owner responsibility + //to have these two match) Replicating bool - // IOThreadRunning is always true except in one testcase - // where we want to test error handling during SetReplicationSource + // IOThreadRunning is always true except in one testcase where + // we want to test error handling during SetReplicationSource. IOThreadRunning bool // CurrentPrimaryPosition is returned by PrimaryPosition - // and ReplicationStatus + // and ReplicationStatus. CurrentPrimaryPosition replication.Position - // CurrentSourceFilePosition is used to determine the executed file based positioning of the replication source. + // CurrentSourceFilePosition is used to determine the executed + // file based positioning of the replication source. CurrentSourceFilePosition replication.Position - // ReplicationStatusError is used by ReplicationStatus + // ReplicationStatusError is used by ReplicationStatus. ReplicationStatusError error - // StartReplicationError is used by StartReplication + // StartReplicationError is used by StartReplication. StartReplicationError error - // PromoteLag is the time for which Promote will stall + // PromoteLag is the time for which Promote will stall. PromoteLag time.Duration - // PrimaryStatusError is used by PrimaryStatus + // PrimaryStatusError is used by PrimaryStatus. PrimaryStatusError error - // CurrentSourceHost is returned by ReplicationStatus + // CurrentSourceHost is returned by ReplicationStatus. CurrentSourceHost string - // CurrentSourcePort is returned by ReplicationStatus + // CurrentSourcePort is returned by ReplicationStatus. CurrentSourcePort int32 - // ReplicationLagSeconds is returned by ReplicationStatus + // ReplicationLagSeconds is returned by ReplicationStatus. ReplicationLagSeconds uint32 - // ReadOnly is the current value of the flag + // ReadOnly is the current value of the flag. ReadOnly bool - // SuperReadOnly is the current value of the flag + // SuperReadOnly is the current value of the flag. SuperReadOnly atomic.Bool - // SetReplicationPositionPos is matched against the input of SetReplicationPosition. - // If it doesn't match, SetReplicationPosition will return an error. + // SetReplicationPositionPos is matched against the input of + // SetReplicationPosition. If it doesn't match, SetReplicationPosition + // will return an error. SetReplicationPositionPos replication.Position - // StartReplicationUntilAfterPos is matched against the input + // StartReplicationUntilAfterPos is matched against the input. StartReplicationUntilAfterPos replication.Position - // SetReplicationSourceInputs are matched against the input of SetReplicationSource - // (as "%v:%v"). If all of them don't match, SetReplicationSource will return an error. + // SetReplicationSourceInputs are matched against the input of + // SetReplicationSource (as "%v:%v"). If all of them don't + // match, SetReplicationSource will return an error. SetReplicationSourceInputs []string - // SetReplicationSourceError is used by SetReplicationSource + // SetReplicationSourceError is used by SetReplicationSource. SetReplicationSourceError error - // StopReplicationError error is used by StopReplication + // StopReplicationError error is used by StopReplication. StopReplicationError error - // WaitPrimaryPositions is checked by WaitSourcePos, if the value is found - // in it, then the function returns nil, else the function returns an error + // WaitPrimaryPositions is checked by WaitSourcePos, if the value + // is found in it, then the function returns nil, else the + // function returns an error. WaitPrimaryPositions []replication.Position - // PromoteResult is returned by Promote + // PromoteResult is returned by Promote. PromoteResult replication.Position - // PromoteError is used by Promote + // PromoteError is used by Promote. PromoteError error // SchemaFunc provides the return value for GetSchema. - // If not defined, the "Schema" field will be used instead, see below. + // If not defined, the "Schema" field will be used instead, see + // below. SchemaFunc func() (*tabletmanagerdatapb.SchemaDefinition, error) // Schema will be returned by GetSchema. If nil we'll // return an error. Schema *tabletmanagerdatapb.SchemaDefinition - // PreflightSchemaChangeResult will be returned by PreflightSchemaChange. - // If nil we'll return an error. + // PreflightSchemaChangeResult will be returned by + // PreflightSchemaChange. If nil we'll return an error. PreflightSchemaChangeResult []*tabletmanagerdatapb.SchemaChangeResult // ApplySchemaChangeResult will be returned by ApplySchemaChange. @@ -153,14 +160,14 @@ type FakeMysqlDaemon struct { // match, ExecuteSuperQueryList will return an error. // Note each string is just a substring if it begins with SUB, // so we support partial queries (useful when queries contain - // data fields like timestamps) + // data fields like timestamps). ExpectedExecuteSuperQueryList []string - // ExpectedExecuteSuperQueryCurrent is the current index of the queries - // we expect + // ExpectedExecuteSuperQueryCurrent is the current index of the + // queries we expect. ExpectedExecuteSuperQueryCurrent int - // FetchSuperQueryResults is used by FetchSuperQuery + // FetchSuperQueryResults is used by FetchSuperQuery. FetchSuperQueryMap map[string]*sqltypes.Result // SemiSyncPrimaryEnabled represents the state of rpl_semi_sync_master_enabled. @@ -168,8 +175,9 @@ type FakeMysqlDaemon struct { // SemiSyncReplicaEnabled represents the state of rpl_semi_sync_slave_enabled. SemiSyncReplicaEnabled bool - // TimeoutHook is a func that can be called at the beginning of any method to fake a timeout. - // all a test needs to do is make it { return context.DeadlineExceeded } + // TimeoutHook is a func that can be called at the beginning of + // any method to fake a timeout. + // All a test needs to do is make it { return context.DeadlineExceeded }. TimeoutHook func() error // Version is the version that will be returned by GetVersionString. @@ -193,7 +201,7 @@ func NewFakeMysqlDaemon(db *fakesqldb.DB) *FakeMysqlDaemon { return result } -// Start is part of the MysqlDaemon interface +// Start is part of the MysqlDaemon interface. func (fmd *FakeMysqlDaemon) Start(ctx context.Context, cnf *Mycnf, mysqldArgs ...string) error { if fmd.Running { return fmt.Errorf("fake mysql daemon already running") @@ -211,7 +219,7 @@ func (fmd *FakeMysqlDaemon) Start(ctx context.Context, cnf *Mycnf, mysqldArgs .. return nil } -// Shutdown is part of the MysqlDaemon interface +// Shutdown is part of the MysqlDaemon interface. func (fmd *FakeMysqlDaemon) Shutdown(ctx context.Context, cnf *Mycnf, waitForMysqld bool) error { if !fmd.Running { return fmt.Errorf("fake mysql daemon not running") @@ -229,27 +237,27 @@ func (fmd *FakeMysqlDaemon) Shutdown(ctx context.Context, cnf *Mycnf, waitForMys return nil } -// RunMysqlUpgrade is part of the MysqlDaemon interface +// RunMysqlUpgrade is part of the MysqlDaemon interface. func (fmd *FakeMysqlDaemon) RunMysqlUpgrade(ctx context.Context) error { return nil } -// ApplyBinlogFile is part of the MysqlDaemon interface +// ApplyBinlogFile is part of the MysqlDaemon interface. func (fmd *FakeMysqlDaemon) ApplyBinlogFile(ctx context.Context, req *mysqlctlpb.ApplyBinlogFileRequest) error { return nil } -// ReadBinlogFilesTimestamps is part of the MysqlDaemon interface +// ReadBinlogFilesTimestamps is part of the MysqlDaemon interface. func (fmd *FakeMysqlDaemon) ReadBinlogFilesTimestamps(ctx context.Context, req *mysqlctlpb.ReadBinlogFilesTimestampsRequest) (*mysqlctlpb.ReadBinlogFilesTimestampsResponse, error) { return nil, nil } -// ReinitConfig is part of the MysqlDaemon interface +// ReinitConfig is part of the MysqlDaemon interface. func (fmd *FakeMysqlDaemon) ReinitConfig(ctx context.Context, cnf *Mycnf) error { return nil } -// RefreshConfig is part of the MysqlDaemon interface +// RefreshConfig is part of the MysqlDaemon interface. func (fmd *FakeMysqlDaemon) RefreshConfig(ctx context.Context, cnf *Mycnf) error { return nil } @@ -259,7 +267,7 @@ func (fmd *FakeMysqlDaemon) Wait(ctx context.Context, cnf *Mycnf) error { return nil } -// GetMysqlPort is part of the MysqlDaemon interface +// GetMysqlPort is part of the MysqlDaemon interface. func (fmd *FakeMysqlDaemon) GetMysqlPort() (int32, error) { if fmd.MysqlPort.Load() == -1 { return 0, fmt.Errorf("FakeMysqlDaemon.GetMysqlPort returns an error") @@ -267,24 +275,24 @@ func (fmd *FakeMysqlDaemon) GetMysqlPort() (int32, error) { return fmd.MysqlPort.Load(), nil } -// GetServerID is part of the MysqlDaemon interface +// GetServerID is part of the MysqlDaemon interface. func (fmd *FakeMysqlDaemon) GetServerID(ctx context.Context) (uint32, error) { return 1, nil } -// GetServerUUID is part of the MysqlDaemon interface +// GetServerUUID is part of the MysqlDaemon interface. func (fmd *FakeMysqlDaemon) GetServerUUID(ctx context.Context) (string, error) { return "000000", nil } -// CurrentPrimaryPositionLocked is thread-safe +// CurrentPrimaryPositionLocked is thread-safe. func (fmd *FakeMysqlDaemon) CurrentPrimaryPositionLocked(pos replication.Position) { fmd.mu.Lock() defer fmd.mu.Unlock() fmd.CurrentPrimaryPosition = pos } -// ReplicationStatus is part of the MysqlDaemon interface +// ReplicationStatus is part of the MysqlDaemon interface. func (fmd *FakeMysqlDaemon) ReplicationStatus() (replication.ReplicationStatus, error) { if fmd.ReplicationStatusError != nil { return replication.ReplicationStatus{}, fmd.ReplicationStatusError @@ -296,8 +304,8 @@ func (fmd *FakeMysqlDaemon) ReplicationStatus() (replication.ReplicationStatus, FilePosition: fmd.CurrentSourceFilePosition, RelayLogSourceBinlogEquivalentPosition: fmd.CurrentSourceFilePosition, ReplicationLagSeconds: fmd.ReplicationLagSeconds, - // implemented as AND to avoid changing all tests that were - // previously using Replicating = false + // Implemented as AND to avoid changing all tests that were + // previously using Replicating = false. IOState: replication.ReplicationStatusToState(fmt.Sprintf("%v", fmd.Replicating && fmd.IOThreadRunning)), SQLState: replication.ReplicationStatusToState(fmt.Sprintf("%v", fmd.Replicating)), SourceHost: fmd.CurrentSourceHost, @@ -305,7 +313,7 @@ func (fmd *FakeMysqlDaemon) ReplicationStatus() (replication.ReplicationStatus, }, nil } -// PrimaryStatus is part of the MysqlDaemon interface +// PrimaryStatus is part of the MysqlDaemon interface. func (fmd *FakeMysqlDaemon) PrimaryStatus(ctx context.Context) (replication.PrimaryStatus, error) { if fmd.PrimaryStatusError != nil { return replication.PrimaryStatus{}, fmd.PrimaryStatusError @@ -316,7 +324,7 @@ func (fmd *FakeMysqlDaemon) PrimaryStatus(ctx context.Context) (replication.Prim }, nil } -// GetGTIDPurged is part of the MysqlDaemon interface +// GetGTIDPurged is part of the MysqlDaemon interface. func (fmd *FakeMysqlDaemon) GetGTIDPurged(ctx context.Context) (replication.Position, error) { return replication.Position{}, nil } @@ -370,28 +378,28 @@ func (fmd *FakeMysqlDaemon) GetPreviousGTIDs(ctx context.Context, binlog string) }) } -// PrimaryPosition is part of the MysqlDaemon interface +// PrimaryPosition is part of the MysqlDaemon interface. func (fmd *FakeMysqlDaemon) PrimaryPosition() (replication.Position, error) { return fmd.CurrentPrimaryPosition, nil } -// IsReadOnly is part of the MysqlDaemon interface +// IsReadOnly is part of the MysqlDaemon interface. func (fmd *FakeMysqlDaemon) IsReadOnly() (bool, error) { return fmd.ReadOnly, nil } -// IsSuperReadOnly is part of the MysqlDaemon interface +// IsSuperReadOnly is part of the MysqlDaemon interface. func (fmd *FakeMysqlDaemon) IsSuperReadOnly() (bool, error) { return fmd.SuperReadOnly.Load(), nil } -// SetReadOnly is part of the MysqlDaemon interface +// SetReadOnly is part of the MysqlDaemon interface. func (fmd *FakeMysqlDaemon) SetReadOnly(on bool) error { fmd.ReadOnly = on return nil } -// SetSuperReadOnly is part of the MysqlDaemon interface +// SetSuperReadOnly is part of the MysqlDaemon interface. func (fmd *FakeMysqlDaemon) SetSuperReadOnly(on bool) (ResetSuperReadOnlyFunc, error) { fmd.SuperReadOnly.Store(on) fmd.ReadOnly = on @@ -483,12 +491,12 @@ func (fmd *FakeMysqlDaemon) SetReplicationSource(ctx context.Context, host strin return fmd.ExecuteSuperQueryList(ctx, cmds) } -// WaitForReparentJournal is part of the MysqlDaemon interface +// WaitForReparentJournal is part of the MysqlDaemon interface. func (fmd *FakeMysqlDaemon) WaitForReparentJournal(ctx context.Context, timeCreatedNS int64) error { return nil } -// WaitSourcePos is part of the MysqlDaemon interface +// WaitSourcePos is part of the MysqlDaemon interface. func (fmd *FakeMysqlDaemon) WaitSourcePos(_ context.Context, pos replication.Position) error { if fmd.TimeoutHook != nil { return fmd.TimeoutHook() @@ -501,7 +509,7 @@ func (fmd *FakeMysqlDaemon) WaitSourcePos(_ context.Context, pos replication.Pos return fmt.Errorf("wrong input for WaitSourcePos: expected a value in %v got %v", fmd.WaitPrimaryPositions, pos) } -// Promote is part of the MysqlDaemon interface +// Promote is part of the MysqlDaemon interface. func (fmd *FakeMysqlDaemon) Promote(hookExtraEnv map[string]string) (replication.Position, error) { if fmd.PromoteLag > 0 { time.Sleep(fmd.PromoteLag) @@ -524,8 +532,8 @@ func (fmd *FakeMysqlDaemon) ExecuteSuperQueryList(ctx context.Context, queryList expected := fmd.ExpectedExecuteSuperQueryList[fmd.ExpectedExecuteSuperQueryCurrent] fmd.ExpectedExecuteSuperQueryCurrent++ if strings.HasPrefix(expected, "SUB") { - // remove the SUB from the expected, - // and truncate the query to length(expected) + // Remove the SUB from the expected, + // and truncate the query to length(expected). expected = expected[3:] if len(query) > len(expected) { query = query[:len(expected)] @@ -535,7 +543,7 @@ func (fmd *FakeMysqlDaemon) ExecuteSuperQueryList(ctx context.Context, queryList return fmt.Errorf("wrong query for ExecuteSuperQueryList: expected %v got %v", expected, query) } - // intercept some queries to update our status + // Intercept some queries to update our status. switch query { case "START SLAVE": fmd.Replicating = true @@ -546,7 +554,7 @@ func (fmd *FakeMysqlDaemon) ExecuteSuperQueryList(ctx context.Context, queryList return nil } -// FetchSuperQuery returns the results from the map, if any +// FetchSuperQuery returns the results from the map, if any. func (fmd *FakeMysqlDaemon) FetchSuperQuery(ctx context.Context, query string) (*sqltypes.Result, error) { if fmd.FetchSuperQueryMap == nil { return nil, fmt.Errorf("unexpected query: %v", query) @@ -563,7 +571,7 @@ func (fmd *FakeMysqlDaemon) FetchSuperQuery(ctx context.Context, query string) ( return nil, fmt.Errorf("unexpected query: %v", query) } -// Close is part of the MysqlDaemon interface +// Close is part of the MysqlDaemon interface. func (fmd *FakeMysqlDaemon) Close() { if fmd.appPool != nil { fmd.appPool.Close() @@ -579,7 +587,7 @@ func (fmd *FakeMysqlDaemon) CheckSuperQueryList() error { return nil } -// GetSchema is part of the MysqlDaemon interface +// GetSchema is part of the MysqlDaemon interface. func (fmd *FakeMysqlDaemon) GetSchema(ctx context.Context, dbName string, request *tabletmanagerdatapb.GetSchemaRequest) (*tabletmanagerdatapb.SchemaDefinition, error) { if fmd.SchemaFunc != nil { return fmd.SchemaFunc() @@ -590,22 +598,22 @@ func (fmd *FakeMysqlDaemon) GetSchema(ctx context.Context, dbName string, reques return tmutils.FilterTables(fmd.Schema, request.Tables, request.ExcludeTables, request.IncludeViews) } -// GetColumns is part of the MysqlDaemon interface +// GetColumns is part of the MysqlDaemon interface. func (fmd *FakeMysqlDaemon) GetColumns(ctx context.Context, dbName, table string) ([]*querypb.Field, []string, error) { return []*querypb.Field{}, []string{}, nil } -// GetPrimaryKeyColumns is part of the MysqlDaemon interface +// GetPrimaryKeyColumns is part of the MysqlDaemon interface. func (fmd *FakeMysqlDaemon) GetPrimaryKeyColumns(ctx context.Context, dbName, table string) ([]string, error) { return []string{}, nil } -// GetPrimaryKeyEquivalentColumns is part of the MysqlDaemon interface -func (fmd *FakeMysqlDaemon) GetPrimaryKeyEquivalentColumns(ctx context.Context, dbName, table string) ([]string, error) { - return []string{}, nil +// GetPrimaryKeyEquivalentColumns is part of the MysqlDaemon interface. +func (fmd *FakeMysqlDaemon) GetPrimaryKeyEquivalentColumns(ctx context.Context, dbName, table string) ([]string, string, error) { + return []string{}, "", nil } -// PreflightSchemaChange is part of the MysqlDaemon interface +// PreflightSchemaChange is part of the MysqlDaemon interface. func (fmd *FakeMysqlDaemon) PreflightSchemaChange(ctx context.Context, dbName string, changes []string) ([]*tabletmanagerdatapb.SchemaChangeResult, error) { if fmd.PreflightSchemaChangeResult == nil { return nil, fmt.Errorf("no preflight result defined") @@ -613,7 +621,7 @@ func (fmd *FakeMysqlDaemon) PreflightSchemaChange(ctx context.Context, dbName st return fmd.PreflightSchemaChangeResult, nil } -// ApplySchemaChange is part of the MysqlDaemon interface +// ApplySchemaChange is part of the MysqlDaemon interface. func (fmd *FakeMysqlDaemon) ApplySchemaChange(ctx context.Context, dbName string, change *tmutils.SchemaChange) (*tabletmanagerdatapb.SchemaChangeResult, error) { beforeSchema, err := fmd.SchemaFunc() if err != nil { diff --git a/go/vt/mysqlctl/mysql_daemon.go b/go/vt/mysqlctl/mysql_daemon.go index c0f97d438e6..f50d368ed7d 100644 --- a/go/vt/mysqlctl/mysql_daemon.go +++ b/go/vt/mysqlctl/mysql_daemon.go @@ -93,7 +93,7 @@ type MysqlDaemon interface { GetSchema(ctx context.Context, dbName string, request *tabletmanagerdatapb.GetSchemaRequest) (*tabletmanagerdatapb.SchemaDefinition, error) GetColumns(ctx context.Context, dbName, table string) ([]*querypb.Field, []string, error) GetPrimaryKeyColumns(ctx context.Context, dbName, table string) ([]string, error) - GetPrimaryKeyEquivalentColumns(ctx context.Context, dbName, table string) ([]string, error) + GetPrimaryKeyEquivalentColumns(ctx context.Context, dbName, table string) ([]string, string, error) PreflightSchemaChange(ctx context.Context, dbName string, changes []string) ([]*tabletmanagerdatapb.SchemaChangeResult, error) ApplySchemaChange(ctx context.Context, dbName string, change *tmutils.SchemaChange) (*tabletmanagerdatapb.SchemaChangeResult, error) diff --git a/go/vt/mysqlctl/schema.go b/go/vt/mysqlctl/schema.go index 397668145ef..8a14d26acce 100644 --- a/go/vt/mysqlctl/schema.go +++ b/go/vt/mysqlctl/schema.go @@ -31,10 +31,11 @@ import ( "vitess.io/vitess/go/vt/concurrency" "vitess.io/vitess/go/vt/log" "vitess.io/vitess/go/vt/mysqlctl/tmutils" + "vitess.io/vitess/go/vt/vterrors" + querypb "vitess.io/vitess/go/vt/proto/query" tabletmanagerdatapb "vitess.io/vitess/go/vt/proto/tabletmanagerdata" - "vitess.io/vitess/go/vt/proto/vtrpc" - "vitess.io/vitess/go/vt/vterrors" + vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc" ) const ( @@ -74,7 +75,7 @@ func encodeEntityName(name string) string { // tableListSQL returns an IN clause "('t1', 't2'...) for a list of tables." func tableListSQL(tables []string) (string, error) { if len(tables) == 0 { - return "", vterrors.New(vtrpc.Code_INTERNAL, "no tables for tableListSQL") + return "", vterrors.New(vtrpcpb.Code_INTERNAL, "no tables for tableListSQL") } encodedTables := make([]string, len(tables)) @@ -566,10 +567,11 @@ func (mysqld *Mysqld) ApplySchemaChange(ctx context.Context, dbName string, chan // GetPrimaryKeyEquivalentColumns can be used if the table has // no defined PRIMARY KEY. It will return the columns in a // viable PRIMARY KEY equivalent (PKE) -- a NON-NULL UNIQUE -// KEY -- in the specified table. When multiple PKE indexes -// are available it will attempt to choose the most efficient -// one based on the column data types and the number of columns -// in the index. See here for the data type storage sizes: +// KEY -- along with that index's name in the specified table. +// When multiple PKE indexes are available it will attempt to +// choose the most efficient one based on the column data types +// and the number of columns in the index. See here for the data +// type storage sizes: // // https://dev.mysql.com/doc/refman/en/storage-requirements.html // @@ -577,16 +579,16 @@ func (mysqld *Mysqld) ApplySchemaChange(ctx context.Context, dbName string, chan // defined PRIMARY KEY then it may return the columns for // that index if it is likely the most efficient one amongst // the available PKE indexes on the table. -func (mysqld *Mysqld) GetPrimaryKeyEquivalentColumns(ctx context.Context, dbName, table string) ([]string, error) { +func (mysqld *Mysqld) GetPrimaryKeyEquivalentColumns(ctx context.Context, dbName, table string) ([]string, string, error) { conn, err := getPoolReconnect(ctx, mysqld.dbaPool) if err != nil { - return nil, err + return nil, "", err } defer conn.Recycle() // We use column name aliases to guarantee lower case for our named results. sql := ` - SELECT COLUMN_NAME AS column_name FROM information_schema.STATISTICS AS index_cols INNER JOIN + SELECT index_cols.COLUMN_NAME AS column_name, index_cols.INDEX_NAME as index_name FROM information_schema.STATISTICS AS index_cols INNER JOIN ( SELECT stats.INDEX_NAME, SUM( CASE LOWER(cols.DATA_TYPE) @@ -629,15 +631,28 @@ func (mysqld *Mysqld) GetPrimaryKeyEquivalentColumns(ctx context.Context, dbName sql = fmt.Sprintf(sql, encodedDbName, encodedTable, encodedDbName, encodedTable, encodedDbName, encodedTable) qr, err := conn.ExecuteFetch(sql, 1000, true) if err != nil { - return nil, err + return nil, "", err } named := qr.Named() cols := make([]string, len(qr.Rows)) + indexName := "" for i, row := range named.Rows { cols[i] = row.AsString("column_name", "") + in := row.AsString("index_name", "") + if in == "" { // This should never happen + return nil, "", vterrors.Errorf(vtrpcpb.Code_INTERNAL, "PKE column (%s) returned with an empty index name", + cols[i]) + } + switch { + case i == 0: + indexName = in + case i > 0 && indexName != in: // This should never happen + return nil, "", vterrors.Errorf(vtrpcpb.Code_INTERNAL, "PKE columns (%s) returned for more than one index: %s, %s", + strings.Join(cols, ","), indexName, in) + } } - return cols, err + return cols, indexName, err } // tableDefinitions is a sortable collection of table definitions diff --git a/go/vt/proto/binlogdata/binlogdata.pb.go b/go/vt/proto/binlogdata/binlogdata.pb.go index 3da747d3832..c0a4bd08860 100644 --- a/go/vt/proto/binlogdata/binlogdata.pb.go +++ b/go/vt/proto/binlogdata/binlogdata.pb.go @@ -2029,6 +2029,10 @@ type MinimalTable struct { Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` Fields []*query.Field `protobuf:"bytes,2,rep,name=fields,proto3" json:"fields,omitempty"` PKColumns []int64 `protobuf:"varint,3,rep,packed,name=p_k_columns,json=pKColumns,proto3" json:"p_k_columns,omitempty"` + // This will be PRIMARY when the actual primary key is used and it + // will be the name of the Primary Key equivalent if one is used + // instead. Otherwise it will be empty. + PKIndexName string `protobuf:"bytes,4,opt,name=p_k_index_name,json=pKIndexName,proto3" json:"p_k_index_name,omitempty"` } func (x *MinimalTable) Reset() { @@ -2084,6 +2088,13 @@ func (x *MinimalTable) GetPKColumns() []int64 { return nil } +func (x *MinimalTable) GetPKIndexName() string { + if x != nil { + return x.PKIndexName + } + return "" +} + type MinimalSchema struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -3239,76 +3250,46 @@ var file_binlogdata_proto_rawDesc = []byte{ 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x17, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, 0x1c, 0x0a, 0x09, 0x74, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x64, 0x18, 0x18, 0x20, 0x01, 0x28, - 0x08, 0x52, 0x09, 0x74, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x64, 0x22, 0x68, 0x0a, 0x0c, - 0x4d, 0x69, 0x6e, 0x69, 0x6d, 0x61, 0x6c, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x12, 0x0a, 0x04, - 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, - 0x12, 0x24, 0x0a, 0x06, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, - 0x32, 0x0c, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x52, 0x06, - 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x12, 0x1e, 0x0a, 0x0b, 0x70, 0x5f, 0x6b, 0x5f, 0x63, 0x6f, - 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x03, 0x52, 0x09, 0x70, 0x4b, 0x43, - 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x22, 0x41, 0x0a, 0x0d, 0x4d, 0x69, 0x6e, 0x69, 0x6d, 0x61, - 0x6c, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x30, 0x0a, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, - 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, - 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4d, 0x69, 0x6e, 0x69, 0x6d, 0x61, 0x6c, 0x54, 0x61, 0x62, 0x6c, - 0x65, 0x52, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x22, 0xc7, 0x02, 0x0a, 0x0e, 0x56, 0x53, - 0x74, 0x72, 0x65, 0x61, 0x6d, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x3f, 0x0a, 0x13, - 0x65, 0x66, 0x66, 0x65, 0x63, 0x74, 0x69, 0x76, 0x65, 0x5f, 0x63, 0x61, 0x6c, 0x6c, 0x65, 0x72, - 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x76, 0x74, 0x72, 0x70, - 0x63, 0x2e, 0x43, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x44, 0x52, 0x11, 0x65, 0x66, 0x66, 0x65, - 0x63, 0x74, 0x69, 0x76, 0x65, 0x43, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x64, 0x12, 0x45, 0x0a, - 0x13, 0x69, 0x6d, 0x6d, 0x65, 0x64, 0x69, 0x61, 0x74, 0x65, 0x5f, 0x63, 0x61, 0x6c, 0x6c, 0x65, - 0x72, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x71, 0x75, 0x65, - 0x72, 0x79, 0x2e, 0x56, 0x54, 0x47, 0x61, 0x74, 0x65, 0x43, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, - 0x44, 0x52, 0x11, 0x69, 0x6d, 0x6d, 0x65, 0x64, 0x69, 0x61, 0x74, 0x65, 0x43, 0x61, 0x6c, 0x6c, - 0x65, 0x72, 0x49, 0x64, 0x12, 0x25, 0x0a, 0x06, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x18, 0x03, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0d, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x54, 0x61, 0x72, - 0x67, 0x65, 0x74, 0x52, 0x06, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x70, - 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x70, - 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x2a, 0x0a, 0x06, 0x66, 0x69, 0x6c, 0x74, 0x65, - 0x72, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, - 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x52, 0x06, 0x66, 0x69, 0x6c, - 0x74, 0x65, 0x72, 0x12, 0x3e, 0x0a, 0x0f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6c, 0x61, 0x73, - 0x74, 0x5f, 0x70, 0x5f, 0x6b, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x62, - 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4c, - 0x61, 0x73, 0x74, 0x50, 0x4b, 0x52, 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4c, 0x61, 0x73, 0x74, - 0x50, 0x4b, 0x73, 0x22, 0x3d, 0x0a, 0x0f, 0x56, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2a, 0x0a, 0x06, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73, - 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, - 0x61, 0x74, 0x61, 0x2e, 0x56, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x52, 0x06, 0x65, 0x76, 0x65, 0x6e, - 0x74, 0x73, 0x22, 0x85, 0x02, 0x0a, 0x12, 0x56, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x52, 0x6f, - 0x77, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x3f, 0x0a, 0x13, 0x65, 0x66, 0x66, - 0x65, 0x63, 0x74, 0x69, 0x76, 0x65, 0x5f, 0x63, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x5f, 0x69, 0x64, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x76, 0x74, 0x72, 0x70, 0x63, 0x2e, 0x43, - 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x44, 0x52, 0x11, 0x65, 0x66, 0x66, 0x65, 0x63, 0x74, 0x69, - 0x76, 0x65, 0x43, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x64, 0x12, 0x45, 0x0a, 0x13, 0x69, 0x6d, - 0x6d, 0x65, 0x64, 0x69, 0x61, 0x74, 0x65, 0x5f, 0x63, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x5f, 0x69, - 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, - 0x56, 0x54, 0x47, 0x61, 0x74, 0x65, 0x43, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x44, 0x52, 0x11, - 0x69, 0x6d, 0x6d, 0x65, 0x64, 0x69, 0x61, 0x74, 0x65, 0x43, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, - 0x64, 0x12, 0x25, 0x0a, 0x06, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x0d, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, - 0x52, 0x06, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x12, 0x14, 0x0a, 0x05, 0x71, 0x75, 0x65, 0x72, - 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x12, 0x2a, - 0x0a, 0x06, 0x6c, 0x61, 0x73, 0x74, 0x70, 0x6b, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, - 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x52, 0x65, 0x73, 0x75, - 0x6c, 0x74, 0x52, 0x06, 0x6c, 0x61, 0x73, 0x74, 0x70, 0x6b, 0x22, 0xf9, 0x01, 0x0a, 0x13, 0x56, - 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x52, 0x6f, 0x77, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x12, 0x24, 0x0a, 0x06, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x18, 0x01, 0x20, 0x03, - 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, - 0x52, 0x06, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x12, 0x28, 0x0a, 0x08, 0x70, 0x6b, 0x66, 0x69, - 0x65, 0x6c, 0x64, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x71, 0x75, 0x65, - 0x72, 0x79, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x52, 0x08, 0x70, 0x6b, 0x66, 0x69, 0x65, 0x6c, - 0x64, 0x73, 0x12, 0x12, 0x0a, 0x04, 0x67, 0x74, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x04, 0x67, 0x74, 0x69, 0x64, 0x12, 0x1e, 0x0a, 0x04, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x04, - 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0a, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x52, 0x6f, 0x77, - 0x52, 0x04, 0x72, 0x6f, 0x77, 0x73, 0x12, 0x22, 0x0a, 0x06, 0x6c, 0x61, 0x73, 0x74, 0x70, 0x6b, - 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0a, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x52, - 0x6f, 0x77, 0x52, 0x06, 0x6c, 0x61, 0x73, 0x74, 0x70, 0x6b, 0x12, 0x1c, 0x0a, 0x09, 0x74, 0x68, - 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x64, 0x18, 0x06, 0x20, 0x01, 0x28, 0x08, 0x52, 0x09, 0x74, - 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x64, 0x12, 0x1c, 0x0a, 0x09, 0x68, 0x65, 0x61, 0x72, - 0x74, 0x62, 0x65, 0x61, 0x74, 0x18, 0x07, 0x20, 0x01, 0x28, 0x08, 0x52, 0x09, 0x68, 0x65, 0x61, - 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x22, 0xc5, 0x01, 0x0a, 0x14, 0x56, 0x53, 0x74, 0x72, 0x65, - 0x61, 0x6d, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, + 0x08, 0x52, 0x09, 0x74, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x64, 0x22, 0x8d, 0x01, 0x0a, + 0x0c, 0x4d, 0x69, 0x6e, 0x69, 0x6d, 0x61, 0x6c, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x12, 0x0a, + 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, + 0x65, 0x12, 0x24, 0x0a, 0x06, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x0c, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x52, + 0x06, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x12, 0x1e, 0x0a, 0x0b, 0x70, 0x5f, 0x6b, 0x5f, 0x63, + 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x03, 0x52, 0x09, 0x70, 0x4b, + 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x12, 0x23, 0x0a, 0x0e, 0x70, 0x5f, 0x6b, 0x5f, 0x69, + 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x0b, 0x70, 0x4b, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0x41, 0x0a, 0x0d, + 0x4d, 0x69, 0x6e, 0x69, 0x6d, 0x61, 0x6c, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x30, 0x0a, + 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x18, 0x2e, + 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4d, 0x69, 0x6e, 0x69, 0x6d, + 0x61, 0x6c, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x22, + 0xc7, 0x02, 0x0a, 0x0e, 0x56, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x12, 0x3f, 0x0a, 0x13, 0x65, 0x66, 0x66, 0x65, 0x63, 0x74, 0x69, 0x76, 0x65, 0x5f, + 0x63, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x0f, 0x2e, 0x76, 0x74, 0x72, 0x70, 0x63, 0x2e, 0x43, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x44, + 0x52, 0x11, 0x65, 0x66, 0x66, 0x65, 0x63, 0x74, 0x69, 0x76, 0x65, 0x43, 0x61, 0x6c, 0x6c, 0x65, + 0x72, 0x49, 0x64, 0x12, 0x45, 0x0a, 0x13, 0x69, 0x6d, 0x6d, 0x65, 0x64, 0x69, 0x61, 0x74, 0x65, + 0x5f, 0x63, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x15, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x56, 0x54, 0x47, 0x61, 0x74, 0x65, 0x43, + 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x44, 0x52, 0x11, 0x69, 0x6d, 0x6d, 0x65, 0x64, 0x69, 0x61, + 0x74, 0x65, 0x43, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x64, 0x12, 0x25, 0x0a, 0x06, 0x74, 0x61, + 0x72, 0x67, 0x65, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0d, 0x2e, 0x71, 0x75, 0x65, + 0x72, 0x79, 0x2e, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x52, 0x06, 0x74, 0x61, 0x72, 0x67, 0x65, + 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x08, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x2a, 0x0a, + 0x06, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, + 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, 0x69, 0x6c, 0x74, 0x65, + 0x72, 0x52, 0x06, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x12, 0x3e, 0x0a, 0x0f, 0x74, 0x61, 0x62, + 0x6c, 0x65, 0x5f, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x70, 0x5f, 0x6b, 0x73, 0x18, 0x06, 0x20, 0x03, + 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, + 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4c, 0x61, 0x73, 0x74, 0x50, 0x4b, 0x52, 0x0c, 0x74, 0x61, 0x62, + 0x6c, 0x65, 0x4c, 0x61, 0x73, 0x74, 0x50, 0x4b, 0x73, 0x22, 0x3d, 0x0a, 0x0f, 0x56, 0x53, 0x74, + 0x72, 0x65, 0x61, 0x6d, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2a, 0x0a, 0x06, + 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x62, + 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x45, 0x76, 0x65, 0x6e, 0x74, + 0x52, 0x06, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x22, 0x85, 0x02, 0x0a, 0x12, 0x56, 0x53, 0x74, + 0x72, 0x65, 0x61, 0x6d, 0x52, 0x6f, 0x77, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x3f, 0x0a, 0x13, 0x65, 0x66, 0x66, 0x65, 0x63, 0x74, 0x69, 0x76, 0x65, 0x5f, 0x63, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x76, 0x74, 0x72, 0x70, 0x63, 0x2e, 0x43, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x44, 0x52, 0x11, 0x65, @@ -3319,101 +3300,133 @@ var file_binlogdata_proto_rawDesc = []byte{ 0x65, 0x72, 0x49, 0x44, 0x52, 0x11, 0x69, 0x6d, 0x6d, 0x65, 0x64, 0x69, 0x61, 0x74, 0x65, 0x43, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x64, 0x12, 0x25, 0x0a, 0x06, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0d, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, - 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x52, 0x06, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x22, 0xde, - 0x01, 0x0a, 0x15, 0x56, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, - 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x74, 0x61, 0x62, 0x6c, - 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x74, 0x61, - 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x24, 0x0a, 0x06, 0x66, 0x69, 0x65, 0x6c, 0x64, - 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, - 0x46, 0x69, 0x65, 0x6c, 0x64, 0x52, 0x06, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x12, 0x28, 0x0a, - 0x08, 0x70, 0x6b, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, - 0x0c, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x52, 0x08, 0x70, - 0x6b, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x12, 0x12, 0x0a, 0x04, 0x67, 0x74, 0x69, 0x64, 0x18, - 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x67, 0x74, 0x69, 0x64, 0x12, 0x1e, 0x0a, 0x04, 0x72, - 0x6f, 0x77, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0a, 0x2e, 0x71, 0x75, 0x65, 0x72, - 0x79, 0x2e, 0x52, 0x6f, 0x77, 0x52, 0x04, 0x72, 0x6f, 0x77, 0x73, 0x12, 0x22, 0x0a, 0x06, 0x6c, - 0x61, 0x73, 0x74, 0x70, 0x6b, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0a, 0x2e, 0x71, 0x75, - 0x65, 0x72, 0x79, 0x2e, 0x52, 0x6f, 0x77, 0x52, 0x06, 0x6c, 0x61, 0x73, 0x74, 0x70, 0x6b, 0x22, - 0x69, 0x0a, 0x0b, 0x4c, 0x61, 0x73, 0x74, 0x50, 0x4b, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x12, 0x3c, - 0x0a, 0x0e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x70, 0x5f, 0x6b, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, - 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4c, 0x61, 0x73, 0x74, 0x50, 0x4b, 0x52, - 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4c, 0x61, 0x73, 0x74, 0x50, 0x4b, 0x12, 0x1c, 0x0a, 0x09, - 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, - 0x09, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x22, 0x58, 0x0a, 0x0b, 0x54, 0x61, - 0x62, 0x6c, 0x65, 0x4c, 0x61, 0x73, 0x74, 0x50, 0x4b, 0x12, 0x1d, 0x0a, 0x0a, 0x74, 0x61, 0x62, - 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x74, - 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x2a, 0x0a, 0x06, 0x6c, 0x61, 0x73, 0x74, - 0x70, 0x6b, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, - 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x52, 0x06, 0x6c, 0x61, - 0x73, 0x74, 0x70, 0x6b, 0x22, 0xdc, 0x01, 0x0a, 0x15, 0x56, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, - 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x3f, - 0x0a, 0x13, 0x65, 0x66, 0x66, 0x65, 0x63, 0x74, 0x69, 0x76, 0x65, 0x5f, 0x63, 0x61, 0x6c, 0x6c, - 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x76, 0x74, - 0x72, 0x70, 0x63, 0x2e, 0x43, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x44, 0x52, 0x11, 0x65, 0x66, - 0x66, 0x65, 0x63, 0x74, 0x69, 0x76, 0x65, 0x43, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x64, 0x12, - 0x45, 0x0a, 0x13, 0x69, 0x6d, 0x6d, 0x65, 0x64, 0x69, 0x61, 0x74, 0x65, 0x5f, 0x63, 0x61, 0x6c, - 0x6c, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x71, - 0x75, 0x65, 0x72, 0x79, 0x2e, 0x56, 0x54, 0x47, 0x61, 0x74, 0x65, 0x43, 0x61, 0x6c, 0x6c, 0x65, - 0x72, 0x49, 0x44, 0x52, 0x11, 0x69, 0x6d, 0x6d, 0x65, 0x64, 0x69, 0x61, 0x74, 0x65, 0x43, 0x61, - 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x64, 0x12, 0x25, 0x0a, 0x06, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, - 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0d, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x54, - 0x61, 0x72, 0x67, 0x65, 0x74, 0x52, 0x06, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x12, 0x14, 0x0a, - 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x71, 0x75, - 0x65, 0x72, 0x79, 0x22, 0x72, 0x0a, 0x16, 0x56, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x52, 0x65, - 0x73, 0x75, 0x6c, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x24, 0x0a, - 0x06, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0c, 0x2e, + 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x52, 0x06, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x12, 0x14, + 0x0a, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x71, + 0x75, 0x65, 0x72, 0x79, 0x12, 0x2a, 0x0a, 0x06, 0x6c, 0x61, 0x73, 0x74, 0x70, 0x6b, 0x18, 0x05, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x51, 0x75, 0x65, + 0x72, 0x79, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x52, 0x06, 0x6c, 0x61, 0x73, 0x74, 0x70, 0x6b, + 0x22, 0xf9, 0x01, 0x0a, 0x13, 0x56, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x52, 0x6f, 0x77, 0x73, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x24, 0x0a, 0x06, 0x66, 0x69, 0x65, 0x6c, + 0x64, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, + 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x52, 0x06, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x12, 0x28, + 0x0a, 0x08, 0x70, 0x6b, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x0c, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x52, 0x08, + 0x70, 0x6b, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x12, 0x12, 0x0a, 0x04, 0x67, 0x74, 0x69, 0x64, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x67, 0x74, 0x69, 0x64, 0x12, 0x1e, 0x0a, 0x04, + 0x72, 0x6f, 0x77, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0a, 0x2e, 0x71, 0x75, 0x65, + 0x72, 0x79, 0x2e, 0x52, 0x6f, 0x77, 0x52, 0x04, 0x72, 0x6f, 0x77, 0x73, 0x12, 0x22, 0x0a, 0x06, + 0x6c, 0x61, 0x73, 0x74, 0x70, 0x6b, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0a, 0x2e, 0x71, + 0x75, 0x65, 0x72, 0x79, 0x2e, 0x52, 0x6f, 0x77, 0x52, 0x06, 0x6c, 0x61, 0x73, 0x74, 0x70, 0x6b, + 0x12, 0x1c, 0x0a, 0x09, 0x74, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x64, 0x18, 0x06, 0x20, + 0x01, 0x28, 0x08, 0x52, 0x09, 0x74, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x64, 0x12, 0x1c, + 0x0a, 0x09, 0x68, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x18, 0x07, 0x20, 0x01, 0x28, + 0x08, 0x52, 0x09, 0x68, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x22, 0xc5, 0x01, 0x0a, + 0x14, 0x56, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x3f, 0x0a, 0x13, 0x65, 0x66, 0x66, 0x65, 0x63, 0x74, 0x69, + 0x76, 0x65, 0x5f, 0x63, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x76, 0x74, 0x72, 0x70, 0x63, 0x2e, 0x43, 0x61, 0x6c, 0x6c, 0x65, + 0x72, 0x49, 0x44, 0x52, 0x11, 0x65, 0x66, 0x66, 0x65, 0x63, 0x74, 0x69, 0x76, 0x65, 0x43, 0x61, + 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x64, 0x12, 0x45, 0x0a, 0x13, 0x69, 0x6d, 0x6d, 0x65, 0x64, 0x69, + 0x61, 0x74, 0x65, 0x5f, 0x63, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x56, 0x54, 0x47, 0x61, + 0x74, 0x65, 0x43, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x44, 0x52, 0x11, 0x69, 0x6d, 0x6d, 0x65, + 0x64, 0x69, 0x61, 0x74, 0x65, 0x43, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x64, 0x12, 0x25, 0x0a, + 0x06, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0d, 0x2e, + 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x52, 0x06, 0x74, 0x61, + 0x72, 0x67, 0x65, 0x74, 0x22, 0xde, 0x01, 0x0a, 0x15, 0x56, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, + 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, + 0x0a, 0x0a, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x09, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x24, 0x0a, + 0x06, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x52, 0x06, 0x66, 0x69, 0x65, - 0x6c, 0x64, 0x73, 0x12, 0x12, 0x0a, 0x04, 0x67, 0x74, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x04, 0x67, 0x74, 0x69, 0x64, 0x12, 0x1e, 0x0a, 0x04, 0x72, 0x6f, 0x77, 0x73, 0x18, - 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0a, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x52, 0x6f, - 0x77, 0x52, 0x04, 0x72, 0x6f, 0x77, 0x73, 0x2a, 0x3e, 0x0a, 0x0b, 0x4f, 0x6e, 0x44, 0x44, 0x4c, - 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x0a, 0x0a, 0x06, 0x49, 0x47, 0x4e, 0x4f, 0x52, 0x45, - 0x10, 0x00, 0x12, 0x08, 0x0a, 0x04, 0x53, 0x54, 0x4f, 0x50, 0x10, 0x01, 0x12, 0x08, 0x0a, 0x04, - 0x45, 0x58, 0x45, 0x43, 0x10, 0x02, 0x12, 0x0f, 0x0a, 0x0b, 0x45, 0x58, 0x45, 0x43, 0x5f, 0x49, - 0x47, 0x4e, 0x4f, 0x52, 0x45, 0x10, 0x03, 0x2a, 0x7b, 0x0a, 0x18, 0x56, 0x52, 0x65, 0x70, 0x6c, - 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, - 0x79, 0x70, 0x65, 0x12, 0x0f, 0x0a, 0x0b, 0x4d, 0x61, 0x74, 0x65, 0x72, 0x69, 0x61, 0x6c, 0x69, - 0x7a, 0x65, 0x10, 0x00, 0x12, 0x0e, 0x0a, 0x0a, 0x4d, 0x6f, 0x76, 0x65, 0x54, 0x61, 0x62, 0x6c, - 0x65, 0x73, 0x10, 0x01, 0x12, 0x15, 0x0a, 0x11, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x4c, 0x6f, - 0x6f, 0x6b, 0x75, 0x70, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x10, 0x02, 0x12, 0x0b, 0x0a, 0x07, 0x4d, - 0x69, 0x67, 0x72, 0x61, 0x74, 0x65, 0x10, 0x03, 0x12, 0x0b, 0x0a, 0x07, 0x52, 0x65, 0x73, 0x68, - 0x61, 0x72, 0x64, 0x10, 0x04, 0x12, 0x0d, 0x0a, 0x09, 0x4f, 0x6e, 0x6c, 0x69, 0x6e, 0x65, 0x44, - 0x44, 0x4c, 0x10, 0x05, 0x2a, 0x44, 0x0a, 0x1b, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x53, 0x75, 0x62, 0x54, - 0x79, 0x70, 0x65, 0x12, 0x08, 0x0a, 0x04, 0x4e, 0x6f, 0x6e, 0x65, 0x10, 0x00, 0x12, 0x0b, 0x0a, - 0x07, 0x50, 0x61, 0x72, 0x74, 0x69, 0x61, 0x6c, 0x10, 0x01, 0x12, 0x0e, 0x0a, 0x0a, 0x41, 0x74, - 0x6f, 0x6d, 0x69, 0x63, 0x43, 0x6f, 0x70, 0x79, 0x10, 0x02, 0x2a, 0x71, 0x0a, 0x19, 0x56, 0x52, + 0x6c, 0x64, 0x73, 0x12, 0x28, 0x0a, 0x08, 0x70, 0x6b, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x18, + 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x46, 0x69, + 0x65, 0x6c, 0x64, 0x52, 0x08, 0x70, 0x6b, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x12, 0x12, 0x0a, + 0x04, 0x67, 0x74, 0x69, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x67, 0x74, 0x69, + 0x64, 0x12, 0x1e, 0x0a, 0x04, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, + 0x0a, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x52, 0x6f, 0x77, 0x52, 0x04, 0x72, 0x6f, 0x77, + 0x73, 0x12, 0x22, 0x0a, 0x06, 0x6c, 0x61, 0x73, 0x74, 0x70, 0x6b, 0x18, 0x06, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x0a, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x52, 0x6f, 0x77, 0x52, 0x06, 0x6c, + 0x61, 0x73, 0x74, 0x70, 0x6b, 0x22, 0x69, 0x0a, 0x0b, 0x4c, 0x61, 0x73, 0x74, 0x50, 0x4b, 0x45, + 0x76, 0x65, 0x6e, 0x74, 0x12, 0x3c, 0x0a, 0x0e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6c, 0x61, + 0x73, 0x74, 0x5f, 0x70, 0x5f, 0x6b, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x62, + 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4c, + 0x61, 0x73, 0x74, 0x50, 0x4b, 0x52, 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4c, 0x61, 0x73, 0x74, + 0x50, 0x4b, 0x12, 0x1c, 0x0a, 0x09, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x09, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, + 0x22, 0x58, 0x0a, 0x0b, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4c, 0x61, 0x73, 0x74, 0x50, 0x4b, 0x12, + 0x1d, 0x0a, 0x0a, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x09, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x2a, + 0x0a, 0x06, 0x6c, 0x61, 0x73, 0x74, 0x70, 0x6b, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, + 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x52, 0x65, 0x73, 0x75, + 0x6c, 0x74, 0x52, 0x06, 0x6c, 0x61, 0x73, 0x74, 0x70, 0x6b, 0x22, 0xdc, 0x01, 0x0a, 0x15, 0x56, + 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x12, 0x3f, 0x0a, 0x13, 0x65, 0x66, 0x66, 0x65, 0x63, 0x74, 0x69, 0x76, + 0x65, 0x5f, 0x63, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x0f, 0x2e, 0x76, 0x74, 0x72, 0x70, 0x63, 0x2e, 0x43, 0x61, 0x6c, 0x6c, 0x65, 0x72, + 0x49, 0x44, 0x52, 0x11, 0x65, 0x66, 0x66, 0x65, 0x63, 0x74, 0x69, 0x76, 0x65, 0x43, 0x61, 0x6c, + 0x6c, 0x65, 0x72, 0x49, 0x64, 0x12, 0x45, 0x0a, 0x13, 0x69, 0x6d, 0x6d, 0x65, 0x64, 0x69, 0x61, + 0x74, 0x65, 0x5f, 0x63, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x56, 0x54, 0x47, 0x61, 0x74, + 0x65, 0x43, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x44, 0x52, 0x11, 0x69, 0x6d, 0x6d, 0x65, 0x64, + 0x69, 0x61, 0x74, 0x65, 0x43, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x64, 0x12, 0x25, 0x0a, 0x06, + 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0d, 0x2e, 0x71, + 0x75, 0x65, 0x72, 0x79, 0x2e, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x52, 0x06, 0x74, 0x61, 0x72, + 0x67, 0x65, 0x74, 0x12, 0x14, 0x0a, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x18, 0x04, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x22, 0x72, 0x0a, 0x16, 0x56, 0x53, 0x74, + 0x72, 0x65, 0x61, 0x6d, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x12, 0x24, 0x0a, 0x06, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x18, 0x01, 0x20, + 0x03, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x46, 0x69, 0x65, 0x6c, + 0x64, 0x52, 0x06, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x12, 0x12, 0x0a, 0x04, 0x67, 0x74, 0x69, + 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x67, 0x74, 0x69, 0x64, 0x12, 0x1e, 0x0a, + 0x04, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0a, 0x2e, 0x71, 0x75, + 0x65, 0x72, 0x79, 0x2e, 0x52, 0x6f, 0x77, 0x52, 0x04, 0x72, 0x6f, 0x77, 0x73, 0x2a, 0x3e, 0x0a, + 0x0b, 0x4f, 0x6e, 0x44, 0x44, 0x4c, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x0a, 0x0a, 0x06, + 0x49, 0x47, 0x4e, 0x4f, 0x52, 0x45, 0x10, 0x00, 0x12, 0x08, 0x0a, 0x04, 0x53, 0x54, 0x4f, 0x50, + 0x10, 0x01, 0x12, 0x08, 0x0a, 0x04, 0x45, 0x58, 0x45, 0x43, 0x10, 0x02, 0x12, 0x0f, 0x0a, 0x0b, + 0x45, 0x58, 0x45, 0x43, 0x5f, 0x49, 0x47, 0x4e, 0x4f, 0x52, 0x45, 0x10, 0x03, 0x2a, 0x7b, 0x0a, + 0x18, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0f, 0x0a, 0x0b, 0x4d, 0x61, 0x74, + 0x65, 0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x10, 0x00, 0x12, 0x0e, 0x0a, 0x0a, 0x4d, 0x6f, + 0x76, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x10, 0x01, 0x12, 0x15, 0x0a, 0x11, 0x43, 0x72, + 0x65, 0x61, 0x74, 0x65, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x10, + 0x02, 0x12, 0x0b, 0x0a, 0x07, 0x4d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x65, 0x10, 0x03, 0x12, 0x0b, + 0x0a, 0x07, 0x52, 0x65, 0x73, 0x68, 0x61, 0x72, 0x64, 0x10, 0x04, 0x12, 0x0d, 0x0a, 0x09, 0x4f, + 0x6e, 0x6c, 0x69, 0x6e, 0x65, 0x44, 0x44, 0x4c, 0x10, 0x05, 0x2a, 0x44, 0x0a, 0x1b, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, - 0x6f, 0x77, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x0b, 0x0a, 0x07, 0x55, 0x6e, 0x6b, 0x6e, 0x6f, - 0x77, 0x6e, 0x10, 0x00, 0x12, 0x08, 0x0a, 0x04, 0x49, 0x6e, 0x69, 0x74, 0x10, 0x01, 0x12, 0x0b, - 0x0a, 0x07, 0x53, 0x74, 0x6f, 0x70, 0x70, 0x65, 0x64, 0x10, 0x02, 0x12, 0x0b, 0x0a, 0x07, 0x43, - 0x6f, 0x70, 0x79, 0x69, 0x6e, 0x67, 0x10, 0x03, 0x12, 0x0b, 0x0a, 0x07, 0x52, 0x75, 0x6e, 0x6e, - 0x69, 0x6e, 0x67, 0x10, 0x04, 0x12, 0x09, 0x0a, 0x05, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x10, 0x05, - 0x12, 0x0b, 0x0a, 0x07, 0x4c, 0x61, 0x67, 0x67, 0x69, 0x6e, 0x67, 0x10, 0x06, 0x2a, 0x8d, 0x02, - 0x0a, 0x0a, 0x56, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0b, 0x0a, 0x07, - 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0x00, 0x12, 0x08, 0x0a, 0x04, 0x47, 0x54, 0x49, - 0x44, 0x10, 0x01, 0x12, 0x09, 0x0a, 0x05, 0x42, 0x45, 0x47, 0x49, 0x4e, 0x10, 0x02, 0x12, 0x0a, - 0x0a, 0x06, 0x43, 0x4f, 0x4d, 0x4d, 0x49, 0x54, 0x10, 0x03, 0x12, 0x0c, 0x0a, 0x08, 0x52, 0x4f, - 0x4c, 0x4c, 0x42, 0x41, 0x43, 0x4b, 0x10, 0x04, 0x12, 0x07, 0x0a, 0x03, 0x44, 0x44, 0x4c, 0x10, - 0x05, 0x12, 0x0a, 0x0a, 0x06, 0x49, 0x4e, 0x53, 0x45, 0x52, 0x54, 0x10, 0x06, 0x12, 0x0b, 0x0a, - 0x07, 0x52, 0x45, 0x50, 0x4c, 0x41, 0x43, 0x45, 0x10, 0x07, 0x12, 0x0a, 0x0a, 0x06, 0x55, 0x50, - 0x44, 0x41, 0x54, 0x45, 0x10, 0x08, 0x12, 0x0a, 0x0a, 0x06, 0x44, 0x45, 0x4c, 0x45, 0x54, 0x45, - 0x10, 0x09, 0x12, 0x07, 0x0a, 0x03, 0x53, 0x45, 0x54, 0x10, 0x0a, 0x12, 0x09, 0x0a, 0x05, 0x4f, - 0x54, 0x48, 0x45, 0x52, 0x10, 0x0b, 0x12, 0x07, 0x0a, 0x03, 0x52, 0x4f, 0x57, 0x10, 0x0c, 0x12, - 0x09, 0x0a, 0x05, 0x46, 0x49, 0x45, 0x4c, 0x44, 0x10, 0x0d, 0x12, 0x0d, 0x0a, 0x09, 0x48, 0x45, - 0x41, 0x52, 0x54, 0x42, 0x45, 0x41, 0x54, 0x10, 0x0e, 0x12, 0x09, 0x0a, 0x05, 0x56, 0x47, 0x54, - 0x49, 0x44, 0x10, 0x0f, 0x12, 0x0b, 0x0a, 0x07, 0x4a, 0x4f, 0x55, 0x52, 0x4e, 0x41, 0x4c, 0x10, - 0x10, 0x12, 0x0b, 0x0a, 0x07, 0x56, 0x45, 0x52, 0x53, 0x49, 0x4f, 0x4e, 0x10, 0x11, 0x12, 0x0a, - 0x0a, 0x06, 0x4c, 0x41, 0x53, 0x54, 0x50, 0x4b, 0x10, 0x12, 0x12, 0x0d, 0x0a, 0x09, 0x53, 0x41, - 0x56, 0x45, 0x50, 0x4f, 0x49, 0x4e, 0x54, 0x10, 0x13, 0x12, 0x12, 0x0a, 0x0e, 0x43, 0x4f, 0x50, - 0x59, 0x5f, 0x43, 0x4f, 0x4d, 0x50, 0x4c, 0x45, 0x54, 0x45, 0x44, 0x10, 0x14, 0x2a, 0x27, 0x0a, - 0x0d, 0x4d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0a, - 0x0a, 0x06, 0x54, 0x41, 0x42, 0x4c, 0x45, 0x53, 0x10, 0x00, 0x12, 0x0a, 0x0a, 0x06, 0x53, 0x48, - 0x41, 0x52, 0x44, 0x53, 0x10, 0x01, 0x42, 0x29, 0x5a, 0x27, 0x76, 0x69, 0x74, 0x65, 0x73, 0x73, - 0x2e, 0x69, 0x6f, 0x2f, 0x76, 0x69, 0x74, 0x65, 0x73, 0x73, 0x2f, 0x67, 0x6f, 0x2f, 0x76, 0x74, - 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, - 0x61, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x6f, 0x77, 0x53, 0x75, 0x62, 0x54, 0x79, 0x70, 0x65, 0x12, 0x08, 0x0a, 0x04, 0x4e, 0x6f, 0x6e, + 0x65, 0x10, 0x00, 0x12, 0x0b, 0x0a, 0x07, 0x50, 0x61, 0x72, 0x74, 0x69, 0x61, 0x6c, 0x10, 0x01, + 0x12, 0x0e, 0x0a, 0x0a, 0x41, 0x74, 0x6f, 0x6d, 0x69, 0x63, 0x43, 0x6f, 0x70, 0x79, 0x10, 0x02, + 0x2a, 0x71, 0x0a, 0x19, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x0b, 0x0a, + 0x07, 0x55, 0x6e, 0x6b, 0x6e, 0x6f, 0x77, 0x6e, 0x10, 0x00, 0x12, 0x08, 0x0a, 0x04, 0x49, 0x6e, + 0x69, 0x74, 0x10, 0x01, 0x12, 0x0b, 0x0a, 0x07, 0x53, 0x74, 0x6f, 0x70, 0x70, 0x65, 0x64, 0x10, + 0x02, 0x12, 0x0b, 0x0a, 0x07, 0x43, 0x6f, 0x70, 0x79, 0x69, 0x6e, 0x67, 0x10, 0x03, 0x12, 0x0b, + 0x0a, 0x07, 0x52, 0x75, 0x6e, 0x6e, 0x69, 0x6e, 0x67, 0x10, 0x04, 0x12, 0x09, 0x0a, 0x05, 0x45, + 0x72, 0x72, 0x6f, 0x72, 0x10, 0x05, 0x12, 0x0b, 0x0a, 0x07, 0x4c, 0x61, 0x67, 0x67, 0x69, 0x6e, + 0x67, 0x10, 0x06, 0x2a, 0x8d, 0x02, 0x0a, 0x0a, 0x56, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x54, 0x79, + 0x70, 0x65, 0x12, 0x0b, 0x0a, 0x07, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0x00, 0x12, + 0x08, 0x0a, 0x04, 0x47, 0x54, 0x49, 0x44, 0x10, 0x01, 0x12, 0x09, 0x0a, 0x05, 0x42, 0x45, 0x47, + 0x49, 0x4e, 0x10, 0x02, 0x12, 0x0a, 0x0a, 0x06, 0x43, 0x4f, 0x4d, 0x4d, 0x49, 0x54, 0x10, 0x03, + 0x12, 0x0c, 0x0a, 0x08, 0x52, 0x4f, 0x4c, 0x4c, 0x42, 0x41, 0x43, 0x4b, 0x10, 0x04, 0x12, 0x07, + 0x0a, 0x03, 0x44, 0x44, 0x4c, 0x10, 0x05, 0x12, 0x0a, 0x0a, 0x06, 0x49, 0x4e, 0x53, 0x45, 0x52, + 0x54, 0x10, 0x06, 0x12, 0x0b, 0x0a, 0x07, 0x52, 0x45, 0x50, 0x4c, 0x41, 0x43, 0x45, 0x10, 0x07, + 0x12, 0x0a, 0x0a, 0x06, 0x55, 0x50, 0x44, 0x41, 0x54, 0x45, 0x10, 0x08, 0x12, 0x0a, 0x0a, 0x06, + 0x44, 0x45, 0x4c, 0x45, 0x54, 0x45, 0x10, 0x09, 0x12, 0x07, 0x0a, 0x03, 0x53, 0x45, 0x54, 0x10, + 0x0a, 0x12, 0x09, 0x0a, 0x05, 0x4f, 0x54, 0x48, 0x45, 0x52, 0x10, 0x0b, 0x12, 0x07, 0x0a, 0x03, + 0x52, 0x4f, 0x57, 0x10, 0x0c, 0x12, 0x09, 0x0a, 0x05, 0x46, 0x49, 0x45, 0x4c, 0x44, 0x10, 0x0d, + 0x12, 0x0d, 0x0a, 0x09, 0x48, 0x45, 0x41, 0x52, 0x54, 0x42, 0x45, 0x41, 0x54, 0x10, 0x0e, 0x12, + 0x09, 0x0a, 0x05, 0x56, 0x47, 0x54, 0x49, 0x44, 0x10, 0x0f, 0x12, 0x0b, 0x0a, 0x07, 0x4a, 0x4f, + 0x55, 0x52, 0x4e, 0x41, 0x4c, 0x10, 0x10, 0x12, 0x0b, 0x0a, 0x07, 0x56, 0x45, 0x52, 0x53, 0x49, + 0x4f, 0x4e, 0x10, 0x11, 0x12, 0x0a, 0x0a, 0x06, 0x4c, 0x41, 0x53, 0x54, 0x50, 0x4b, 0x10, 0x12, + 0x12, 0x0d, 0x0a, 0x09, 0x53, 0x41, 0x56, 0x45, 0x50, 0x4f, 0x49, 0x4e, 0x54, 0x10, 0x13, 0x12, + 0x12, 0x0a, 0x0e, 0x43, 0x4f, 0x50, 0x59, 0x5f, 0x43, 0x4f, 0x4d, 0x50, 0x4c, 0x45, 0x54, 0x45, + 0x44, 0x10, 0x14, 0x2a, 0x27, 0x0a, 0x0d, 0x4d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x54, 0x79, 0x70, 0x65, 0x12, 0x0a, 0x0a, 0x06, 0x54, 0x41, 0x42, 0x4c, 0x45, 0x53, 0x10, 0x00, + 0x12, 0x0a, 0x0a, 0x06, 0x53, 0x48, 0x41, 0x52, 0x44, 0x53, 0x10, 0x01, 0x42, 0x29, 0x5a, 0x27, + 0x76, 0x69, 0x74, 0x65, 0x73, 0x73, 0x2e, 0x69, 0x6f, 0x2f, 0x76, 0x69, 0x74, 0x65, 0x73, 0x73, + 0x2f, 0x67, 0x6f, 0x2f, 0x76, 0x74, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x62, 0x69, 0x6e, + 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( diff --git a/go/vt/proto/binlogdata/binlogdata_vtproto.pb.go b/go/vt/proto/binlogdata/binlogdata_vtproto.pb.go index 379583b0354..fd1fe5e459d 100644 --- a/go/vt/proto/binlogdata/binlogdata_vtproto.pb.go +++ b/go/vt/proto/binlogdata/binlogdata_vtproto.pb.go @@ -541,7 +541,8 @@ func (m *MinimalTable) CloneVT() *MinimalTable { return (*MinimalTable)(nil) } r := &MinimalTable{ - Name: m.Name, + Name: m.Name, + PKIndexName: m.PKIndexName, } if rhs := m.Fields; rhs != nil { tmpContainer := make([]*query.Field, len(rhs)) @@ -2262,6 +2263,13 @@ func (m *MinimalTable) MarshalToSizedBufferVT(dAtA []byte) (int, error) { i -= len(m.unknownFields) copy(dAtA[i:], m.unknownFields) } + if len(m.PKIndexName) > 0 { + i -= len(m.PKIndexName) + copy(dAtA[i:], m.PKIndexName) + i = encodeVarint(dAtA, i, uint64(len(m.PKIndexName))) + i-- + dAtA[i] = 0x22 + } if len(m.PKColumns) > 0 { var pksize2 int for _, num := range m.PKColumns { @@ -3730,6 +3738,10 @@ func (m *MinimalTable) SizeVT() (n int) { } n += 1 + sov(uint64(l)) + l } + l = len(m.PKIndexName) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } n += len(m.unknownFields) return n } @@ -8052,6 +8064,38 @@ func (m *MinimalTable) UnmarshalVT(dAtA []byte) error { } else { return fmt.Errorf("proto: wrong wireType = %d for field PKColumns", wireType) } + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field PKIndexName", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.PKIndexName = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) diff --git a/go/vt/vttablet/tabletmanager/vreplication/vreplicator.go b/go/vt/vttablet/tabletmanager/vreplication/vreplicator.go index 0e63068d7a1..addf8de2019 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/vreplicator.go +++ b/go/vt/vttablet/tabletmanager/vreplication/vreplicator.go @@ -348,14 +348,14 @@ func (vr *vreplicator) buildColInfoMap(ctx context.Context) (map[string][]*Colum var pks []string if len(td.PrimaryKeyColumns) != 0 { - // Use the PK + // Use the PK. pks = td.PrimaryKeyColumns } else { - // Use a PK equivalent if one exists - if pks, err = vr.mysqld.GetPrimaryKeyEquivalentColumns(ctx, vr.dbClient.DBName(), td.Name); err != nil { + // Use a PK equivalent if one exists. + if pks, _, err = vr.mysqld.GetPrimaryKeyEquivalentColumns(ctx, vr.dbClient.DBName(), td.Name); err != nil { return nil, err } - // Fall back to using every column in the table if there's no PK or PKE + // Fall back to using every column in the table if there's no PK or PKE. if len(pks) == 0 { pks = td.Columns } diff --git a/go/vt/vttablet/tabletmanager/vreplication/vreplicator_test.go b/go/vt/vttablet/tabletmanager/vreplication/vreplicator_test.go index 346e6b67eb3..dd9649645c0 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/vreplicator_test.go +++ b/go/vt/vttablet/tabletmanager/vreplication/vreplicator_test.go @@ -20,7 +20,6 @@ import ( "context" "encoding/json" "fmt" - "reflect" "regexp" "strings" "sync" @@ -90,30 +89,34 @@ func TestRecalculatePKColsInfoByColumnNames(t *testing.T) { func TestPrimaryKeyEquivalentColumns(t *testing.T) { ctx := context.Background() tests := []struct { - name string - table string - ddl string - want []string - wantErr bool + name string + table string + ddl string + wantCols []string + wantIndex string + wantErr bool }{ { name: "WITHPK", table: "withpk_t", ddl: `CREATE TABLE withpk_t (pkid INT NOT NULL AUTO_INCREMENT, col1 VARCHAR(25), PRIMARY KEY (pkid))`, - want: []string{"pkid"}, + wantCols: []string{"pkid"}, + wantIndex: "PRIMARY", }, { - name: "0PKE", - table: "zeropke_t", - ddl: `CREATE TABLE zeropke_t (id INT NULL, col1 VARCHAR(25), UNIQUE KEY (id))`, - want: []string{}, + name: "0PKE", + table: "zeropke_t", + ddl: `CREATE TABLE zeropke_t (id INT NULL, col1 VARCHAR(25), UNIQUE KEY (id))`, + wantCols: []string{}, + wantIndex: "", }, { - name: "1PKE", - table: "onepke_t", - ddl: `CREATE TABLE onepke_t (id INT NOT NULL, col1 VARCHAR(25), UNIQUE KEY (id))`, - want: []string{"id"}, + name: "1PKE", + table: "onepke_t", + ddl: `CREATE TABLE onepke_t (id INT NOT NULL, col1 VARCHAR(25), UNIQUE KEY (id))`, + wantCols: []string{"id"}, + wantIndex: "id", }, { name: "3MULTICOL1PKE", @@ -122,7 +125,8 @@ func TestPrimaryKeyEquivalentColumns(t *testing.T) { col3 VARCHAR(25) NOT NULL, col4 VARCHAR(25), UNIQUE KEY c4_c2_c1 (col4, col2, col1), UNIQUE KEY c1_c2 (col1, col2), UNIQUE KEY c1_c2_c4 (col1, col2, col4), KEY nc1_nc2 (col1, col2))`, - want: []string{"col1", "col2"}, + wantCols: []string{"col1", "col2"}, + wantIndex: "c1_c2", }, { name: "3MULTICOL2PKE", @@ -130,7 +134,8 @@ func TestPrimaryKeyEquivalentColumns(t *testing.T) { ddl: `CREATE TABLE twomcpke_t (col1 VARCHAR(25) NOT NULL, col2 VARCHAR(25) NOT NULL, col3 VARCHAR(25) NOT NULL, col4 VARCHAR(25), UNIQUE KEY (col4), UNIQUE KEY c4_c2_c1 (col4, col2, col1), UNIQUE KEY c1_c2_c3 (col1, col2, col3), UNIQUE KEY c1_c2 (col1, col2))`, - want: []string{"col1", "col2"}, + wantCols: []string{"col1", "col2"}, + wantIndex: "c1_c2", }, { name: "1INTPKE1CHARPKE", @@ -138,48 +143,52 @@ func TestPrimaryKeyEquivalentColumns(t *testing.T) { ddl: `CREATE TABLE oneintpke1charpke_t (col1 VARCHAR(25) NOT NULL, col2 VARCHAR(25) NOT NULL, col3 VARCHAR(25) NOT NULL, id1 INT NOT NULL, id2 INT NOT NULL, UNIQUE KEY c1_c2 (col1, col2), UNIQUE KEY id1_id2 (id1, id2))`, - want: []string{"id1", "id2"}, + wantCols: []string{"id1", "id2"}, + wantIndex: "id1_id2", }, { name: "INTINTVSVCHAR", table: "twointvsvcharpke_t", ddl: `CREATE TABLE twointvsvcharpke_t (col1 VARCHAR(25) NOT NULL, id1 INT NOT NULL, id2 INT NOT NULL, UNIQUE KEY c1 (col1), UNIQUE KEY id1_id2 (id1, id2))`, - want: []string{"id1", "id2"}, + wantCols: []string{"id1", "id2"}, + wantIndex: "id1_id2", }, { name: "TINYINTVSBIGINT", table: "tinyintvsbigint_t", ddl: `CREATE TABLE tinyintvsbigint_t (tid1 TINYINT NOT NULL, id1 INT NOT NULL, UNIQUE KEY tid1 (tid1), UNIQUE KEY id1 (id1))`, - want: []string{"tid1"}, + wantCols: []string{"tid1"}, + wantIndex: "tid1", }, { name: "VCHARINTVSINT2VARCHAR", table: "vcharintvsinttwovchar_t", ddl: `CREATE TABLE vcharintvsinttwovchar_t (id1 INT NOT NULL, col1 VARCHAR(25) NOT NULL, col2 VARCHAR(25) NOT NULL, UNIQUE KEY col1_id1 (col1, id1), UNIQUE KEY id1_col1_col2 (id1, col1, col2))`, - want: []string{"col1", "id1"}, + wantCols: []string{"col1", "id1"}, + wantIndex: "col1_id1", }, { name: "VCHARVSINT3", table: "vcharvsintthree_t", ddl: `CREATE TABLE vcharvsintthree_t (id1 INT NOT NULL, id2 INT NOT NULL, id3 INT NOT NULL, col1 VARCHAR(50) NOT NULL, UNIQUE KEY col1 (col1), UNIQUE KEY id1_id2_id3 (id1, id2, id3))`, - want: []string{"id1", "id2", "id3"}, + wantCols: []string{"id1", "id2", "id3"}, + wantIndex: "id1_id2_id3", }, } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { require.NoError(t, env.Mysqld.ExecuteSuperQuery(ctx, tt.ddl)) - got, err := env.Mysqld.GetPrimaryKeyEquivalentColumns(ctx, env.Dbcfgs.DBName, tt.table) + cols, indexName, err := env.Mysqld.GetPrimaryKeyEquivalentColumns(ctx, env.Dbcfgs.DBName, tt.table) if (err != nil) != tt.wantErr { t.Errorf("Mysqld.GetPrimaryKeyEquivalentColumns() error = %v, wantErr %v", err, tt.wantErr) return } - if !reflect.DeepEqual(got, tt.want) { - t.Errorf("Mysqld.GetPrimaryKeyEquivalentColumns() = %v, want %v", got, tt.want) - } + require.Equalf(t, cols, tt.wantCols, "Mysqld.GetPrimaryKeyEquivalentColumns() columns = %v, want %v", cols, tt.wantCols) + require.Equalf(t, indexName, tt.wantIndex, "Mysqld.GetPrimaryKeyEquivalentColumns() index = %v, want %v", indexName, tt.wantIndex) }) } } diff --git a/go/vt/vttablet/tabletserver/vstreamer/engine.go b/go/vt/vttablet/tabletserver/vstreamer/engine.go index adbd117c2f2..2862601bf1b 100644 --- a/go/vt/vttablet/tabletserver/vstreamer/engine.go +++ b/go/vt/vttablet/tabletserver/vstreamer/engine.go @@ -592,10 +592,13 @@ func (vse *Engine) getMySQLEndpoint(ctx context.Context, db dbconfigs.Connector) // and maps the column names to field indexes in the MinimalTable struct. func (vse *Engine) mapPKEquivalentCols(ctx context.Context, table *binlogdatapb.MinimalTable) ([]int, error) { mysqld := mysqlctl.NewMysqld(vse.env.Config().DB) - pkeColNames, err := mysqld.GetPrimaryKeyEquivalentColumns(ctx, vse.env.Config().DB.DBName, table.Name) + pkeColNames, indexName, err := mysqld.GetPrimaryKeyEquivalentColumns(ctx, vse.env.Config().DB.DBName, table.Name) if err != nil { return nil, err } + if len(pkeColNames) > 0 && indexName != "" { + table.PKIndexName = indexName + } pkeCols := make([]int, len(pkeColNames)) matches := 0 for n, field := range table.Fields { diff --git a/go/vt/vttablet/tabletserver/vstreamer/rowstreamer.go b/go/vt/vttablet/tabletserver/vstreamer/rowstreamer.go index 00194b5cc0b..25a5cd7bf87 100644 --- a/go/vt/vttablet/tabletserver/vstreamer/rowstreamer.go +++ b/go/vt/vttablet/tabletserver/vstreamer/rowstreamer.go @@ -24,6 +24,7 @@ import ( "vitess.io/vitess/go/mysql/collations" "vitess.io/vitess/go/mysql/replication" + "vitess.io/vitess/go/sqlescape" "vitess.io/vitess/go/sqltypes" "vitess.io/vitess/go/textutil" "vitess.io/vitess/go/timer" @@ -118,7 +119,7 @@ func (rs *rowStreamer) Cancel() { } func (rs *rowStreamer) Stream() error { - // Ensure sh is Open. If vttablet came up in a non_serving role, + // Ensure se is Open. If vttablet came up in a non_serving role, // the schema engine may not have been initialized. if err := rs.se.Open(); err != nil { return err @@ -154,11 +155,11 @@ func (rs *rowStreamer) buildPlan() error { // with vitess migrations, based on vreplication. // Vitess migrations use an elaborate cut-over flow where tables are swapped away while traffic is // being blocked. The RENAME flow is such that at some point the table is renamed away, leaving a - // "puncture"; this is an event the is captured by vstreamer. The completion of the flow fixes the + // "puncture"; this is an event that is captured by vstreamer. The completion of the flow fixes the // puncture, and places a new table under the original table's name, but the way it is done does not // cause vstreamer to refresh schema state. - // there is therefore a reproducable valid sequence of events where vstreamer thinks a table does not exist, - // where it in fact does exist. + // There is therefore a reproducable valid sequence of events where vstreamer thinks a table does not + // exist, where it in fact does exist. // For this reason we give vstreamer a "second chance" to review the up-to-date state of the schema. // In the future, we will reduce this operation to reading a single table rather than the entire schema. rs.se.ReloadAt(context.Background(), replication.Position{}) @@ -198,7 +199,7 @@ func (rs *rowStreamer) buildPlan() error { if err != nil { return err } - rs.sendQuery, err = rs.buildSelect() + rs.sendQuery, err = rs.buildSelect(st) if err != nil { return err } @@ -226,13 +227,13 @@ func (rs *rowStreamer) buildPKColumns(st *binlogdatapb.MinimalTable) ([]int, err } var pkColumns = make([]int, 0) if len(st.PKColumns) == 0 { - // Use a PK equivalent if one exists + // Use a PK equivalent if one exists. pkColumns, err := rs.vse.mapPKEquivalentCols(rs.ctx, st) if err == nil && len(pkColumns) != 0 { return pkColumns, nil } - // Fall back to using every column in the table if there's no PK or PKE + // Fall back to using every column in the table if there's no PK or PKE. pkColumns = make([]int, len(st.Fields)) for i := range st.Fields { pkColumns[i] = i @@ -245,10 +246,11 @@ func (rs *rowStreamer) buildPKColumns(st *binlogdatapb.MinimalTable) ([]int, err } pkColumns = append(pkColumns, int(pk)) } + st.PKIndexName = "PRIMARY" return pkColumns, nil } -func (rs *rowStreamer) buildSelect() (string, error) { +func (rs *rowStreamer) buildSelect(st *binlogdatapb.MinimalTable) (string, error) { buf := sqlparser.NewTrackedBuffer(nil) // We could have used select *, but being explicit is more predictable. buf.Myprintf("select ") @@ -263,18 +265,28 @@ func (rs *rowStreamer) buildSelect() (string, error) { } prefix = ", " } - buf.Myprintf(" from %v", sqlparser.NewIdentifierCS(rs.plan.Table.Name)) + // If we know the index name that we should be using then tell MySQL + // to use it if possible. This helps to ensure that we are able to + // leverage the ordering from the index itself and avoid having to + // do a FILESORT of all the results. This index should contain all + // of the PK columns which are used in the ORDER BY clause below. + var indexHint string + if st.PKIndexName != "" { + indexHint = fmt.Sprintf(" force index (%s)", + sqlescape.EscapeID(sqlescape.UnescapeID(st.PKIndexName))) + } + buf.Myprintf(" from %v%s", sqlparser.NewIdentifierCS(rs.plan.Table.Name), indexHint) if len(rs.lastpk) != 0 { if len(rs.lastpk) != len(rs.pkColumns) { return "", fmt.Errorf("primary key values don't match length: %v vs %v", rs.lastpk, rs.pkColumns) } buf.WriteString(" where ") prefix := "" - // This loop handles the case for composite pks. For example, + // This loop handles the case for composite PKs. For example, // if lastpk was (1,2), the where clause would be: // (col1 = 1 and col2 > 2) or (col1 > 1). // A tuple inequality like (col1,col2) > (1,2) ends up - // being a full table scan for mysql. + // being a full table scan for MySQL. for lastcol := len(rs.pkColumns) - 1; lastcol >= 0; lastcol-- { buf.Myprintf("%s(", prefix) prefix = " or " diff --git a/go/vt/vttablet/tabletserver/vstreamer/rowstreamer_test.go b/go/vt/vttablet/tabletserver/vstreamer/rowstreamer_test.go index 6ba5a3a5d02..fcc179dbc5b 100644 --- a/go/vt/vttablet/tabletserver/vstreamer/rowstreamer_test.go +++ b/go/vt/vttablet/tabletserver/vstreamer/rowstreamer_test.go @@ -51,6 +51,9 @@ func TestStreamRowsScan(t *testing.T) { // Three-column PK "create table t4(id1 int, id2 int, id3 int, val varbinary(128), primary key(id1, id2, id3))", "insert into t4 values (1, 2, 3, 'aaa'), (2, 3, 4, 'bbb')", + // PK equivalent + "create table t5(id1 int not null, id2 int not null, id3 int not null, val varbinary(128), unique key id1_id2_id3 (id1, id2, id3))", + "insert into t5 values (1, 2, 3, 'aaa'), (2, 3, 4, 'bbb')", }) defer execStatements(t, []string{ @@ -58,6 +61,7 @@ func TestStreamRowsScan(t *testing.T) { "drop table t2", "drop table t3", "drop table t4", + "drop table t5", }) engine.se.Reload(context.Background()) @@ -67,7 +71,7 @@ func TestStreamRowsScan(t *testing.T) { `fields:{name:"1" type:INT64 charset:63} pkfields:{name:"id" type:INT32 charset:63}`, `rows:{lengths:1 values:"1"} rows:{lengths:1 values:"1"} lastpk:{lengths:1 values:"2"}`, } - wantQuery := "select id, val from t1 order by id" + wantQuery := "select id, val from t1 force index (`PRIMARY`) order by id" checkStream(t, "select 1 from t1", nil, wantQuery, wantStream) // t1: simulates rollup, with non-pk column @@ -75,7 +79,7 @@ func TestStreamRowsScan(t *testing.T) { `fields:{name:"1" type:INT64 charset:63} fields:{name:"val" type:VARBINARY table:"t1" org_table:"t1" database:"vttest" org_name:"val" column_length:128 charset:63 column_type:"varbinary(128)"} pkfields:{name:"id" type:INT32 charset:63}`, `rows:{lengths:1 lengths:3 values:"1aaa"} rows:{lengths:1 lengths:3 values:"1bbb"} lastpk:{lengths:1 values:"2"}`, } - wantQuery = "select id, val from t1 order by id" + wantQuery = "select id, val from t1 force index (`PRIMARY`) order by id" checkStream(t, "select 1, val from t1", nil, wantQuery, wantStream) // t1: simulates rollup, with pk and non-pk column @@ -83,7 +87,7 @@ func TestStreamRowsScan(t *testing.T) { `fields:{name:"1" type:INT64 charset:63} fields:{name:"id" type:INT32 table:"t1" org_table:"t1" database:"vttest" org_name:"id" column_length:11 charset:63 column_type:"int(11)"} fields:{name:"val" type:VARBINARY table:"t1" org_table:"t1" database:"vttest" org_name:"val" column_length:128 charset:63 column_type:"varbinary(128)"} pkfields:{name:"id" type:INT32 charset:63}`, `rows:{lengths:1 lengths:1 lengths:3 values:"11aaa"} rows:{lengths:1 lengths:1 lengths:3 values:"12bbb"} lastpk:{lengths:1 values:"2"}`, } - wantQuery = "select id, val from t1 order by id" + wantQuery = "select id, val from t1 force index (`PRIMARY`) order by id" checkStream(t, "select 1, id, val from t1", nil, wantQuery, wantStream) // t1: no pk in select list @@ -91,7 +95,7 @@ func TestStreamRowsScan(t *testing.T) { `fields:{name:"val" type:VARBINARY table:"t1" org_table:"t1" database:"vttest" org_name:"val" column_length:128 charset:63 column_type:"varbinary(128)"} pkfields:{name:"id" type:INT32 charset:63}`, `rows:{lengths:3 values:"aaa"} rows:{lengths:3 values:"bbb"} lastpk:{lengths:1 values:"2"}`, } - wantQuery = "select id, val from t1 order by id" + wantQuery = "select id, val from t1 force index (`PRIMARY`) order by id" checkStream(t, "select val from t1", nil, wantQuery, wantStream) // t1: all rows @@ -99,7 +103,7 @@ func TestStreamRowsScan(t *testing.T) { `fields:{name:"id" type:INT32 table:"t1" org_table:"t1" database:"vttest" org_name:"id" column_length:11 charset:63 column_type:"int(11)"} fields:{name:"val" type:VARBINARY table:"t1" org_table:"t1" database:"vttest" org_name:"val" column_length:128 charset:63 column_type:"varbinary(128)"} pkfields:{name:"id" type:INT32 charset:63}`, `rows:{lengths:1 lengths:3 values:"1aaa"} rows:{lengths:1 lengths:3 values:"2bbb"} lastpk:{lengths:1 values:"2"}`, } - wantQuery = "select id, val from t1 order by id" + wantQuery = "select id, val from t1 force index (`PRIMARY`) order by id" checkStream(t, "select * from t1", nil, wantQuery, wantStream) // t1: lastpk=1 @@ -107,7 +111,7 @@ func TestStreamRowsScan(t *testing.T) { `fields:{name:"id" type:INT32 table:"t1" org_table:"t1" database:"vttest" org_name:"id" column_length:11 charset:63 column_type:"int(11)"} fields:{name:"val" type:VARBINARY table:"t1" org_table:"t1" database:"vttest" org_name:"val" column_length:128 charset:63 column_type:"varbinary(128)"} pkfields:{name:"id" type:INT32 charset:63}`, `rows:{lengths:1 lengths:3 values:"2bbb"} lastpk:{lengths:1 values:"2"}`, } - wantQuery = "select id, val from t1 where (id > 1) order by id" + wantQuery = "select id, val from t1 force index (`PRIMARY`) where (id > 1) order by id" checkStream(t, "select * from t1", []sqltypes.Value{sqltypes.NewInt64(1)}, wantQuery, wantStream) // t1: different column ordering @@ -115,7 +119,7 @@ func TestStreamRowsScan(t *testing.T) { `fields:{name:"val" type:VARBINARY table:"t1" org_table:"t1" database:"vttest" org_name:"val" column_length:128 charset:63 column_type:"varbinary(128)"} fields:{name:"id" type:INT32 table:"t1" org_table:"t1" database:"vttest" org_name:"id" column_length:11 charset:63 column_type:"int(11)"} pkfields:{name:"id" type:INT32 charset:63}`, `rows:{lengths:3 lengths:1 values:"aaa1"} rows:{lengths:3 lengths:1 values:"bbb2"} lastpk:{lengths:1 values:"2"}`, } - wantQuery = "select id, val from t1 order by id" + wantQuery = "select id, val from t1 force index (`PRIMARY`) order by id" checkStream(t, "select val, id from t1", nil, wantQuery, wantStream) // t2: all rows @@ -123,7 +127,7 @@ func TestStreamRowsScan(t *testing.T) { `fields:{name:"id1" type:INT32 table:"t2" org_table:"t2" database:"vttest" org_name:"id1" column_length:11 charset:63 column_type:"int(11)"} fields:{name:"id2" type:INT32 table:"t2" org_table:"t2" database:"vttest" org_name:"id2" column_length:11 charset:63 column_type:"int(11)"} fields:{name:"val" type:VARBINARY table:"t2" org_table:"t2" database:"vttest" org_name:"val" column_length:128 charset:63 column_type:"varbinary(128)"} pkfields:{name:"id1" type:INT32 charset:63} pkfields:{name:"id2" type:INT32 charset:63}`, `rows:{lengths:1 lengths:1 lengths:3 values:"12aaa"} rows:{lengths:1 lengths:1 lengths:3 values:"13bbb"} lastpk:{lengths:1 lengths:1 values:"13"}`, } - wantQuery = "select id1, id2, val from t2 order by id1, id2" + wantQuery = "select id1, id2, val from t2 force index (`PRIMARY`) order by id1, id2" checkStream(t, "select * from t2", nil, wantQuery, wantStream) // t2: lastpk=1,2 @@ -131,7 +135,7 @@ func TestStreamRowsScan(t *testing.T) { `fields:{name:"id1" type:INT32 table:"t2" org_table:"t2" database:"vttest" org_name:"id1" column_length:11 charset:63 column_type:"int(11)"} fields:{name:"id2" type:INT32 table:"t2" org_table:"t2" database:"vttest" org_name:"id2" column_length:11 charset:63 column_type:"int(11)"} fields:{name:"val" type:VARBINARY table:"t2" org_table:"t2" database:"vttest" org_name:"val" column_length:128 charset:63 column_type:"varbinary(128)"} pkfields:{name:"id1" type:INT32 charset:63} pkfields:{name:"id2" type:INT32 charset:63}`, `rows:{lengths:1 lengths:1 lengths:3 values:"13bbb"} lastpk:{lengths:1 lengths:1 values:"13"}`, } - wantQuery = "select id1, id2, val from t2 where (id1 = 1 and id2 > 2) or (id1 > 1) order by id1, id2" + wantQuery = "select id1, id2, val from t2 force index (`PRIMARY`) where (id1 = 1 and id2 > 2) or (id1 > 1) order by id1, id2" checkStream(t, "select * from t2", []sqltypes.Value{sqltypes.NewInt64(1), sqltypes.NewInt64(2)}, wantQuery, wantStream) // t3: all rows @@ -155,7 +159,7 @@ func TestStreamRowsScan(t *testing.T) { `fields:{name:"id1" type:INT32 table:"t4" org_table:"t4" database:"vttest" org_name:"id1" column_length:11 charset:63 column_type:"int(11)"} fields:{name:"id2" type:INT32 table:"t4" org_table:"t4" database:"vttest" org_name:"id2" column_length:11 charset:63 column_type:"int(11)"} fields:{name:"id3" type:INT32 table:"t4" org_table:"t4" database:"vttest" org_name:"id3" column_length:11 charset:63 column_type:"int(11)"} fields:{name:"val" type:VARBINARY table:"t4" org_table:"t4" database:"vttest" org_name:"val" column_length:128 charset:63 column_type:"varbinary(128)"} pkfields:{name:"id1" type:INT32 charset:63} pkfields:{name:"id2" type:INT32 charset:63} pkfields:{name:"id3" type:INT32 charset:63}`, `rows:{lengths:1 lengths:1 lengths:1 lengths:3 values:"123aaa"} rows:{lengths:1 lengths:1 lengths:1 lengths:3 values:"234bbb"} lastpk:{lengths:1 lengths:1 lengths:1 values:"234"}`, } - wantQuery = "select id1, id2, id3, val from t4 order by id1, id2, id3" + wantQuery = "select id1, id2, id3, val from t4 force index (`PRIMARY`) order by id1, id2, id3" checkStream(t, "select * from t4", nil, wantQuery, wantStream) // t4: lastpk: 1,2,3 @@ -163,9 +167,17 @@ func TestStreamRowsScan(t *testing.T) { `fields:{name:"id1" type:INT32 table:"t4" org_table:"t4" database:"vttest" org_name:"id1" column_length:11 charset:63 column_type:"int(11)"} fields:{name:"id2" type:INT32 table:"t4" org_table:"t4" database:"vttest" org_name:"id2" column_length:11 charset:63 column_type:"int(11)"} fields:{name:"id3" type:INT32 table:"t4" org_table:"t4" database:"vttest" org_name:"id3" column_length:11 charset:63 column_type:"int(11)"} fields:{name:"val" type:VARBINARY table:"t4" org_table:"t4" database:"vttest" org_name:"val" column_length:128 charset:63 column_type:"varbinary(128)"} pkfields:{name:"id1" type:INT32 charset:63} pkfields:{name:"id2" type:INT32 charset:63} pkfields:{name:"id3" type:INT32 charset:63}`, `rows:{lengths:1 lengths:1 lengths:1 lengths:3 values:"234bbb"} lastpk:{lengths:1 lengths:1 lengths:1 values:"234"}`, } - wantQuery = "select id1, id2, id3, val from t4 where (id1 = 1 and id2 = 2 and id3 > 3) or (id1 = 1 and id2 > 2) or (id1 > 1) order by id1, id2, id3" + wantQuery = "select id1, id2, id3, val from t4 force index (`PRIMARY`) where (id1 = 1 and id2 = 2 and id3 > 3) or (id1 = 1 and id2 > 2) or (id1 > 1) order by id1, id2, id3" checkStream(t, "select * from t4", []sqltypes.Value{sqltypes.NewInt64(1), sqltypes.NewInt64(2), sqltypes.NewInt64(3)}, wantQuery, wantStream) + // t5: No PK, but a PKE + wantStream = []string{ + `fields:{name:"id1" type:INT32 table:"t5" org_table:"t5" database:"vttest" org_name:"id1" column_length:11 charset:63 column_type:"int(11)"} fields:{name:"id2" type:INT32 table:"t5" org_table:"t5" database:"vttest" org_name:"id2" column_length:11 charset:63 column_type:"int(11)"} fields:{name:"id3" type:INT32 table:"t5" org_table:"t5" database:"vttest" org_name:"id3" column_length:11 charset:63 column_type:"int(11)"} fields:{name:"val" type:VARBINARY table:"t5" org_table:"t5" database:"vttest" org_name:"val" column_length:128 charset:63 column_type:"varbinary(128)"} pkfields:{name:"id1" type:INT32 charset:63} pkfields:{name:"id2" type:INT32 charset:63} pkfields:{name:"id3" type:INT32 charset:63}`, + `rows:{lengths:1 lengths:1 lengths:1 lengths:3 values:"234bbb"} lastpk:{lengths:1 lengths:1 lengths:1 values:"234"}`, + } + wantQuery = "select id1, id2, id3, val from t5 force index (`id1_id2_id3`) where (id1 = 1 and id2 = 2 and id3 > 3) or (id1 = 1 and id2 > 2) or (id1 > 1) order by id1, id2, id3" + checkStream(t, "select * from t5", []sqltypes.Value{sqltypes.NewInt64(1), sqltypes.NewInt64(2), sqltypes.NewInt64(3)}, wantQuery, wantStream) + // t1: test for unsupported integer literal wantError := "only the integer literal 1 is supported" expectStreamError(t, "select 2 from t1", wantError) @@ -258,7 +270,7 @@ func TestStreamRowsKeyRange(t *testing.T) { `fields:{name:"id1" type:INT32 table:"t1" org_table:"t1" database:"vttest" org_name:"id1" column_length:11 charset:63 column_type:"int(11)"} fields:{name:"val" type:VARBINARY table:"t1" org_table:"t1" database:"vttest" org_name:"val" column_length:128 charset:63 column_type:"varbinary(128)"} pkfields:{name:"id1" type:INT32 charset:63}`, `rows:{lengths:1 lengths:3 values:"1aaa"} lastpk:{lengths:1 values:"6"}`, } - wantQuery := "select id1, val from t1 order by id1" + wantQuery := "select id1, val from t1 force index (`PRIMARY`) order by id1" checkStream(t, "select * from t1 where in_keyrange('-80')", nil, wantQuery, wantStream) } @@ -290,7 +302,7 @@ func TestStreamRowsFilterInt(t *testing.T) { `fields:{name:"id1" type:INT32 table:"t1" org_table:"t1" database:"vttest" org_name:"id1" column_length:11 charset:63 column_type:"int(11)"} fields:{name:"val" type:VARBINARY table:"t1" org_table:"t1" database:"vttest" org_name:"val" column_length:128 charset:63 column_type:"varbinary(128)"} pkfields:{name:"id1" type:INT32 charset:63}`, `rows:{lengths:1 lengths:3 values:"1aaa"} rows:{lengths:1 lengths:3 values:"4ddd"} lastpk:{lengths:1 values:"5"}`, } - wantQuery := "select id1, id2, val from t1 order by id1" + wantQuery := "select id1, id2, val from t1 force index (`PRIMARY`) order by id1" checkStream(t, "select id1, val from t1 where id2 = 100", nil, wantQuery, wantStream) require.Equal(t, int64(0), engine.rowStreamerNumPackets.Get()) require.Equal(t, int64(2), engine.rowStreamerNumRows.Get()) @@ -323,7 +335,7 @@ func TestStreamRowsFilterVarBinary(t *testing.T) { `fields:{name:"id1" type:INT32 table:"t1" org_table:"t1" database:"vttest" org_name:"id1" column_length:11 charset:63 column_type:"int(11)"} fields:{name:"val" type:VARBINARY table:"t1" org_table:"t1" database:"vttest" org_name:"val" column_length:128 charset:63 column_type:"varbinary(128)"} pkfields:{name:"id1" type:INT32 charset:63}`, `rows:{lengths:1 lengths:6 values:"2newton"} rows:{lengths:1 lengths:6 values:"3newton"} rows:{lengths:1 lengths:6 values:"5newton"} lastpk:{lengths:1 values:"6"}`, } - wantQuery := "select id1, val from t1 order by id1" + wantQuery := "select id1, val from t1 force index (`PRIMARY`) order by id1" checkStream(t, "select id1, val from t1 where val = 'newton'", nil, wantQuery, wantStream) } @@ -351,7 +363,7 @@ func TestStreamRowsMultiPacket(t *testing.T) { `rows:{lengths:1 lengths:10 values:"42345678901"} lastpk:{lengths:1 values:"4"}`, `rows:{lengths:1 lengths:1 values:"52"} lastpk:{lengths:1 values:"5"}`, } - wantQuery := "select id, val from t1 order by id" + wantQuery := "select id, val from t1 force index (`PRIMARY`) order by id" checkStream(t, "select * from t1", nil, wantQuery, wantStream) } diff --git a/go/vt/vttablet/tabletserver/vstreamer/uvstreamer_flaky_test.go b/go/vt/vttablet/tabletserver/vstreamer/uvstreamer_flaky_test.go index 34f860d2120..203052e981e 100644 --- a/go/vt/vttablet/tabletserver/vstreamer/uvstreamer_flaky_test.go +++ b/go/vt/vttablet/tabletserver/vstreamer/uvstreamer_flaky_test.go @@ -126,7 +126,7 @@ func TestVStreamCopyFilterValidations(t *testing.T) { require.Error(t, uvs.init(), expectedError) return } - require.Equal(t, len(expected), len(uvs.plans)) + require.Equalf(t, len(expected), len(uvs.plans), "Plans: %+v", uvs.plans) for _, tableName := range expected { require.True(t, uvs.plans[tableName].tablePK.TableName == tableName) if tablePKs == nil { diff --git a/proto/binlogdata.proto b/proto/binlogdata.proto index 660f8fedfce..b1f36261522 100644 --- a/proto/binlogdata.proto +++ b/proto/binlogdata.proto @@ -453,6 +453,10 @@ message MinimalTable { string name = 1; repeated query.Field fields = 2; repeated int64 p_k_columns = 3; + // This will be PRIMARY when the actual primary key is used and it + // will be the name of the Primary Key equivalent if one is used + // instead. Otherwise it will be empty. + string p_k_index_name = 4; } message MinimalSchema { diff --git a/web/vtadmin/src/proto/vtadmin.d.ts b/web/vtadmin/src/proto/vtadmin.d.ts index 8a917c4391a..6da506ae677 100644 --- a/web/vtadmin/src/proto/vtadmin.d.ts +++ b/web/vtadmin/src/proto/vtadmin.d.ts @@ -31016,6 +31016,9 @@ export namespace binlogdata { /** MinimalTable p_k_columns */ p_k_columns?: ((number|Long)[]|null); + + /** MinimalTable p_k_index_name */ + p_k_index_name?: (string|null); } /** Represents a MinimalTable. */ @@ -31036,6 +31039,9 @@ export namespace binlogdata { /** MinimalTable p_k_columns. */ public p_k_columns: (number|Long)[]; + /** MinimalTable p_k_index_name. */ + public p_k_index_name: string; + /** * Creates a new MinimalTable instance using the specified properties. * @param [properties] Properties to set diff --git a/web/vtadmin/src/proto/vtadmin.js b/web/vtadmin/src/proto/vtadmin.js index 20407de7633..e5273e0d955 100644 --- a/web/vtadmin/src/proto/vtadmin.js +++ b/web/vtadmin/src/proto/vtadmin.js @@ -72970,6 +72970,7 @@ export const binlogdata = $root.binlogdata = (() => { * @property {string|null} [name] MinimalTable name * @property {Array.|null} [fields] MinimalTable fields * @property {Array.|null} [p_k_columns] MinimalTable p_k_columns + * @property {string|null} [p_k_index_name] MinimalTable p_k_index_name */ /** @@ -73013,6 +73014,14 @@ export const binlogdata = $root.binlogdata = (() => { */ MinimalTable.prototype.p_k_columns = $util.emptyArray; + /** + * MinimalTable p_k_index_name. + * @member {string} p_k_index_name + * @memberof binlogdata.MinimalTable + * @instance + */ + MinimalTable.prototype.p_k_index_name = ""; + /** * Creates a new MinimalTable instance using the specified properties. * @function create @@ -73048,6 +73057,8 @@ export const binlogdata = $root.binlogdata = (() => { writer.int64(message.p_k_columns[i]); writer.ldelim(); } + if (message.p_k_index_name != null && Object.hasOwnProperty.call(message, "p_k_index_name")) + writer.uint32(/* id 4, wireType 2 =*/34).string(message.p_k_index_name); return writer; }; @@ -73103,6 +73114,10 @@ export const binlogdata = $root.binlogdata = (() => { message.p_k_columns.push(reader.int64()); break; } + case 4: { + message.p_k_index_name = reader.string(); + break; + } default: reader.skipType(tag & 7); break; @@ -73157,6 +73172,9 @@ export const binlogdata = $root.binlogdata = (() => { if (!$util.isInteger(message.p_k_columns[i]) && !(message.p_k_columns[i] && $util.isInteger(message.p_k_columns[i].low) && $util.isInteger(message.p_k_columns[i].high))) return "p_k_columns: integer|Long[] expected"; } + if (message.p_k_index_name != null && message.hasOwnProperty("p_k_index_name")) + if (!$util.isString(message.p_k_index_name)) + return "p_k_index_name: string expected"; return null; }; @@ -73198,6 +73216,8 @@ export const binlogdata = $root.binlogdata = (() => { else if (typeof object.p_k_columns[i] === "object") message.p_k_columns[i] = new $util.LongBits(object.p_k_columns[i].low >>> 0, object.p_k_columns[i].high >>> 0).toNumber(); } + if (object.p_k_index_name != null) + message.p_k_index_name = String(object.p_k_index_name); return message; }; @@ -73218,8 +73238,10 @@ export const binlogdata = $root.binlogdata = (() => { object.fields = []; object.p_k_columns = []; } - if (options.defaults) + if (options.defaults) { object.name = ""; + object.p_k_index_name = ""; + } if (message.name != null && message.hasOwnProperty("name")) object.name = message.name; if (message.fields && message.fields.length) { @@ -73235,6 +73257,8 @@ export const binlogdata = $root.binlogdata = (() => { else object.p_k_columns[j] = options.longs === String ? $util.Long.prototype.toString.call(message.p_k_columns[j]) : options.longs === Number ? new $util.LongBits(message.p_k_columns[j].low >>> 0, message.p_k_columns[j].high >>> 0).toNumber() : message.p_k_columns[j]; } + if (message.p_k_index_name != null && message.hasOwnProperty("p_k_index_name")) + object.p_k_index_name = message.p_k_index_name; return object; };