From 7e10b5dc90b45d517f80450cac7697bb5bb1b57d Mon Sep 17 00:00:00 2001 From: Matt Lord Date: Tue, 29 Aug 2023 22:57:39 -0400 Subject: [PATCH 1/7] Ensure that rowstreamer uses optimal index when possible This is the index that should contain the columns specified in the ORDER BY clause and allows us to leverage the record ordering in the index and avoid having to do a filesort. Signed-off-by: Matt Lord --- go/vt/mysqlctl/fakemysqldaemon.go | 4 +- go/vt/mysqlctl/mysql_daemon.go | 2 +- go/vt/mysqlctl/schema.go | 12 +- go/vt/proto/binlogdata/binlogdata.pb.go | 283 +++++++++--------- .../proto/binlogdata/binlogdata_vtproto.pb.go | 43 +++ .../tabletmanager/vreplication/vreplicator.go | 2 +- .../vreplication/vreplicator_test.go | 61 ++-- .../vttablet/tabletserver/vstreamer/engine.go | 5 +- .../tabletserver/vstreamer/rowstreamer.go | 16 +- .../vstreamer/rowstreamer_test.go | 41 ++- proto/binlogdata.proto | 4 + web/vtadmin/src/proto/vtadmin.d.ts | 6 + web/vtadmin/src/proto/vtadmin.js | 26 +- 13 files changed, 315 insertions(+), 190 deletions(-) diff --git a/go/vt/mysqlctl/fakemysqldaemon.go b/go/vt/mysqlctl/fakemysqldaemon.go index 39ecca84156..f50d118d3b5 100644 --- a/go/vt/mysqlctl/fakemysqldaemon.go +++ b/go/vt/mysqlctl/fakemysqldaemon.go @@ -601,8 +601,8 @@ func (fmd *FakeMysqlDaemon) GetPrimaryKeyColumns(ctx context.Context, dbName, ta } // GetPrimaryKeyEquivalentColumns is part of the MysqlDaemon interface -func (fmd *FakeMysqlDaemon) GetPrimaryKeyEquivalentColumns(ctx context.Context, dbName, table string) ([]string, error) { - return []string{}, nil +func (fmd *FakeMysqlDaemon) GetPrimaryKeyEquivalentColumns(ctx context.Context, dbName, table string) ([]string, string, error) { + return []string{}, "", nil } // PreflightSchemaChange is part of the MysqlDaemon interface 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..5c2fcdb2356 100644 --- a/go/vt/mysqlctl/schema.go +++ b/go/vt/mysqlctl/schema.go @@ -577,16 +577,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 +629,17 @@ 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)) + index := "" for i, row := range named.Rows { cols[i] = row.AsString("column_name", "") + index = row.AsString("index_name", "") } - return cols, err + return cols, index, 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 66a9b2c6d92..f6f5316c0ba 100644 --- a/go/vt/proto/binlogdata/binlogdata.pb.go +++ b/go/vt/proto/binlogdata/binlogdata.pb.go @@ -2018,6 +2018,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. + PKIndex string `protobuf:"bytes,4,opt,name=p_k_index,json=pKIndex,proto3" json:"p_k_index,omitempty"` } func (x *MinimalTable) Reset() { @@ -2073,6 +2077,13 @@ func (x *MinimalTable) GetPKColumns() []int64 { return nil } +func (x *MinimalTable) GetPKIndex() string { + if x != nil { + return x.PKIndex + } + return "" +} + type MinimalSchema struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -3075,43 +3086,20 @@ var file_binlogdata_proto_rawDesc = []byte{ 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, + 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x64, 0x22, 0x84, 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, 0x1a, 0x0a, 0x09, 0x70, 0x5f, 0x6b, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x04, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x70, 0x4b, 0x49, 0x6e, 0x64, 0x65, 0x78, 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, @@ -3123,105 +3111,130 @@ var file_binlogdata_proto_rawDesc = []byte{ 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, 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, + 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, 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, 0x34, 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, 0x2a, 0x71, 0x0a, 0x19, 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, 0x34, 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, 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, + 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 926ea5434c8..31b82b84970 100644 --- a/go/vt/proto/binlogdata/binlogdata_vtproto.pb.go +++ b/go/vt/proto/binlogdata/binlogdata_vtproto.pb.go @@ -1422,6 +1422,13 @@ func (m *MinimalTable) MarshalToSizedBufferVT(dAtA []byte) (int, error) { i -= len(m.unknownFields) copy(dAtA[i:], m.unknownFields) } + if len(m.PKIndex) > 0 { + i -= len(m.PKIndex) + copy(dAtA[i:], m.PKIndex) + i = encodeVarint(dAtA, i, uint64(len(m.PKIndex))) + i-- + dAtA[i] = 0x22 + } if len(m.PKColumns) > 0 { var pksize2 int for _, num := range m.PKColumns { @@ -2684,6 +2691,10 @@ func (m *MinimalTable) SizeVT() (n int) { } n += 1 + sov(uint64(l)) + l } + l = len(m.PKIndex) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } n += len(m.unknownFields) return n } @@ -6925,6 +6936,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 PKIndex", 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.PKIndex = 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 39bd8db2519..81e80856085 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/vreplicator.go +++ b/go/vt/vttablet/tabletmanager/vreplication/vreplicator.go @@ -341,7 +341,7 @@ func (vr *vreplicator) buildColInfoMap(ctx context.Context) (map[string][]*Colum 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 { + 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 diff --git a/go/vt/vttablet/tabletmanager/vreplication/vreplicator_test.go b/go/vt/vttablet/tabletmanager/vreplication/vreplicator_test.go index 58f38246af3..496fa120149 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) + got, 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, got, tt.wantCols, "Mysqld.GetPrimaryKeyEquivalentColumns() columns = %v, want %v", got, 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 c55c312f442..12691127e81 100644 --- a/go/vt/vttablet/tabletserver/vstreamer/engine.go +++ b/go/vt/vttablet/tabletserver/vstreamer/engine.go @@ -547,10 +547,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.PKIndex = 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 86b50947cdf..1dd4a83882e 100644 --- a/go/vt/vttablet/tabletserver/vstreamer/rowstreamer.go +++ b/go/vt/vttablet/tabletserver/vstreamer/rowstreamer.go @@ -180,7 +180,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 } @@ -220,6 +220,8 @@ func (rs *rowStreamer) buildPKColumns(st *binlogdatapb.MinimalTable) ([]int, err pkColumns[i] = i } return pkColumns, nil + } else { + st.PKIndex = "PRIMARY" } for _, pk := range st.PKColumns { if pk >= int64(len(st.Fields)) { @@ -230,7 +232,7 @@ func (rs *rowStreamer) buildPKColumns(st *binlogdatapb.MinimalTable) ([]int, err 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 ") @@ -245,7 +247,15 @@ 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. + var indexHint string + if st.PKIndex != "" { + indexHint = fmt.Sprintf(" force index (`%s`)", st.PKIndex) + } + 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) diff --git a/go/vt/vttablet/tabletserver/vstreamer/rowstreamer_test.go b/go/vt/vttablet/tabletserver/vstreamer/rowstreamer_test.go index 6ba5a3a5d02..ccd6a3bf1f6 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{ @@ -67,7 +70,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 +78,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 +86,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 +94,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 +102,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 +110,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 +118,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 +126,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 +134,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 +158,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 +166,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 +269,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 +301,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 +334,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 +362,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/proto/binlogdata.proto b/proto/binlogdata.proto index 2141aa99c3a..a13bfd44445 100644 --- a/proto/binlogdata.proto +++ b/proto/binlogdata.proto @@ -451,6 +451,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. + string p_k_index = 4; } message MinimalSchema { diff --git a/web/vtadmin/src/proto/vtadmin.d.ts b/web/vtadmin/src/proto/vtadmin.d.ts index 20eb3d4a749..b7ec54d0a45 100644 --- a/web/vtadmin/src/proto/vtadmin.d.ts +++ b/web/vtadmin/src/proto/vtadmin.d.ts @@ -31009,6 +31009,9 @@ export namespace binlogdata { /** MinimalTable p_k_columns */ p_k_columns?: ((number|Long)[]|null); + + /** MinimalTable p_k_index */ + p_k_index?: (string|null); } /** Represents a MinimalTable. */ @@ -31029,6 +31032,9 @@ export namespace binlogdata { /** MinimalTable p_k_columns. */ public p_k_columns: (number|Long)[]; + /** MinimalTable p_k_index. */ + public p_k_index: 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 d69f14e8ca8..375d3b54c13 100644 --- a/web/vtadmin/src/proto/vtadmin.js +++ b/web/vtadmin/src/proto/vtadmin.js @@ -72935,6 +72935,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] MinimalTable p_k_index */ /** @@ -72978,6 +72979,14 @@ export const binlogdata = $root.binlogdata = (() => { */ MinimalTable.prototype.p_k_columns = $util.emptyArray; + /** + * MinimalTable p_k_index. + * @member {string} p_k_index + * @memberof binlogdata.MinimalTable + * @instance + */ + MinimalTable.prototype.p_k_index = ""; + /** * Creates a new MinimalTable instance using the specified properties. * @function create @@ -73013,6 +73022,8 @@ export const binlogdata = $root.binlogdata = (() => { writer.int64(message.p_k_columns[i]); writer.ldelim(); } + if (message.p_k_index != null && Object.hasOwnProperty.call(message, "p_k_index")) + writer.uint32(/* id 4, wireType 2 =*/34).string(message.p_k_index); return writer; }; @@ -73068,6 +73079,10 @@ export const binlogdata = $root.binlogdata = (() => { message.p_k_columns.push(reader.int64()); break; } + case 4: { + message.p_k_index = reader.string(); + break; + } default: reader.skipType(tag & 7); break; @@ -73122,6 +73137,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 != null && message.hasOwnProperty("p_k_index")) + if (!$util.isString(message.p_k_index)) + return "p_k_index: string expected"; return null; }; @@ -73163,6 +73181,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 != null) + message.p_k_index = String(object.p_k_index); return message; }; @@ -73183,8 +73203,10 @@ export const binlogdata = $root.binlogdata = (() => { object.fields = []; object.p_k_columns = []; } - if (options.defaults) + if (options.defaults) { object.name = ""; + object.p_k_index = ""; + } if (message.name != null && message.hasOwnProperty("name")) object.name = message.name; if (message.fields && message.fields.length) { @@ -73200,6 +73222,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 != null && message.hasOwnProperty("p_k_index")) + object.p_k_index = message.p_k_index; return object; }; From 1de39a1a4e6eca73a620a969a3bfa3969c24a503 Mon Sep 17 00:00:00 2001 From: Matt Lord Date: Wed, 30 Aug 2023 16:59:19 -0400 Subject: [PATCH 2/7] Fix unit tests Signed-off-by: Matt Lord --- go/vt/vttablet/tabletserver/vstreamer/rowstreamer_test.go | 1 + go/vt/vttablet/tabletserver/vstreamer/uvstreamer_flaky_test.go | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/go/vt/vttablet/tabletserver/vstreamer/rowstreamer_test.go b/go/vt/vttablet/tabletserver/vstreamer/rowstreamer_test.go index ccd6a3bf1f6..fcc179dbc5b 100644 --- a/go/vt/vttablet/tabletserver/vstreamer/rowstreamer_test.go +++ b/go/vt/vttablet/tabletserver/vstreamer/rowstreamer_test.go @@ -61,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()) diff --git a/go/vt/vttablet/tabletserver/vstreamer/uvstreamer_flaky_test.go b/go/vt/vttablet/tabletserver/vstreamer/uvstreamer_flaky_test.go index 91505ba14af..b6328e6291b 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 { From e8c62283cd6bce0b4ff6fca515b8f29af2b97226 Mon Sep 17 00:00:00 2001 From: Matt Lord Date: Wed, 30 Aug 2023 22:19:48 -0400 Subject: [PATCH 3/7] Minor tweaks from self review Signed-off-by: Matt Lord --- .../tabletserver/vstreamer/rowstreamer.go | 22 +++++++++---------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/go/vt/vttablet/tabletserver/vstreamer/rowstreamer.go b/go/vt/vttablet/tabletserver/vstreamer/rowstreamer.go index 1dd4a83882e..b095db09027 100644 --- a/go/vt/vttablet/tabletserver/vstreamer/rowstreamer.go +++ b/go/vt/vttablet/tabletserver/vstreamer/rowstreamer.go @@ -103,7 +103,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 @@ -136,11 +136,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{}) @@ -208,20 +208,18 @@ 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 } return pkColumns, nil - } else { - st.PKIndex = "PRIMARY" } for _, pk := range st.PKColumns { if pk >= int64(len(st.Fields)) { @@ -229,6 +227,7 @@ func (rs *rowStreamer) buildPKColumns(st *binlogdatapb.MinimalTable) ([]int, err } pkColumns = append(pkColumns, int(pk)) } + st.PKIndex = "PRIMARY" return pkColumns, nil } @@ -250,7 +249,8 @@ func (rs *rowStreamer) buildSelect(st *binlogdatapb.MinimalTable) (string, error // 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. + // 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.PKIndex != "" { indexHint = fmt.Sprintf(" force index (`%s`)", st.PKIndex) @@ -262,11 +262,11 @@ func (rs *rowStreamer) buildSelect(st *binlogdatapb.MinimalTable) (string, error } 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 " From d2b116eecc7a28a9317215c6f03f241d380bff09 Mon Sep 17 00:00:00 2001 From: Matt Lord Date: Wed, 6 Sep 2023 13:43:48 -0400 Subject: [PATCH 4/7] Minor changes after self review Signed-off-by: Matt Lord --- go/vt/mysqlctl/schema.go | 33 +- go/vt/proto/binlogdata/binlogdata.pb.go | 350 +++++++++--------- .../proto/binlogdata/binlogdata_vtproto.pb.go | 18 +- .../tabletmanager/vreplication/vreplicator.go | 6 +- .../vttablet/tabletserver/vstreamer/engine.go | 2 +- .../tabletserver/vstreamer/rowstreamer.go | 6 +- proto/binlogdata.proto | 4 +- web/vtadmin/src/proto/vtadmin.d.ts | 8 +- web/vtadmin/src/proto/vtadmin.js | 30 +- 9 files changed, 235 insertions(+), 222 deletions(-) diff --git a/go/vt/mysqlctl/schema.go b/go/vt/mysqlctl/schema.go index 5c2fcdb2356..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 // @@ -634,12 +636,23 @@ func (mysqld *Mysqld) GetPrimaryKeyEquivalentColumns(ctx context.Context, dbName named := qr.Named() cols := make([]string, len(qr.Rows)) - index := "" + indexName := "" for i, row := range named.Rows { cols[i] = row.AsString("column_name", "") - index = row.AsString("index_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, index, 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 402fa5d7480..c0a4bd08860 100644 --- a/go/vt/proto/binlogdata/binlogdata.pb.go +++ b/go/vt/proto/binlogdata/binlogdata.pb.go @@ -2031,8 +2031,8 @@ type MinimalTable struct { 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. - PKIndex string `protobuf:"bytes,4,opt,name=p_k_index,json=pKIndex,proto3" json:"p_k_index,omitempty"` + // 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() { @@ -2088,9 +2088,9 @@ func (x *MinimalTable) GetPKColumns() []int64 { return nil } -func (x *MinimalTable) GetPKIndex() string { +func (x *MinimalTable) GetPKIndexName() string { if x != nil { - return x.PKIndex + return x.PKIndexName } return "" } @@ -3250,183 +3250,183 @@ 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, 0x84, 0x01, 0x0a, + 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, 0x1a, 0x0a, 0x09, 0x70, 0x5f, 0x6b, 0x5f, 0x69, - 0x6e, 0x64, 0x65, 0x78, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x70, 0x4b, 0x49, 0x6e, - 0x64, 0x65, 0x78, 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, 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, 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, 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, + 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, + 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, 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, 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, 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 33a4f52f503..fd1fe5e459d 100644 --- a/go/vt/proto/binlogdata/binlogdata_vtproto.pb.go +++ b/go/vt/proto/binlogdata/binlogdata_vtproto.pb.go @@ -541,8 +541,8 @@ func (m *MinimalTable) CloneVT() *MinimalTable { return (*MinimalTable)(nil) } r := &MinimalTable{ - Name: m.Name, - PKIndex: m.PKIndex, + Name: m.Name, + PKIndexName: m.PKIndexName, } if rhs := m.Fields; rhs != nil { tmpContainer := make([]*query.Field, len(rhs)) @@ -2263,10 +2263,10 @@ func (m *MinimalTable) MarshalToSizedBufferVT(dAtA []byte) (int, error) { i -= len(m.unknownFields) copy(dAtA[i:], m.unknownFields) } - if len(m.PKIndex) > 0 { - i -= len(m.PKIndex) - copy(dAtA[i:], m.PKIndex) - i = encodeVarint(dAtA, i, uint64(len(m.PKIndex))) + 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 } @@ -3738,7 +3738,7 @@ func (m *MinimalTable) SizeVT() (n int) { } n += 1 + sov(uint64(l)) + l } - l = len(m.PKIndex) + l = len(m.PKIndexName) if l > 0 { n += 1 + l + sov(uint64(l)) } @@ -8066,7 +8066,7 @@ func (m *MinimalTable) UnmarshalVT(dAtA []byte) error { } case 4: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field PKIndex", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field PKIndexName", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -8094,7 +8094,7 @@ func (m *MinimalTable) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.PKIndex = string(dAtA[iNdEx:postIndex]) + m.PKIndexName = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex default: iNdEx = preIndex diff --git a/go/vt/vttablet/tabletmanager/vreplication/vreplicator.go b/go/vt/vttablet/tabletmanager/vreplication/vreplicator.go index 7d927728866..b936b49e8a5 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/vreplicator.go +++ b/go/vt/vttablet/tabletmanager/vreplication/vreplicator.go @@ -346,14 +346,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 + // 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/tabletserver/vstreamer/engine.go b/go/vt/vttablet/tabletserver/vstreamer/engine.go index 9d49d88d849..2862601bf1b 100644 --- a/go/vt/vttablet/tabletserver/vstreamer/engine.go +++ b/go/vt/vttablet/tabletserver/vstreamer/engine.go @@ -597,7 +597,7 @@ func (vse *Engine) mapPKEquivalentCols(ctx context.Context, table *binlogdatapb. return nil, err } if len(pkeColNames) > 0 && indexName != "" { - table.PKIndex = indexName + table.PKIndexName = indexName } pkeCols := make([]int, len(pkeColNames)) matches := 0 diff --git a/go/vt/vttablet/tabletserver/vstreamer/rowstreamer.go b/go/vt/vttablet/tabletserver/vstreamer/rowstreamer.go index f63e0407a37..bdc36e46567 100644 --- a/go/vt/vttablet/tabletserver/vstreamer/rowstreamer.go +++ b/go/vt/vttablet/tabletserver/vstreamer/rowstreamer.go @@ -245,7 +245,7 @@ func (rs *rowStreamer) buildPKColumns(st *binlogdatapb.MinimalTable) ([]int, err } pkColumns = append(pkColumns, int(pk)) } - st.PKIndex = "PRIMARY" + st.PKIndexName = "PRIMARY" return pkColumns, nil } @@ -270,8 +270,8 @@ func (rs *rowStreamer) buildSelect(st *binlogdatapb.MinimalTable) (string, error // 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.PKIndex != "" { - indexHint = fmt.Sprintf(" force index (`%s`)", st.PKIndex) + if st.PKIndexName != "" { + indexHint = fmt.Sprintf(" force index (`%s`)", st.PKIndexName) } buf.Myprintf(" from %v%s", sqlparser.NewIdentifierCS(rs.plan.Table.Name), indexHint) if len(rs.lastpk) != 0 { diff --git a/proto/binlogdata.proto b/proto/binlogdata.proto index 3f581d7a774..b1f36261522 100644 --- a/proto/binlogdata.proto +++ b/proto/binlogdata.proto @@ -455,8 +455,8 @@ message MinimalTable { 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. - string p_k_index = 4; + // 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 41f5419420e..bd5aa135af9 100644 --- a/web/vtadmin/src/proto/vtadmin.d.ts +++ b/web/vtadmin/src/proto/vtadmin.d.ts @@ -31017,8 +31017,8 @@ export namespace binlogdata { /** MinimalTable p_k_columns */ p_k_columns?: ((number|Long)[]|null); - /** MinimalTable p_k_index */ - p_k_index?: (string|null); + /** MinimalTable p_k_index_name */ + p_k_index_name?: (string|null); } /** Represents a MinimalTable. */ @@ -31039,8 +31039,8 @@ export namespace binlogdata { /** MinimalTable p_k_columns. */ public p_k_columns: (number|Long)[]; - /** MinimalTable p_k_index. */ - public p_k_index: string; + /** MinimalTable p_k_index_name. */ + public p_k_index_name: string; /** * Creates a new MinimalTable instance using the specified properties. diff --git a/web/vtadmin/src/proto/vtadmin.js b/web/vtadmin/src/proto/vtadmin.js index 153ce8ee0d7..b4377d884a2 100644 --- a/web/vtadmin/src/proto/vtadmin.js +++ b/web/vtadmin/src/proto/vtadmin.js @@ -72970,7 +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] MinimalTable p_k_index + * @property {string|null} [p_k_index_name] MinimalTable p_k_index_name */ /** @@ -73015,12 +73015,12 @@ export const binlogdata = $root.binlogdata = (() => { MinimalTable.prototype.p_k_columns = $util.emptyArray; /** - * MinimalTable p_k_index. - * @member {string} p_k_index + * MinimalTable p_k_index_name. + * @member {string} p_k_index_name * @memberof binlogdata.MinimalTable * @instance */ - MinimalTable.prototype.p_k_index = ""; + MinimalTable.prototype.p_k_index_name = ""; /** * Creates a new MinimalTable instance using the specified properties. @@ -73057,8 +73057,8 @@ export const binlogdata = $root.binlogdata = (() => { writer.int64(message.p_k_columns[i]); writer.ldelim(); } - if (message.p_k_index != null && Object.hasOwnProperty.call(message, "p_k_index")) - writer.uint32(/* id 4, wireType 2 =*/34).string(message.p_k_index); + 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; }; @@ -73115,7 +73115,7 @@ export const binlogdata = $root.binlogdata = (() => { break; } case 4: { - message.p_k_index = reader.string(); + message.p_k_index_name = reader.string(); break; } default: @@ -73172,9 +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 != null && message.hasOwnProperty("p_k_index")) - if (!$util.isString(message.p_k_index)) - return "p_k_index: string 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; }; @@ -73216,8 +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 != null) - message.p_k_index = String(object.p_k_index); + if (object.p_k_index_name != null) + message.p_k_index_name = String(object.p_k_index_name); return message; }; @@ -73240,7 +73240,7 @@ export const binlogdata = $root.binlogdata = (() => { } if (options.defaults) { object.name = ""; - object.p_k_index = ""; + object.p_k_index_name = ""; } if (message.name != null && message.hasOwnProperty("name")) object.name = message.name; @@ -73257,8 +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 != null && message.hasOwnProperty("p_k_index")) - object.p_k_index = message.p_k_index; + 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; }; From 294685a68597f5076b6ea71135509982533c1fec Mon Sep 17 00:00:00 2001 From: Matt Lord Date: Wed, 6 Sep 2023 13:51:50 -0400 Subject: [PATCH 5/7] Align comment style while here... Signed-off-by: Matt Lord --- go/vt/mysqlctl/fakemysqldaemon.go | 160 ++++++++++++++++-------------- 1 file changed, 84 insertions(+), 76 deletions(-) diff --git a/go/vt/mysqlctl/fakemysqldaemon.go b/go/vt/mysqlctl/fakemysqldaemon.go index f50d118d3b5..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 +// 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 { From b1657e1529d6d5b40faab2bd22361260aaeb5a62 Mon Sep 17 00:00:00 2001 From: Matt Lord Date: Wed, 6 Sep 2023 14:15:57 -0400 Subject: [PATCH 6/7] Nit on unit test var name (and kick CI) Signed-off-by: Matt Lord --- go/vt/vttablet/tabletmanager/vreplication/vreplicator_test.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/go/vt/vttablet/tabletmanager/vreplication/vreplicator_test.go b/go/vt/vttablet/tabletmanager/vreplication/vreplicator_test.go index 496fa120149..0a8d3a94ab7 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/vreplicator_test.go +++ b/go/vt/vttablet/tabletmanager/vreplication/vreplicator_test.go @@ -182,12 +182,12 @@ func TestPrimaryKeyEquivalentColumns(t *testing.T) { for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { require.NoError(t, env.Mysqld.ExecuteSuperQuery(ctx, tt.ddl)) - got, indexName, 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 } - require.Equalf(t, got, tt.wantCols, "Mysqld.GetPrimaryKeyEquivalentColumns() columns = %v, want %v", got, tt.wantCols) + 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) }) } From 08e36177fb7a48171e4292c2a9e4de6234fb38ac Mon Sep 17 00:00:00 2001 From: Matt Lord Date: Sun, 1 Oct 2023 16:09:46 -0400 Subject: [PATCH 7/7] Peform safe escaping of index name Signed-off-by: Matt Lord --- go/vt/vttablet/tabletserver/vstreamer/rowstreamer.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/go/vt/vttablet/tabletserver/vstreamer/rowstreamer.go b/go/vt/vttablet/tabletserver/vstreamer/rowstreamer.go index bdc36e46567..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" @@ -271,7 +272,8 @@ func (rs *rowStreamer) buildSelect(st *binlogdatapb.MinimalTable) (string, error // 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`)", 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 {