From b97ad0e1340a424e0f86e4b074766caae9c37e11 Mon Sep 17 00:00:00 2001 From: Matt Lord Date: Wed, 8 Jan 2025 19:14:51 -0500 Subject: [PATCH 1/7] Add lastpk value for source and target Signed-off-by: Matt Lord --- .../tabletmanager/vdiff/table_differ.go | 85 ++++++++++++++----- .../tabletmanager/vdiff/table_plan.go | 5 ++ go/vt/vttablet/tabletmanager/vdiff/utils.go | 11 --- .../tabletmanager/vdiff/workflow_differ.go | 65 ++++++++++++-- 4 files changed, 124 insertions(+), 42 deletions(-) diff --git a/go/vt/vttablet/tabletmanager/vdiff/table_differ.go b/go/vt/vttablet/tabletmanager/vdiff/table_differ.go index 102d7535af9..eeaced94166 100644 --- a/go/vt/vttablet/tabletmanager/vdiff/table_differ.go +++ b/go/vt/vttablet/tabletmanager/vdiff/table_differ.go @@ -86,9 +86,10 @@ type tableDiffer struct { targetPrimitive engine.Primitive // sourceQuery is computed from the associated query for this table in the vreplication workflow's Rule Filter - sourceQuery string - table *tabletmanagerdatapb.TableDefinition - lastPK *querypb.QueryResult + sourceQuery string + table *tabletmanagerdatapb.TableDefinition + lastSourcePK *querypb.QueryResult + lastTargetPK *querypb.QueryResult // wgShardStreamers is used, with a cancellable context, to wait for all shard streamers // to finish after each diff is complete. @@ -349,7 +350,7 @@ func (td *tableDiffer) startTargetDataStream(ctx context.Context) error { ct := td.wd.ct gtidch := make(chan string, 1) ct.targetShardStreamer.result = make(chan *sqltypes.Result, 1) - go td.streamOneShard(ctx, ct.targetShardStreamer, td.tablePlan.targetQuery, td.lastPK, gtidch) + go td.streamOneShard(ctx, ct.targetShardStreamer, td.tablePlan.targetQuery, td.lastTargetPK, gtidch) gtid, ok := <-gtidch if !ok { log.Infof("streaming error: %v", ct.targetShardStreamer.err) @@ -364,7 +365,7 @@ func (td *tableDiffer) startSourceDataStreams(ctx context.Context) error { if err := td.forEachSource(func(source *migrationSource) error { gtidch := make(chan string, 1) source.result = make(chan *sqltypes.Result, 1) - go td.streamOneShard(ctx, source.shardStreamer, td.tablePlan.sourceQuery, td.lastPK, gtidch) + go td.streamOneShard(ctx, source.shardStreamer, td.tablePlan.sourceQuery, td.lastSourcePK, gtidch) gtid, ok := <-gtidch if !ok { @@ -721,42 +722,60 @@ func (td *tableDiffer) updateTableProgress(dbClient binlogplayer.DBClient, dr *D if dr == nil { return fmt.Errorf("cannot update progress with a nil diff report") } - var lastPK []byte var err error var query string rpt, err := json.Marshal(dr) if err != nil { return err } - if lastRow != nil { - lastPK, err = td.lastPKFromRow(lastRow) + + if lastRow == nil { + query, err = sqlparser.ParseAndBind(sqlUpdateTableNoProgress, + sqltypes.Int64BindVariable(dr.ProcessedRows), + sqltypes.StringBindVariable(string(rpt)), + sqltypes.Int64BindVariable(td.wd.ct.id), + sqltypes.StringBindVariable(td.table.Name), + ) if err != nil { return err } - + } else { + var lastSourcePK, lastTargetPK []byte + lastPK := make(map[string]string, 2) + if lastRow != nil { + lastSourcePK, err = td.lastSourcePKFromRow(lastRow) + if err != nil { + return err + } + lastPK["source"] = string(lastSourcePK) + lastTargetPK, err = td.lastTargetPKFromRow(lastRow) + if err != nil { + return err + } + lastPK["target"] = string(lastTargetPK) + } if td.wd.opts.CoreOptions.MaxDiffSeconds > 0 { // Update the in-memory lastPK as well so that we can restart the table // diff if needed. - lastpkpb := &querypb.QueryResult{} - if err := prototext.Unmarshal(lastPK, lastpkpb); err != nil { + lastSourcePKPB, lastTargetPKPB := &querypb.QueryResult{}, &querypb.QueryResult{} + if err := prototext.Unmarshal(lastSourcePK, lastSourcePKPB); err != nil { return err } - td.lastPK = lastpkpb + td.lastSourcePK = lastSourcePKPB + if err := prototext.Unmarshal(lastTargetPK, lastTargetPKPB); err != nil { + return err + } + td.lastTargetPK = lastTargetPKPB } - - query, err = sqlparser.ParseAndBind(sqlUpdateTableProgress, - sqltypes.Int64BindVariable(dr.ProcessedRows), - sqltypes.StringBindVariable(string(lastPK)), - sqltypes.StringBindVariable(string(rpt)), - sqltypes.Int64BindVariable(td.wd.ct.id), - sqltypes.StringBindVariable(td.table.Name), - ) + //log.Errorf("DEBUG: updateTableProgress lastPK map: %v", lastPK) + lastPKJS, err := json.Marshal(lastPK) if err != nil { return err } - } else { - query, err = sqlparser.ParseAndBind(sqlUpdateTableNoProgress, + //log.Errorf("DEBUG: updateTableProgress lastPK JSON: %v", lastPKJS) + query, err = sqlparser.ParseAndBind(sqlUpdateTableProgress, sqltypes.Int64BindVariable(dr.ProcessedRows), + sqltypes.StringBindVariable(string(lastPKJS)), sqltypes.StringBindVariable(string(rpt)), sqltypes.Int64BindVariable(td.wd.ct.id), sqltypes.StringBindVariable(td.table.Name), @@ -832,7 +851,7 @@ func updateTableMismatch(dbClient binlogplayer.DBClient, vdiffID int64, table st return nil } -func (td *tableDiffer) lastPKFromRow(row []sqltypes.Value) ([]byte, error) { +func (td *tableDiffer) lastTargetPKFromRow(row []sqltypes.Value) ([]byte, error) { pkColCnt := len(td.tablePlan.pkCols) pkFields := make([]*querypb.Field, pkColCnt) pkVals := make([]sqltypes.Value, pkColCnt) @@ -847,6 +866,26 @@ func (td *tableDiffer) lastPKFromRow(row []sqltypes.Value) ([]byte, error) { return buf, err } +func (td *tableDiffer) lastSourcePKFromRow(row []sqltypes.Value) ([]byte, error) { + if len(td.tablePlan.sourcePkCols) == 0 { + // If there are no PKs on the source then we use + // the same PK[E] columns as the target. + td.tablePlan.sourcePkCols = td.tablePlan.pkCols + } + pkColCnt := len(td.tablePlan.sourcePkCols) + pkFields := make([]*querypb.Field, pkColCnt) + pkVals := make([]sqltypes.Value, pkColCnt) + for i, colIndex := range td.tablePlan.sourcePkCols { + pkFields[i] = td.tablePlan.table.Fields[colIndex] + pkVals[i] = row[colIndex] + } + buf, err := prototext.Marshal(&querypb.QueryResult{ + Fields: pkFields, + Rows: []*querypb.Row{sqltypes.RowToProto3(pkVals)}, + }) + return buf, err +} + // If SourceTimeZone is defined in the BinlogSource (_vt.vreplication.source), the // VReplication workflow would have converted the datetime columns expecting the // source to have been in the SourceTimeZone and target in TargetTimeZone. We need diff --git a/go/vt/vttablet/tabletmanager/vdiff/table_plan.go b/go/vt/vttablet/tabletmanager/vdiff/table_plan.go index 836df8ffe94..f03cf20fb0a 100644 --- a/go/vt/vttablet/tabletmanager/vdiff/table_plan.go +++ b/go/vt/vttablet/tabletmanager/vdiff/table_plan.go @@ -53,6 +53,10 @@ type tablePlan struct { comparePKs []compareColInfo // pkCols has the indices of PK cols in the select list pkCols []int + // sourcePkCols has the indices of PK cols in the select + // list, but from the source keyspace. This is needed to + // properly store the lastpk for the source. + sourcePkCols []int // selectPks is the list of pk columns as they appear in the select clause for the diff. selectPks []int @@ -207,6 +211,7 @@ func (tp *tablePlan) findPKs(dbClient binlogplayer.DBClient, targetSelect *sqlpa if len(tp.table.PrimaryKeyColumns) == 0 { return nil } + var orderby sqlparser.OrderBy for _, pk := range tp.table.PrimaryKeyColumns { found := false diff --git a/go/vt/vttablet/tabletmanager/vdiff/utils.go b/go/vt/vttablet/tabletmanager/vdiff/utils.go index aeaa28972e0..68e8a6acb57 100644 --- a/go/vt/vttablet/tabletmanager/vdiff/utils.go +++ b/go/vt/vttablet/tabletmanager/vdiff/utils.go @@ -80,17 +80,6 @@ func insertVDiffLog(ctx context.Context, dbClient binlogplayer.DBClient, vdiffID } } -func stringListContains(lst []string, item string) bool { - contains := false - for _, t := range lst { - if t == item { - contains = true - break - } - } - return contains -} - // copyNonKeyRangeExpressions copies all expressions from the input WHERE clause // to the output WHERE clause except for any in_keyrange() expressions. func copyNonKeyRangeExpressions(where *sqlparser.Where) *sqlparser.Where { diff --git a/go/vt/vttablet/tabletmanager/vdiff/workflow_differ.go b/go/vt/vttablet/tabletmanager/vdiff/workflow_differ.go index ef30d8f14b0..f5b4584e1be 100644 --- a/go/vt/vttablet/tabletmanager/vdiff/workflow_differ.go +++ b/go/vt/vttablet/tabletmanager/vdiff/workflow_differ.go @@ -18,12 +18,15 @@ package vdiff import ( "context" + "encoding/json" "errors" "fmt" "reflect" + "slices" "strings" "time" + "golang.org/x/exp/maps" "google.golang.org/protobuf/encoding/prototext" "vitess.io/vitess/go/mysql/collations" @@ -344,7 +347,7 @@ func (wd *workflowDiffer) buildPlan(dbClient binlogplayer.DBClient, filter *binl for _, table := range schm.TableDefinitions { // if user specified tables explicitly only use those, otherwise diff all tables in workflow - if len(specifiedTables) != 0 && !stringListContains(specifiedTables, table.Name) { + if len(specifiedTables) != 0 && !slices.Contains(specifiedTables, table.Name) { continue } if schema.IsInternalOperationTableName(table.Name) && !schema.IsOnlineDDLTableName(table.Name) { @@ -370,15 +373,52 @@ func (wd *workflowDiffer) buildPlan(dbClient binlogplayer.DBClient, filter *binl } td := newTableDiffer(wd, table, sourceQuery) - lastpkpb, err := wd.getTableLastPK(dbClient, table.Name) + + lastPK, err := wd.getTableLastPK(dbClient, table.Name) if err != nil { return err } - td.lastPK = lastpkpb + td.lastSourcePK = lastPK["source"] + td.lastTargetPK = lastPK["target"] wd.tableDiffers[table.Name] = td if _, err := td.buildTablePlan(dbClient, wd.ct.vde.dbName, wd.collationEnv); err != nil { return err } + + // We get the PK columns from the source schema as well, as they can + // differ and determine the proper lastPK to use when saving progress. + // We use the first sourceShard as all of them should have the same schema. + sourceShardName := maps.Keys(wd.ct.sources)[0] + sourceShard, err := wd.ct.ts.GetShard(wd.ct.vde.ctx, wd.ct.sourceKeyspace, sourceShardName) + if err != nil { + return err + } + if sourceShard.PrimaryAlias == nil { + return fmt.Errorf("source shard %s has no primary", sourceShardName) + } + sourceTablet, err := wd.ct.ts.GetTablet(wd.ct.vde.ctx, sourceShard.PrimaryAlias) + if err != nil { + return fmt.Errorf("failed to get source shard %s primary", sourceShardName) + } + sourceSchema, err := wd.ct.tmc.GetSchema(wd.ct.vde.ctx, sourceTablet.Tablet, &tabletmanagerdatapb.GetSchemaRequest{ + Tables: []string{table.Name}, + }) + if err != nil { + return err + } + //log.Errorf("DEBUG: sourceTable.PrimaryKeyColumns: %v", sourceSchema.TableDefinitions[0].PrimaryKeyColumns) + sourcePKColumns := make(map[string]struct{}, len(sourceSchema.TableDefinitions[0].PrimaryKeyColumns)) + td.tablePlan.sourcePkCols = make([]int, 0, len(sourceSchema.TableDefinitions[0].PrimaryKeyColumns)) + for _, pkc := range sourceSchema.TableDefinitions[0].PrimaryKeyColumns { + sourcePKColumns[pkc] = struct{}{} + } + //log.Errorf("DEBUG: sourcePKColumns: %v", sourcePKColumns) + for i, pkc := range table.PrimaryKeyColumns { + if _, ok := sourcePKColumns[pkc]; ok { + td.tablePlan.sourcePkCols = append(td.tablePlan.sourcePkCols, i) + } + } + //log.Errorf("DEBUG: td.tablePlan.sourcePkCols: %v", td.tablePlan.sourcePkCols) } if len(wd.tableDiffers) == 0 { return fmt.Errorf("no tables found to diff, %s:%s, on tablet %v", @@ -388,7 +428,7 @@ func (wd *workflowDiffer) buildPlan(dbClient binlogplayer.DBClient, filter *binl } // getTableLastPK gets the lastPK protobuf message for a given vdiff table. -func (wd *workflowDiffer) getTableLastPK(dbClient binlogplayer.DBClient, tableName string) (*querypb.QueryResult, error) { +func (wd *workflowDiffer) getTableLastPK(dbClient binlogplayer.DBClient, tableName string) (map[string]*querypb.QueryResult, error) { query, err := sqlparser.ParseAndBind(sqlGetVDiffTable, sqltypes.Int64BindVariable(wd.ct.id), sqltypes.StringBindVariable(tableName), @@ -406,11 +446,20 @@ func (wd *workflowDiffer) getTableLastPK(dbClient binlogplayer.DBClient, tableNa return nil, err } if len(lastpk) != 0 { - var lastpkpb querypb.QueryResult - if err := prototext.Unmarshal(lastpk, &lastpkpb); err != nil { - return nil, err + lastPK := make(map[string]string, 2) + lastPKResults := make(map[string]*querypb.QueryResult, 2) + if err := json.Unmarshal(lastpk, &lastPK); err != nil { + return nil, vterrors.Wrapf(err, "failed to unmarshal lastpk JSON for table %s", tableName) + } + //log.Errorf("DEBUG: getTabletLastPK lastPKBytes: %v", lastPK) + for k, v := range lastPK { + lastPKResults[k] = &querypb.QueryResult{} + if err := prototext.Unmarshal([]byte(v), lastPKResults[k]); err != nil { + return nil, vterrors.Wrapf(err, "failed to unmarshal lastpk QueryResult for table %s", tableName) + } } - return &lastpkpb, nil + //log.Errorf("DEBUG: getTabletLastPK lastPKRResults: %v", lastPKResults) + return lastPKResults, nil } } return nil, nil From c9f57f79594f91752e00ddd24e14cc8b946d26b1 Mon Sep 17 00:00:00 2001 From: Matt Lord Date: Wed, 8 Jan 2025 23:46:34 -0500 Subject: [PATCH 2/7] Fix Mount+Migrate Signed-off-by: Matt Lord --- .../tabletmanager/vdiff/table_differ.go | 12 +++--------- .../tabletmanager/vdiff/workflow_differ.go | 18 ++++++++++++++++-- 2 files changed, 19 insertions(+), 11 deletions(-) diff --git a/go/vt/vttablet/tabletmanager/vdiff/table_differ.go b/go/vt/vttablet/tabletmanager/vdiff/table_differ.go index eeaced94166..28b36f74ee8 100644 --- a/go/vt/vttablet/tabletmanager/vdiff/table_differ.go +++ b/go/vt/vttablet/tabletmanager/vdiff/table_differ.go @@ -239,15 +239,9 @@ func (td *tableDiffer) selectTablets(ctx context.Context) error { sourceCells := strings.Split(td.wd.opts.PickerOptions.SourceCell, ",") targetCells := strings.Split(td.wd.opts.PickerOptions.TargetCell, ",") - // For Mount+Migrate, the source tablets will be in a different - // Vitess cluster with its own TopoServer. - sourceTopoServer := td.wd.ct.ts - if td.wd.ct.externalCluster != "" { - extTS, err := td.wd.ct.ts.OpenExternalVitessClusterServer(ctx, td.wd.ct.externalCluster) - if err != nil { - return err - } - sourceTopoServer = extTS + sourceTopoServer, err := td.wd.getSourceTopoServer() + if err != nil { + return vterrors.Wrap(err, "failed to get source topo server") } tabletPickerOptions := discovery.TabletPickerOptions{} wg.Add(1) diff --git a/go/vt/vttablet/tabletmanager/vdiff/workflow_differ.go b/go/vt/vttablet/tabletmanager/vdiff/workflow_differ.go index f5b4584e1be..9e0ee6cbc41 100644 --- a/go/vt/vttablet/tabletmanager/vdiff/workflow_differ.go +++ b/go/vt/vttablet/tabletmanager/vdiff/workflow_differ.go @@ -36,6 +36,7 @@ import ( "vitess.io/vitess/go/vt/log" "vitess.io/vitess/go/vt/schema" "vitess.io/vitess/go/vt/sqlparser" + "vitess.io/vitess/go/vt/topo" "vitess.io/vitess/go/vt/vtctl/schematools" "vitess.io/vitess/go/vt/vterrors" "vitess.io/vitess/go/vt/vtgate/vindexes" @@ -389,14 +390,18 @@ func (wd *workflowDiffer) buildPlan(dbClient binlogplayer.DBClient, filter *binl // differ and determine the proper lastPK to use when saving progress. // We use the first sourceShard as all of them should have the same schema. sourceShardName := maps.Keys(wd.ct.sources)[0] - sourceShard, err := wd.ct.ts.GetShard(wd.ct.vde.ctx, wd.ct.sourceKeyspace, sourceShardName) + sourceTS, err := wd.getSourceTopoServer() + if err != nil { + return vterrors.Wrap(err, "failed to get source topo server") + } + sourceShard, err := sourceTS.GetShard(wd.ct.vde.ctx, wd.ct.sourceKeyspace, sourceShardName) if err != nil { return err } if sourceShard.PrimaryAlias == nil { return fmt.Errorf("source shard %s has no primary", sourceShardName) } - sourceTablet, err := wd.ct.ts.GetTablet(wd.ct.vde.ctx, sourceShard.PrimaryAlias) + sourceTablet, err := sourceTS.GetTablet(wd.ct.vde.ctx, sourceShard.PrimaryAlias) if err != nil { return fmt.Errorf("failed to get source shard %s primary", sourceShardName) } @@ -537,3 +542,12 @@ func (wd *workflowDiffer) initVDiffTables(dbClient binlogplayer.DBClient) error } return nil } + +// getSourceTopoServer returns the source topo server as for Mount+Migrate the +// source tablets will be in a different Vitess cluster with its own TopoServer. +func (wd *workflowDiffer) getSourceTopoServer() (*topo.Server, error) { + if wd.ct.externalCluster == "" { + return wd.ct.ts, nil + } + return wd.ct.ts.OpenExternalVitessClusterServer(wd.ct.vde.ctx, wd.ct.externalCluster) +} From 6ef34ad63d20b5eb9473097aad39868101e53bec Mon Sep 17 00:00:00 2001 From: Matt Lord Date: Thu, 9 Jan 2025 00:14:31 -0500 Subject: [PATCH 3/7] Code improvements Signed-off-by: Matt Lord --- .../tabletmanager/vdiff/table_differ.go | 97 ++++++++++++------- .../tabletmanager/vdiff/workflow_differ.go | 51 +++------- 2 files changed, 72 insertions(+), 76 deletions(-) diff --git a/go/vt/vttablet/tabletmanager/vdiff/table_differ.go b/go/vt/vttablet/tabletmanager/vdiff/table_differ.go index 28b36f74ee8..f509f8fe734 100644 --- a/go/vt/vttablet/tabletmanager/vdiff/table_differ.go +++ b/go/vt/vttablet/tabletmanager/vdiff/table_differ.go @@ -20,10 +20,12 @@ import ( "context" "encoding/json" "fmt" + "slices" "strings" "sync" "time" + "golang.org/x/exp/maps" "google.golang.org/protobuf/encoding/prototext" "vitess.io/vitess/go/mysql/collations" @@ -194,7 +196,7 @@ func (td *tableDiffer) stopTargetVReplicationStreams(ctx context.Context, dbClie return fmt.Errorf("stream %d has not started on tablet %v", id, td.wd.ct.vde.thisTablet.Alias) } - sourceBytes, err := row["source"].ToBytes() + sourceBytes, err := row[source].ToBytes() if err != nil { return err } @@ -520,8 +522,8 @@ func (td *tableDiffer) diff(ctx context.Context, coreOpts *tabletmanagerdatapb.V } dr.TableName = td.table.Name - sourceExecutor := newPrimitiveExecutor(ctx, td.sourcePrimitive, "source") - targetExecutor := newPrimitiveExecutor(ctx, td.targetPrimitive, "target") + sourceExecutor := newPrimitiveExecutor(ctx, td.sourcePrimitive, source) + targetExecutor := newPrimitiveExecutor(ctx, td.targetPrimitive, target) var sourceRow, lastProcessedRow, targetRow []sqltypes.Value advanceSource := true advanceTarget := true @@ -736,17 +738,19 @@ func (td *tableDiffer) updateTableProgress(dbClient binlogplayer.DBClient, dr *D } else { var lastSourcePK, lastTargetPK []byte lastPK := make(map[string]string, 2) - if lastRow != nil { - lastSourcePK, err = td.lastSourcePKFromRow(lastRow) - if err != nil { - return err - } - lastPK["source"] = string(lastSourcePK) - lastTargetPK, err = td.lastTargetPKFromRow(lastRow) + lastTargetPK, err = td.lastPKFromRow(lastRow, td.tablePlan.pkCols) + if err != nil { + return err + } + lastPK[target] = string(lastTargetPK) + if len(td.tablePlan.sourcePkCols) == 0 || slices.Equal(td.tablePlan.sourcePkCols, td.tablePlan.pkCols) { + lastPK[source] = string(lastTargetPK) + } else { + lastSourcePK, err = td.lastPKFromRow(lastRow, td.tablePlan.sourcePkCols) if err != nil { return err } - lastPK["target"] = string(lastTargetPK) + lastPK[source] = string(lastSourcePK) } if td.wd.opts.CoreOptions.MaxDiffSeconds > 0 { // Update the in-memory lastPK as well so that we can restart the table @@ -761,12 +765,10 @@ func (td *tableDiffer) updateTableProgress(dbClient binlogplayer.DBClient, dr *D } td.lastTargetPK = lastTargetPKPB } - //log.Errorf("DEBUG: updateTableProgress lastPK map: %v", lastPK) lastPKJS, err := json.Marshal(lastPK) if err != nil { return err } - //log.Errorf("DEBUG: updateTableProgress lastPK JSON: %v", lastPKJS) query, err = sqlparser.ParseAndBind(sqlUpdateTableProgress, sqltypes.Int64BindVariable(dr.ProcessedRows), sqltypes.StringBindVariable(string(lastPKJS)), @@ -845,31 +847,11 @@ func updateTableMismatch(dbClient binlogplayer.DBClient, vdiffID int64, table st return nil } -func (td *tableDiffer) lastTargetPKFromRow(row []sqltypes.Value) ([]byte, error) { - pkColCnt := len(td.tablePlan.pkCols) - pkFields := make([]*querypb.Field, pkColCnt) - pkVals := make([]sqltypes.Value, pkColCnt) - for i, colIndex := range td.tablePlan.pkCols { - pkFields[i] = td.tablePlan.table.Fields[colIndex] - pkVals[i] = row[colIndex] - } - buf, err := prototext.Marshal(&querypb.QueryResult{ - Fields: pkFields, - Rows: []*querypb.Row{sqltypes.RowToProto3(pkVals)}, - }) - return buf, err -} - -func (td *tableDiffer) lastSourcePKFromRow(row []sqltypes.Value) ([]byte, error) { - if len(td.tablePlan.sourcePkCols) == 0 { - // If there are no PKs on the source then we use - // the same PK[E] columns as the target. - td.tablePlan.sourcePkCols = td.tablePlan.pkCols - } - pkColCnt := len(td.tablePlan.sourcePkCols) +func (td *tableDiffer) lastPKFromRow(row []sqltypes.Value, pkCols []int) ([]byte, error) { + pkColCnt := len(pkCols) pkFields := make([]*querypb.Field, pkColCnt) pkVals := make([]sqltypes.Value, pkColCnt) - for i, colIndex := range td.tablePlan.sourcePkCols { + for i, colIndex := range pkCols { pkFields[i] = td.tablePlan.table.Fields[colIndex] pkVals[i] = row[colIndex] } @@ -926,6 +908,49 @@ func (td *tableDiffer) adjustForSourceTimeZone(targetSelectExprs sqlparser.Selec return targetSelectExprs } +// getSourcePKCols populates the sourcePkCols field in the tablePlan. +// We need this information in order to save the lastpk value for the +// source as the PK columns may differ between the source and target. +func (td *tableDiffer) getSourcePKCols() error { + ctx, cancel := context.WithTimeout(td.wd.ct.vde.ctx, topo.RemoteOperationTimeout*3) + defer cancel() + // We use the first sourceShard as all of them should have the same schema. + sourceShardName := maps.Keys(td.wd.ct.sources)[0] + sourceTS, err := td.wd.getSourceTopoServer() + if err != nil { + return vterrors.Wrap(err, "failed to get source topo server") + } + sourceShard, err := sourceTS.GetShard(ctx, td.wd.ct.sourceKeyspace, sourceShardName) + if err != nil { + return err + } + if sourceShard.PrimaryAlias == nil { + return fmt.Errorf("source shard %s has no primary", sourceShardName) + } + sourceTablet, err := sourceTS.GetTablet(ctx, sourceShard.PrimaryAlias) + if err != nil { + return fmt.Errorf("failed to get source shard %s primary", sourceShardName) + } + sourceSchema, err := td.wd.ct.tmc.GetSchema(ctx, sourceTablet.Tablet, &tabletmanagerdatapb.GetSchemaRequest{ + Tables: []string{td.table.Name}, + }) + if err != nil { + return err + } + sourceTable := sourceSchema.TableDefinitions[0] + sourcePKColumns := make(map[string]struct{}, len(sourceTable.PrimaryKeyColumns)) + td.tablePlan.sourcePkCols = make([]int, 0, len(sourceTable.PrimaryKeyColumns)) + for _, pkc := range sourceTable.PrimaryKeyColumns { + sourcePKColumns[pkc] = struct{}{} + } + for i, pkc := range td.table.PrimaryKeyColumns { + if _, ok := sourcePKColumns[pkc]; ok { + td.tablePlan.sourcePkCols = append(td.tablePlan.sourcePkCols, i) + } + } + return nil +} + func getColumnNameForSelectExpr(selectExpression sqlparser.SelectExpr) (string, error) { aliasedExpr := selectExpression.(*sqlparser.AliasedExpr) expr := aliasedExpr.Expr diff --git a/go/vt/vttablet/tabletmanager/vdiff/workflow_differ.go b/go/vt/vttablet/tabletmanager/vdiff/workflow_differ.go index 9e0ee6cbc41..34fd3cfad19 100644 --- a/go/vt/vttablet/tabletmanager/vdiff/workflow_differ.go +++ b/go/vt/vttablet/tabletmanager/vdiff/workflow_differ.go @@ -26,7 +26,6 @@ import ( "strings" "time" - "golang.org/x/exp/maps" "google.golang.org/protobuf/encoding/prototext" "vitess.io/vitess/go/mysql/collations" @@ -49,6 +48,11 @@ import ( vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc" ) +const ( + source = "source" + target = "target" +) + // workflowDiffer has metadata and state for the vdiff of a single workflow on this tablet // only one vdiff can be running for a workflow at any time. type workflowDiffer struct { @@ -379,8 +383,8 @@ func (wd *workflowDiffer) buildPlan(dbClient binlogplayer.DBClient, filter *binl if err != nil { return err } - td.lastSourcePK = lastPK["source"] - td.lastTargetPK = lastPK["target"] + td.lastSourcePK = lastPK[source] + td.lastTargetPK = lastPK[target] wd.tableDiffers[table.Name] = td if _, err := td.buildTablePlan(dbClient, wd.ct.vde.dbName, wd.collationEnv); err != nil { return err @@ -388,42 +392,9 @@ func (wd *workflowDiffer) buildPlan(dbClient binlogplayer.DBClient, filter *binl // We get the PK columns from the source schema as well, as they can // differ and determine the proper lastPK to use when saving progress. - // We use the first sourceShard as all of them should have the same schema. - sourceShardName := maps.Keys(wd.ct.sources)[0] - sourceTS, err := wd.getSourceTopoServer() - if err != nil { - return vterrors.Wrap(err, "failed to get source topo server") - } - sourceShard, err := sourceTS.GetShard(wd.ct.vde.ctx, wd.ct.sourceKeyspace, sourceShardName) - if err != nil { - return err - } - if sourceShard.PrimaryAlias == nil { - return fmt.Errorf("source shard %s has no primary", sourceShardName) - } - sourceTablet, err := sourceTS.GetTablet(wd.ct.vde.ctx, sourceShard.PrimaryAlias) - if err != nil { - return fmt.Errorf("failed to get source shard %s primary", sourceShardName) - } - sourceSchema, err := wd.ct.tmc.GetSchema(wd.ct.vde.ctx, sourceTablet.Tablet, &tabletmanagerdatapb.GetSchemaRequest{ - Tables: []string{table.Name}, - }) - if err != nil { + if err := td.getSourcePKCols(); err != nil { return err } - //log.Errorf("DEBUG: sourceTable.PrimaryKeyColumns: %v", sourceSchema.TableDefinitions[0].PrimaryKeyColumns) - sourcePKColumns := make(map[string]struct{}, len(sourceSchema.TableDefinitions[0].PrimaryKeyColumns)) - td.tablePlan.sourcePkCols = make([]int, 0, len(sourceSchema.TableDefinitions[0].PrimaryKeyColumns)) - for _, pkc := range sourceSchema.TableDefinitions[0].PrimaryKeyColumns { - sourcePKColumns[pkc] = struct{}{} - } - //log.Errorf("DEBUG: sourcePKColumns: %v", sourcePKColumns) - for i, pkc := range table.PrimaryKeyColumns { - if _, ok := sourcePKColumns[pkc]; ok { - td.tablePlan.sourcePkCols = append(td.tablePlan.sourcePkCols, i) - } - } - //log.Errorf("DEBUG: td.tablePlan.sourcePkCols: %v", td.tablePlan.sourcePkCols) } if len(wd.tableDiffers) == 0 { return fmt.Errorf("no tables found to diff, %s:%s, on tablet %v", @@ -456,14 +427,12 @@ func (wd *workflowDiffer) getTableLastPK(dbClient binlogplayer.DBClient, tableNa if err := json.Unmarshal(lastpk, &lastPK); err != nil { return nil, vterrors.Wrapf(err, "failed to unmarshal lastpk JSON for table %s", tableName) } - //log.Errorf("DEBUG: getTabletLastPK lastPKBytes: %v", lastPK) for k, v := range lastPK { lastPKResults[k] = &querypb.QueryResult{} if err := prototext.Unmarshal([]byte(v), lastPKResults[k]); err != nil { return nil, vterrors.Wrapf(err, "failed to unmarshal lastpk QueryResult for table %s", tableName) } } - //log.Errorf("DEBUG: getTabletLastPK lastPKRResults: %v", lastPKResults) return lastPKResults, nil } } @@ -549,5 +518,7 @@ func (wd *workflowDiffer) getSourceTopoServer() (*topo.Server, error) { if wd.ct.externalCluster == "" { return wd.ct.ts, nil } - return wd.ct.ts.OpenExternalVitessClusterServer(wd.ct.vde.ctx, wd.ct.externalCluster) + ctx, cancel := context.WithTimeout(wd.ct.vde.ctx, topo.RemoteOperationTimeout) + defer cancel() + return wd.ct.ts.OpenExternalVitessClusterServer(ctx, wd.ct.externalCluster) } From a325187da981cafa8707906bcab909de501c391b Mon Sep 17 00:00:00 2001 From: Matt Lord Date: Thu, 9 Jan 2025 09:45:20 -0500 Subject: [PATCH 4/7] Refactor vdiff lastpk handling Signed-off-by: Matt Lord --- .../tabletmanagerdata/tabletmanagerdata.pb.go | 1042 +++++++++-------- .../tabletmanagerdata_vtproto.pb.go | 212 ++++ .../tabletmanager/vdiff/table_differ.go | 76 +- .../tabletmanager/vdiff/workflow_differ.go | 31 +- proto/tabletmanagerdata.proto | 5 + web/vtadmin/src/proto/vtadmin.d.ts | 103 ++ web/vtadmin/src/proto/vtadmin.js | 237 ++++ 7 files changed, 1152 insertions(+), 554 deletions(-) diff --git a/go/vt/proto/tabletmanagerdata/tabletmanagerdata.pb.go b/go/vt/proto/tabletmanagerdata/tabletmanagerdata.pb.go index 67eae2395ad..0f804371153 100644 --- a/go/vt/proto/tabletmanagerdata/tabletmanagerdata.pb.go +++ b/go/vt/proto/tabletmanagerdata/tabletmanagerdata.pb.go @@ -6766,6 +6766,59 @@ func (x *VDiffOptions) GetReportOptions() *VDiffReportOptions { return nil } +type VDiffTableLastPK struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Source *query.QueryResult `protobuf:"bytes,1,opt,name=source,proto3" json:"source,omitempty"` + Target *query.QueryResult `protobuf:"bytes,2,opt,name=target,proto3" json:"target,omitempty"` +} + +func (x *VDiffTableLastPK) Reset() { + *x = VDiffTableLastPK{} + mi := &file_tabletmanagerdata_proto_msgTypes[132] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *VDiffTableLastPK) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*VDiffTableLastPK) ProtoMessage() {} + +func (x *VDiffTableLastPK) ProtoReflect() protoreflect.Message { + mi := &file_tabletmanagerdata_proto_msgTypes[132] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use VDiffTableLastPK.ProtoReflect.Descriptor instead. +func (*VDiffTableLastPK) Descriptor() ([]byte, []int) { + return file_tabletmanagerdata_proto_rawDescGZIP(), []int{132} +} + +func (x *VDiffTableLastPK) GetSource() *query.QueryResult { + if x != nil { + return x.Source + } + return nil +} + +func (x *VDiffTableLastPK) GetTarget() *query.QueryResult { + if x != nil { + return x.Target + } + return nil +} + // UpdateVReplicationWorkflowRequest is used to update an existing VReplication // workflow. Note that the following fields MUST have an explicit value provided // if you do NOT wish to update the existing value to the given type's ZeroValue: @@ -6788,7 +6841,7 @@ type UpdateVReplicationWorkflowRequest struct { func (x *UpdateVReplicationWorkflowRequest) Reset() { *x = UpdateVReplicationWorkflowRequest{} - mi := &file_tabletmanagerdata_proto_msgTypes[132] + mi := &file_tabletmanagerdata_proto_msgTypes[133] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6800,7 +6853,7 @@ func (x *UpdateVReplicationWorkflowRequest) String() string { func (*UpdateVReplicationWorkflowRequest) ProtoMessage() {} func (x *UpdateVReplicationWorkflowRequest) ProtoReflect() protoreflect.Message { - mi := &file_tabletmanagerdata_proto_msgTypes[132] + mi := &file_tabletmanagerdata_proto_msgTypes[133] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6813,7 +6866,7 @@ func (x *UpdateVReplicationWorkflowRequest) ProtoReflect() protoreflect.Message // Deprecated: Use UpdateVReplicationWorkflowRequest.ProtoReflect.Descriptor instead. func (*UpdateVReplicationWorkflowRequest) Descriptor() ([]byte, []int) { - return file_tabletmanagerdata_proto_rawDescGZIP(), []int{132} + return file_tabletmanagerdata_proto_rawDescGZIP(), []int{133} } func (x *UpdateVReplicationWorkflowRequest) GetWorkflow() string { @@ -6875,7 +6928,7 @@ type UpdateVReplicationWorkflowResponse struct { func (x *UpdateVReplicationWorkflowResponse) Reset() { *x = UpdateVReplicationWorkflowResponse{} - mi := &file_tabletmanagerdata_proto_msgTypes[133] + mi := &file_tabletmanagerdata_proto_msgTypes[134] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6887,7 +6940,7 @@ func (x *UpdateVReplicationWorkflowResponse) String() string { func (*UpdateVReplicationWorkflowResponse) ProtoMessage() {} func (x *UpdateVReplicationWorkflowResponse) ProtoReflect() protoreflect.Message { - mi := &file_tabletmanagerdata_proto_msgTypes[133] + mi := &file_tabletmanagerdata_proto_msgTypes[134] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6900,7 +6953,7 @@ func (x *UpdateVReplicationWorkflowResponse) ProtoReflect() protoreflect.Message // Deprecated: Use UpdateVReplicationWorkflowResponse.ProtoReflect.Descriptor instead. func (*UpdateVReplicationWorkflowResponse) Descriptor() ([]byte, []int) { - return file_tabletmanagerdata_proto_rawDescGZIP(), []int{133} + return file_tabletmanagerdata_proto_rawDescGZIP(), []int{134} } func (x *UpdateVReplicationWorkflowResponse) GetResult() *query.QueryResult { @@ -6931,7 +6984,7 @@ type UpdateVReplicationWorkflowsRequest struct { func (x *UpdateVReplicationWorkflowsRequest) Reset() { *x = UpdateVReplicationWorkflowsRequest{} - mi := &file_tabletmanagerdata_proto_msgTypes[134] + mi := &file_tabletmanagerdata_proto_msgTypes[135] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6943,7 +6996,7 @@ func (x *UpdateVReplicationWorkflowsRequest) String() string { func (*UpdateVReplicationWorkflowsRequest) ProtoMessage() {} func (x *UpdateVReplicationWorkflowsRequest) ProtoReflect() protoreflect.Message { - mi := &file_tabletmanagerdata_proto_msgTypes[134] + mi := &file_tabletmanagerdata_proto_msgTypes[135] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6956,7 +7009,7 @@ func (x *UpdateVReplicationWorkflowsRequest) ProtoReflect() protoreflect.Message // Deprecated: Use UpdateVReplicationWorkflowsRequest.ProtoReflect.Descriptor instead. func (*UpdateVReplicationWorkflowsRequest) Descriptor() ([]byte, []int) { - return file_tabletmanagerdata_proto_rawDescGZIP(), []int{134} + return file_tabletmanagerdata_proto_rawDescGZIP(), []int{135} } func (x *UpdateVReplicationWorkflowsRequest) GetAllWorkflows() bool { @@ -7011,7 +7064,7 @@ type UpdateVReplicationWorkflowsResponse struct { func (x *UpdateVReplicationWorkflowsResponse) Reset() { *x = UpdateVReplicationWorkflowsResponse{} - mi := &file_tabletmanagerdata_proto_msgTypes[135] + mi := &file_tabletmanagerdata_proto_msgTypes[136] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7023,7 +7076,7 @@ func (x *UpdateVReplicationWorkflowsResponse) String() string { func (*UpdateVReplicationWorkflowsResponse) ProtoMessage() {} func (x *UpdateVReplicationWorkflowsResponse) ProtoReflect() protoreflect.Message { - mi := &file_tabletmanagerdata_proto_msgTypes[135] + mi := &file_tabletmanagerdata_proto_msgTypes[136] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7036,7 +7089,7 @@ func (x *UpdateVReplicationWorkflowsResponse) ProtoReflect() protoreflect.Messag // Deprecated: Use UpdateVReplicationWorkflowsResponse.ProtoReflect.Descriptor instead. func (*UpdateVReplicationWorkflowsResponse) Descriptor() ([]byte, []int) { - return file_tabletmanagerdata_proto_rawDescGZIP(), []int{135} + return file_tabletmanagerdata_proto_rawDescGZIP(), []int{136} } func (x *UpdateVReplicationWorkflowsResponse) GetResult() *query.QueryResult { @@ -7056,7 +7109,7 @@ type ResetSequencesRequest struct { func (x *ResetSequencesRequest) Reset() { *x = ResetSequencesRequest{} - mi := &file_tabletmanagerdata_proto_msgTypes[136] + mi := &file_tabletmanagerdata_proto_msgTypes[137] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7068,7 +7121,7 @@ func (x *ResetSequencesRequest) String() string { func (*ResetSequencesRequest) ProtoMessage() {} func (x *ResetSequencesRequest) ProtoReflect() protoreflect.Message { - mi := &file_tabletmanagerdata_proto_msgTypes[136] + mi := &file_tabletmanagerdata_proto_msgTypes[137] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7081,7 +7134,7 @@ func (x *ResetSequencesRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ResetSequencesRequest.ProtoReflect.Descriptor instead. func (*ResetSequencesRequest) Descriptor() ([]byte, []int) { - return file_tabletmanagerdata_proto_rawDescGZIP(), []int{136} + return file_tabletmanagerdata_proto_rawDescGZIP(), []int{137} } func (x *ResetSequencesRequest) GetTables() []string { @@ -7099,7 +7152,7 @@ type ResetSequencesResponse struct { func (x *ResetSequencesResponse) Reset() { *x = ResetSequencesResponse{} - mi := &file_tabletmanagerdata_proto_msgTypes[137] + mi := &file_tabletmanagerdata_proto_msgTypes[138] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7111,7 +7164,7 @@ func (x *ResetSequencesResponse) String() string { func (*ResetSequencesResponse) ProtoMessage() {} func (x *ResetSequencesResponse) ProtoReflect() protoreflect.Message { - mi := &file_tabletmanagerdata_proto_msgTypes[137] + mi := &file_tabletmanagerdata_proto_msgTypes[138] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7124,7 +7177,7 @@ func (x *ResetSequencesResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ResetSequencesResponse.ProtoReflect.Descriptor instead. func (*ResetSequencesResponse) Descriptor() ([]byte, []int) { - return file_tabletmanagerdata_proto_rawDescGZIP(), []int{137} + return file_tabletmanagerdata_proto_rawDescGZIP(), []int{138} } type CheckThrottlerRequest struct { @@ -7145,7 +7198,7 @@ type CheckThrottlerRequest struct { func (x *CheckThrottlerRequest) Reset() { *x = CheckThrottlerRequest{} - mi := &file_tabletmanagerdata_proto_msgTypes[138] + mi := &file_tabletmanagerdata_proto_msgTypes[139] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7157,7 +7210,7 @@ func (x *CheckThrottlerRequest) String() string { func (*CheckThrottlerRequest) ProtoMessage() {} func (x *CheckThrottlerRequest) ProtoReflect() protoreflect.Message { - mi := &file_tabletmanagerdata_proto_msgTypes[138] + mi := &file_tabletmanagerdata_proto_msgTypes[139] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7170,7 +7223,7 @@ func (x *CheckThrottlerRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use CheckThrottlerRequest.ProtoReflect.Descriptor instead. func (*CheckThrottlerRequest) Descriptor() ([]byte, []int) { - return file_tabletmanagerdata_proto_rawDescGZIP(), []int{138} + return file_tabletmanagerdata_proto_rawDescGZIP(), []int{139} } func (x *CheckThrottlerRequest) GetAppName() string { @@ -7239,7 +7292,7 @@ type CheckThrottlerResponse struct { func (x *CheckThrottlerResponse) Reset() { *x = CheckThrottlerResponse{} - mi := &file_tabletmanagerdata_proto_msgTypes[139] + mi := &file_tabletmanagerdata_proto_msgTypes[140] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7251,7 +7304,7 @@ func (x *CheckThrottlerResponse) String() string { func (*CheckThrottlerResponse) ProtoMessage() {} func (x *CheckThrottlerResponse) ProtoReflect() protoreflect.Message { - mi := &file_tabletmanagerdata_proto_msgTypes[139] + mi := &file_tabletmanagerdata_proto_msgTypes[140] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7264,7 +7317,7 @@ func (x *CheckThrottlerResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use CheckThrottlerResponse.ProtoReflect.Descriptor instead. func (*CheckThrottlerResponse) Descriptor() ([]byte, []int) { - return file_tabletmanagerdata_proto_rawDescGZIP(), []int{139} + return file_tabletmanagerdata_proto_rawDescGZIP(), []int{140} } func (x *CheckThrottlerResponse) GetStatusCode() int32 { @@ -7345,7 +7398,7 @@ type GetThrottlerStatusRequest struct { func (x *GetThrottlerStatusRequest) Reset() { *x = GetThrottlerStatusRequest{} - mi := &file_tabletmanagerdata_proto_msgTypes[140] + mi := &file_tabletmanagerdata_proto_msgTypes[141] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7357,7 +7410,7 @@ func (x *GetThrottlerStatusRequest) String() string { func (*GetThrottlerStatusRequest) ProtoMessage() {} func (x *GetThrottlerStatusRequest) ProtoReflect() protoreflect.Message { - mi := &file_tabletmanagerdata_proto_msgTypes[140] + mi := &file_tabletmanagerdata_proto_msgTypes[141] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7370,7 +7423,7 @@ func (x *GetThrottlerStatusRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use GetThrottlerStatusRequest.ProtoReflect.Descriptor instead. func (*GetThrottlerStatusRequest) Descriptor() ([]byte, []int) { - return file_tabletmanagerdata_proto_rawDescGZIP(), []int{140} + return file_tabletmanagerdata_proto_rawDescGZIP(), []int{141} } type GetThrottlerStatusResponse struct { @@ -7418,7 +7471,7 @@ type GetThrottlerStatusResponse struct { func (x *GetThrottlerStatusResponse) Reset() { *x = GetThrottlerStatusResponse{} - mi := &file_tabletmanagerdata_proto_msgTypes[141] + mi := &file_tabletmanagerdata_proto_msgTypes[142] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7430,7 +7483,7 @@ func (x *GetThrottlerStatusResponse) String() string { func (*GetThrottlerStatusResponse) ProtoMessage() {} func (x *GetThrottlerStatusResponse) ProtoReflect() protoreflect.Message { - mi := &file_tabletmanagerdata_proto_msgTypes[141] + mi := &file_tabletmanagerdata_proto_msgTypes[142] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7443,7 +7496,7 @@ func (x *GetThrottlerStatusResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use GetThrottlerStatusResponse.ProtoReflect.Descriptor instead. func (*GetThrottlerStatusResponse) Descriptor() ([]byte, []int) { - return file_tabletmanagerdata_proto_rawDescGZIP(), []int{141} + return file_tabletmanagerdata_proto_rawDescGZIP(), []int{142} } func (x *GetThrottlerStatusResponse) GetTabletAlias() string { @@ -7583,7 +7636,7 @@ type ChangeTagsRequest struct { func (x *ChangeTagsRequest) Reset() { *x = ChangeTagsRequest{} - mi := &file_tabletmanagerdata_proto_msgTypes[142] + mi := &file_tabletmanagerdata_proto_msgTypes[143] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7595,7 +7648,7 @@ func (x *ChangeTagsRequest) String() string { func (*ChangeTagsRequest) ProtoMessage() {} func (x *ChangeTagsRequest) ProtoReflect() protoreflect.Message { - mi := &file_tabletmanagerdata_proto_msgTypes[142] + mi := &file_tabletmanagerdata_proto_msgTypes[143] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7608,7 +7661,7 @@ func (x *ChangeTagsRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ChangeTagsRequest.ProtoReflect.Descriptor instead. func (*ChangeTagsRequest) Descriptor() ([]byte, []int) { - return file_tabletmanagerdata_proto_rawDescGZIP(), []int{142} + return file_tabletmanagerdata_proto_rawDescGZIP(), []int{143} } func (x *ChangeTagsRequest) GetTags() map[string]string { @@ -7635,7 +7688,7 @@ type ChangeTagsResponse struct { func (x *ChangeTagsResponse) Reset() { *x = ChangeTagsResponse{} - mi := &file_tabletmanagerdata_proto_msgTypes[143] + mi := &file_tabletmanagerdata_proto_msgTypes[144] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7647,7 +7700,7 @@ func (x *ChangeTagsResponse) String() string { func (*ChangeTagsResponse) ProtoMessage() {} func (x *ChangeTagsResponse) ProtoReflect() protoreflect.Message { - mi := &file_tabletmanagerdata_proto_msgTypes[143] + mi := &file_tabletmanagerdata_proto_msgTypes[144] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7660,7 +7713,7 @@ func (x *ChangeTagsResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ChangeTagsResponse.ProtoReflect.Descriptor instead. func (*ChangeTagsResponse) Descriptor() ([]byte, []int) { - return file_tabletmanagerdata_proto_rawDescGZIP(), []int{143} + return file_tabletmanagerdata_proto_rawDescGZIP(), []int{144} } func (x *ChangeTagsResponse) GetTags() map[string]string { @@ -7693,7 +7746,7 @@ type ReadVReplicationWorkflowResponse_Stream struct { func (x *ReadVReplicationWorkflowResponse_Stream) Reset() { *x = ReadVReplicationWorkflowResponse_Stream{} - mi := &file_tabletmanagerdata_proto_msgTypes[149] + mi := &file_tabletmanagerdata_proto_msgTypes[150] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7705,7 +7758,7 @@ func (x *ReadVReplicationWorkflowResponse_Stream) String() string { func (*ReadVReplicationWorkflowResponse_Stream) ProtoMessage() {} func (x *ReadVReplicationWorkflowResponse_Stream) ProtoReflect() protoreflect.Message { - mi := &file_tabletmanagerdata_proto_msgTypes[149] + mi := &file_tabletmanagerdata_proto_msgTypes[150] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7844,7 +7897,7 @@ type CheckThrottlerResponse_Metric struct { func (x *CheckThrottlerResponse_Metric) Reset() { *x = CheckThrottlerResponse_Metric{} - mi := &file_tabletmanagerdata_proto_msgTypes[152] + mi := &file_tabletmanagerdata_proto_msgTypes[153] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7856,7 +7909,7 @@ func (x *CheckThrottlerResponse_Metric) String() string { func (*CheckThrottlerResponse_Metric) ProtoMessage() {} func (x *CheckThrottlerResponse_Metric) ProtoReflect() protoreflect.Message { - mi := &file_tabletmanagerdata_proto_msgTypes[152] + mi := &file_tabletmanagerdata_proto_msgTypes[153] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7869,7 +7922,7 @@ func (x *CheckThrottlerResponse_Metric) ProtoReflect() protoreflect.Message { // Deprecated: Use CheckThrottlerResponse_Metric.ProtoReflect.Descriptor instead. func (*CheckThrottlerResponse_Metric) Descriptor() ([]byte, []int) { - return file_tabletmanagerdata_proto_rawDescGZIP(), []int{139, 0} + return file_tabletmanagerdata_proto_rawDescGZIP(), []int{140, 0} } func (x *CheckThrottlerResponse_Metric) GetName() string { @@ -7939,7 +7992,7 @@ type GetThrottlerStatusResponse_MetricResult struct { func (x *GetThrottlerStatusResponse_MetricResult) Reset() { *x = GetThrottlerStatusResponse_MetricResult{} - mi := &file_tabletmanagerdata_proto_msgTypes[154] + mi := &file_tabletmanagerdata_proto_msgTypes[155] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7951,7 +8004,7 @@ func (x *GetThrottlerStatusResponse_MetricResult) String() string { func (*GetThrottlerStatusResponse_MetricResult) ProtoMessage() {} func (x *GetThrottlerStatusResponse_MetricResult) ProtoReflect() protoreflect.Message { - mi := &file_tabletmanagerdata_proto_msgTypes[154] + mi := &file_tabletmanagerdata_proto_msgTypes[155] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7964,7 +8017,7 @@ func (x *GetThrottlerStatusResponse_MetricResult) ProtoReflect() protoreflect.Me // Deprecated: Use GetThrottlerStatusResponse_MetricResult.ProtoReflect.Descriptor instead. func (*GetThrottlerStatusResponse_MetricResult) Descriptor() ([]byte, []int) { - return file_tabletmanagerdata_proto_rawDescGZIP(), []int{141, 0} + return file_tabletmanagerdata_proto_rawDescGZIP(), []int{142, 0} } func (x *GetThrottlerStatusResponse_MetricResult) GetValue() float64 { @@ -7992,7 +8045,7 @@ type GetThrottlerStatusResponse_MetricHealth struct { func (x *GetThrottlerStatusResponse_MetricHealth) Reset() { *x = GetThrottlerStatusResponse_MetricHealth{} - mi := &file_tabletmanagerdata_proto_msgTypes[157] + mi := &file_tabletmanagerdata_proto_msgTypes[158] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8004,7 +8057,7 @@ func (x *GetThrottlerStatusResponse_MetricHealth) String() string { func (*GetThrottlerStatusResponse_MetricHealth) ProtoMessage() {} func (x *GetThrottlerStatusResponse_MetricHealth) ProtoReflect() protoreflect.Message { - mi := &file_tabletmanagerdata_proto_msgTypes[157] + mi := &file_tabletmanagerdata_proto_msgTypes[158] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8017,7 +8070,7 @@ func (x *GetThrottlerStatusResponse_MetricHealth) ProtoReflect() protoreflect.Me // Deprecated: Use GetThrottlerStatusResponse_MetricHealth.ProtoReflect.Descriptor instead. func (*GetThrottlerStatusResponse_MetricHealth) Descriptor() ([]byte, []int) { - return file_tabletmanagerdata_proto_rawDescGZIP(), []int{141, 3} + return file_tabletmanagerdata_proto_rawDescGZIP(), []int{142, 3} } func (x *GetThrottlerStatusResponse_MetricHealth) GetLastHealthyAt() *vttime.Time { @@ -8047,7 +8100,7 @@ type GetThrottlerStatusResponse_RecentApp struct { func (x *GetThrottlerStatusResponse_RecentApp) Reset() { *x = GetThrottlerStatusResponse_RecentApp{} - mi := &file_tabletmanagerdata_proto_msgTypes[161] + mi := &file_tabletmanagerdata_proto_msgTypes[162] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8059,7 +8112,7 @@ func (x *GetThrottlerStatusResponse_RecentApp) String() string { func (*GetThrottlerStatusResponse_RecentApp) ProtoMessage() {} func (x *GetThrottlerStatusResponse_RecentApp) ProtoReflect() protoreflect.Message { - mi := &file_tabletmanagerdata_proto_msgTypes[161] + mi := &file_tabletmanagerdata_proto_msgTypes[162] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8072,7 +8125,7 @@ func (x *GetThrottlerStatusResponse_RecentApp) ProtoReflect() protoreflect.Messa // Deprecated: Use GetThrottlerStatusResponse_RecentApp.ProtoReflect.Descriptor instead. func (*GetThrottlerStatusResponse_RecentApp) Descriptor() ([]byte, []int) { - return file_tabletmanagerdata_proto_rawDescGZIP(), []int{141, 7} + return file_tabletmanagerdata_proto_rawDescGZIP(), []int{142, 7} } func (x *GetThrottlerStatusResponse_RecentApp) GetCheckedAt() *vttime.Time { @@ -8961,317 +9014,323 @@ var file_tabletmanagerdata_proto_rawDesc = []byte{ 0x6e, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x44, 0x69, 0x66, 0x66, 0x52, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, - 0x0d, 0x72, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0xed, - 0x04, 0x0a, 0x21, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, - 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, - 0x12, 0x14, 0x0a, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, - 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x12, 0x37, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, - 0x5f, 0x74, 0x79, 0x70, 0x65, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0e, 0x32, 0x14, 0x2e, 0x74, - 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x54, 0x79, - 0x70, 0x65, 0x52, 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x54, 0x79, 0x70, 0x65, 0x73, 0x12, - 0x71, 0x0a, 0x1b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x73, 0x65, 0x6c, 0x65, 0x63, 0x74, - 0x69, 0x6f, 0x6e, 0x5f, 0x70, 0x72, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x18, 0x04, - 0x20, 0x01, 0x28, 0x0e, 0x32, 0x2c, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, - 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x53, - 0x65, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x72, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, - 0x63, 0x65, 0x48, 0x00, 0x52, 0x19, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x53, 0x65, 0x6c, 0x65, - 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x72, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x88, - 0x01, 0x01, 0x12, 0x33, 0x0a, 0x06, 0x6f, 0x6e, 0x5f, 0x64, 0x64, 0x6c, 0x18, 0x05, 0x20, 0x01, - 0x28, 0x0e, 0x32, 0x17, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, - 0x4f, 0x6e, 0x44, 0x44, 0x4c, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x01, 0x52, 0x05, 0x6f, - 0x6e, 0x44, 0x64, 0x6c, 0x88, 0x01, 0x01, 0x12, 0x40, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, - 0x18, 0x06, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x25, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, - 0x61, 0x74, 0x61, 0x2e, 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, 0x48, 0x02, 0x52, - 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x88, 0x01, 0x01, 0x12, 0x74, 0x0a, 0x10, 0x63, 0x6f, 0x6e, - 0x66, 0x69, 0x67, 0x5f, 0x6f, 0x76, 0x65, 0x72, 0x72, 0x69, 0x64, 0x65, 0x73, 0x18, 0x08, 0x20, - 0x03, 0x28, 0x0b, 0x32, 0x49, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, - 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x56, 0x52, - 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, - 0x6f, 0x77, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, - 0x4f, 0x76, 0x65, 0x72, 0x72, 0x69, 0x64, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0f, - 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x4f, 0x76, 0x65, 0x72, 0x72, 0x69, 0x64, 0x65, 0x73, 0x1a, - 0x42, 0x0a, 0x14, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x4f, 0x76, 0x65, 0x72, 0x72, 0x69, 0x64, - 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, - 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, - 0x02, 0x38, 0x01, 0x42, 0x1e, 0x0a, 0x1c, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x73, - 0x65, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x70, 0x72, 0x65, 0x66, 0x65, 0x72, 0x65, - 0x6e, 0x63, 0x65, 0x42, 0x09, 0x0a, 0x07, 0x5f, 0x6f, 0x6e, 0x5f, 0x64, 0x64, 0x6c, 0x42, 0x08, - 0x0a, 0x06, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x4a, 0x04, 0x08, 0x07, 0x10, 0x08, 0x22, 0x50, - 0x0a, 0x22, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2a, 0x0a, 0x06, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x18, 0x01, - 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, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, - 0x22, 0xd6, 0x02, 0x0a, 0x22, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x56, 0x52, 0x65, 0x70, 0x6c, - 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, - 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x23, 0x0a, 0x0d, 0x61, 0x6c, 0x6c, 0x5f, 0x77, - 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0c, - 0x61, 0x6c, 0x6c, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x12, 0x2b, 0x0a, 0x11, - 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, - 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x10, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, - 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x12, 0x2b, 0x0a, 0x11, 0x65, 0x78, 0x63, - 0x6c, 0x75, 0x64, 0x65, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x18, 0x03, - 0x20, 0x03, 0x28, 0x09, 0x52, 0x10, 0x65, 0x78, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x57, 0x6f, 0x72, - 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x12, 0x40, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, - 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x25, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, - 0x74, 0x61, 0x2e, 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, 0x48, 0x00, 0x52, 0x05, - 0x73, 0x74, 0x61, 0x74, 0x65, 0x88, 0x01, 0x01, 0x12, 0x1d, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, - 0x61, 0x67, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x48, 0x01, 0x52, 0x07, 0x6d, 0x65, 0x73, - 0x73, 0x61, 0x67, 0x65, 0x88, 0x01, 0x01, 0x12, 0x28, 0x0a, 0x0d, 0x73, 0x74, 0x6f, 0x70, 0x5f, - 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x48, 0x02, - 0x52, 0x0c, 0x73, 0x74, 0x6f, 0x70, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x88, 0x01, - 0x01, 0x42, 0x08, 0x0a, 0x06, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x42, 0x0a, 0x0a, 0x08, 0x5f, - 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x42, 0x10, 0x0a, 0x0e, 0x5f, 0x73, 0x74, 0x6f, 0x70, - 0x5f, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x51, 0x0a, 0x23, 0x55, 0x70, 0x64, - 0x61, 0x74, 0x65, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, - 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x0d, 0x72, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0x6a, + 0x0a, 0x10, 0x56, 0x44, 0x69, 0x66, 0x66, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4c, 0x61, 0x73, 0x74, + 0x50, 0x4b, 0x12, 0x2a, 0x0a, 0x06, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x18, 0x01, 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, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x12, 0x2a, + 0x0a, 0x06, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x18, 0x02, 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, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x22, 0xed, 0x04, 0x0a, 0x21, 0x55, + 0x70, 0x64, 0x61, 0x74, 0x65, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x12, 0x1a, 0x0a, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x12, 0x14, 0x0a, 0x05, + 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x05, 0x63, 0x65, 0x6c, + 0x6c, 0x73, 0x12, 0x37, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x74, 0x79, 0x70, + 0x65, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0e, 0x32, 0x14, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, + 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0b, + 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x54, 0x79, 0x70, 0x65, 0x73, 0x12, 0x71, 0x0a, 0x1b, 0x74, + 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x73, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, + 0x70, 0x72, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, + 0x32, 0x2c, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, + 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x53, 0x65, 0x6c, 0x65, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x72, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x48, 0x00, + 0x52, 0x19, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x50, 0x72, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x88, 0x01, 0x01, 0x12, 0x33, + 0x0a, 0x06, 0x6f, 0x6e, 0x5f, 0x64, 0x64, 0x6c, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x17, + 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4f, 0x6e, 0x44, 0x44, + 0x4c, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x01, 0x52, 0x05, 0x6f, 0x6e, 0x44, 0x64, 0x6c, + 0x88, 0x01, 0x01, 0x12, 0x40, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x06, 0x20, 0x01, + 0x28, 0x0e, 0x32, 0x25, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, + 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, 0x48, 0x02, 0x52, 0x05, 0x73, 0x74, 0x61, + 0x74, 0x65, 0x88, 0x01, 0x01, 0x12, 0x74, 0x0a, 0x10, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x5f, + 0x6f, 0x76, 0x65, 0x72, 0x72, 0x69, 0x64, 0x65, 0x73, 0x18, 0x08, 0x20, 0x03, 0x28, 0x0b, 0x32, + 0x49, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, + 0x61, 0x74, 0x61, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, + 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x4f, 0x76, 0x65, 0x72, + 0x72, 0x69, 0x64, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0f, 0x63, 0x6f, 0x6e, 0x66, + 0x69, 0x67, 0x4f, 0x76, 0x65, 0x72, 0x72, 0x69, 0x64, 0x65, 0x73, 0x1a, 0x42, 0x0a, 0x14, 0x43, + 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x4f, 0x76, 0x65, 0x72, 0x72, 0x69, 0x64, 0x65, 0x73, 0x45, 0x6e, + 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x42, + 0x1e, 0x0a, 0x1c, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x73, 0x65, 0x6c, 0x65, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x70, 0x72, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x42, + 0x09, 0x0a, 0x07, 0x5f, 0x6f, 0x6e, 0x5f, 0x64, 0x64, 0x6c, 0x42, 0x08, 0x0a, 0x06, 0x5f, 0x73, + 0x74, 0x61, 0x74, 0x65, 0x4a, 0x04, 0x08, 0x07, 0x10, 0x08, 0x22, 0x50, 0x0a, 0x22, 0x55, 0x70, + 0x64, 0x61, 0x74, 0x65, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2a, 0x0a, 0x06, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x18, 0x01, 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, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x22, 0x2f, 0x0a, 0x15, - 0x52, 0x65, 0x73, 0x65, 0x74, 0x53, 0x65, 0x71, 0x75, 0x65, 0x6e, 0x63, 0x65, 0x73, 0x52, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x18, - 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x22, 0x18, 0x0a, - 0x16, 0x52, 0x65, 0x73, 0x65, 0x74, 0x53, 0x65, 0x71, 0x75, 0x65, 0x6e, 0x63, 0x65, 0x73, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xdd, 0x01, 0x0a, 0x15, 0x43, 0x68, 0x65, 0x63, - 0x6b, 0x54, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x12, 0x19, 0x0a, 0x08, 0x61, 0x70, 0x70, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x07, 0x61, 0x70, 0x70, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x14, 0x0a, 0x05, - 0x73, 0x63, 0x6f, 0x70, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x63, 0x6f, - 0x70, 0x65, 0x12, 0x36, 0x0a, 0x17, 0x73, 0x6b, 0x69, 0x70, 0x5f, 0x72, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x5f, 0x68, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x73, 0x18, 0x03, 0x20, - 0x01, 0x28, 0x08, 0x52, 0x15, 0x73, 0x6b, 0x69, 0x70, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x48, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x73, 0x12, 0x27, 0x0a, 0x10, 0x6f, 0x6b, - 0x5f, 0x69, 0x66, 0x5f, 0x6e, 0x6f, 0x74, 0x5f, 0x65, 0x78, 0x69, 0x73, 0x74, 0x73, 0x18, 0x04, - 0x20, 0x01, 0x28, 0x08, 0x52, 0x0d, 0x6f, 0x6b, 0x49, 0x66, 0x4e, 0x6f, 0x74, 0x45, 0x78, 0x69, - 0x73, 0x74, 0x73, 0x12, 0x32, 0x0a, 0x15, 0x6d, 0x75, 0x6c, 0x74, 0x69, 0x5f, 0x6d, 0x65, 0x74, - 0x72, 0x69, 0x63, 0x73, 0x5f, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x64, 0x18, 0x05, 0x20, 0x01, - 0x28, 0x08, 0x52, 0x13, 0x6d, 0x75, 0x6c, 0x74, 0x69, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, - 0x45, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x64, 0x22, 0x9f, 0x06, 0x0a, 0x16, 0x43, 0x68, 0x65, 0x63, - 0x6b, 0x54, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x5f, 0x63, 0x6f, 0x64, - 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0a, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x43, - 0x6f, 0x64, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x01, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x74, 0x68, 0x72, - 0x65, 0x73, 0x68, 0x6f, 0x6c, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x01, 0x52, 0x09, 0x74, 0x68, - 0x72, 0x65, 0x73, 0x68, 0x6f, 0x6c, 0x64, 0x12, 0x14, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, - 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x18, 0x0a, - 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, - 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x29, 0x0a, 0x10, 0x72, 0x65, 0x63, 0x65, 0x6e, - 0x74, 0x6c, 0x79, 0x5f, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x65, 0x64, 0x18, 0x06, 0x20, 0x01, 0x28, - 0x08, 0x52, 0x0f, 0x72, 0x65, 0x63, 0x65, 0x6e, 0x74, 0x6c, 0x79, 0x43, 0x68, 0x65, 0x63, 0x6b, - 0x65, 0x64, 0x12, 0x50, 0x0a, 0x07, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x18, 0x07, 0x20, - 0x03, 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, - 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x54, 0x68, 0x72, - 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x4d, - 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x07, 0x6d, 0x65, 0x74, - 0x72, 0x69, 0x63, 0x73, 0x12, 0x19, 0x0a, 0x08, 0x61, 0x70, 0x70, 0x5f, 0x6e, 0x61, 0x6d, 0x65, - 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x61, 0x70, 0x70, 0x4e, 0x61, 0x6d, 0x65, 0x12, - 0x18, 0x0a, 0x07, 0x73, 0x75, 0x6d, 0x6d, 0x61, 0x72, 0x79, 0x18, 0x09, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x07, 0x73, 0x75, 0x6d, 0x6d, 0x61, 0x72, 0x79, 0x12, 0x52, 0x0a, 0x0d, 0x72, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x5f, 0x63, 0x6f, 0x64, 0x65, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0e, - 0x32, 0x2d, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, - 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x54, 0x68, 0x72, 0x6f, 0x74, 0x74, - 0x6c, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x43, 0x6f, 0x64, 0x65, 0x52, - 0x0c, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x43, 0x6f, 0x64, 0x65, 0x1a, 0x8b, 0x02, - 0x0a, 0x06, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x1f, 0x0a, 0x0b, - 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x5f, 0x63, 0x6f, 0x64, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x05, 0x52, 0x0a, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x43, 0x6f, 0x64, 0x65, 0x12, 0x14, 0x0a, - 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x01, 0x52, 0x05, 0x76, 0x61, - 0x6c, 0x75, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x74, 0x68, 0x72, 0x65, 0x73, 0x68, 0x6f, 0x6c, 0x64, - 0x18, 0x04, 0x20, 0x01, 0x28, 0x01, 0x52, 0x09, 0x74, 0x68, 0x72, 0x65, 0x73, 0x68, 0x6f, 0x6c, - 0x64, 0x12, 0x14, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, - 0x67, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, - 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x63, 0x6f, 0x70, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x05, 0x73, 0x63, 0x6f, 0x70, 0x65, 0x12, 0x52, 0x0a, 0x0d, 0x72, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x5f, 0x63, 0x6f, 0x64, 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x2d, - 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, - 0x74, 0x61, 0x2e, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x54, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, - 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x43, 0x6f, 0x64, 0x65, 0x52, 0x0c, 0x72, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x43, 0x6f, 0x64, 0x65, 0x1a, 0x6c, 0x0a, 0x0c, 0x4d, - 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, - 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x46, 0x0a, - 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x30, 0x2e, 0x74, + 0x73, 0x75, 0x6c, 0x74, 0x52, 0x06, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x22, 0xd6, 0x02, 0x0a, + 0x22, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x12, 0x23, 0x0a, 0x0d, 0x61, 0x6c, 0x6c, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0c, 0x61, 0x6c, 0x6c, 0x57, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x12, 0x2b, 0x0a, 0x11, 0x69, 0x6e, 0x63, 0x6c, + 0x75, 0x64, 0x65, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x18, 0x02, 0x20, + 0x03, 0x28, 0x09, 0x52, 0x10, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x57, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x12, 0x2b, 0x0a, 0x11, 0x65, 0x78, 0x63, 0x6c, 0x75, 0x64, 0x65, + 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, + 0x52, 0x10, 0x65, 0x78, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x73, 0x12, 0x40, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, + 0x0e, 0x32, 0x25, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 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, 0x48, 0x00, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, + 0x65, 0x88, 0x01, 0x01, 0x12, 0x1d, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, + 0x05, 0x20, 0x01, 0x28, 0x09, 0x48, 0x01, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, + 0x88, 0x01, 0x01, 0x12, 0x28, 0x0a, 0x0d, 0x73, 0x74, 0x6f, 0x70, 0x5f, 0x70, 0x6f, 0x73, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x48, 0x02, 0x52, 0x0c, 0x73, 0x74, + 0x6f, 0x70, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x88, 0x01, 0x01, 0x42, 0x08, 0x0a, + 0x06, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x42, 0x0a, 0x0a, 0x08, 0x5f, 0x6d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x65, 0x42, 0x10, 0x0a, 0x0e, 0x5f, 0x73, 0x74, 0x6f, 0x70, 0x5f, 0x70, 0x6f, 0x73, + 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x51, 0x0a, 0x23, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x56, + 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2a, 0x0a, 0x06, + 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x18, 0x01, 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, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x22, 0x2f, 0x0a, 0x15, 0x52, 0x65, 0x73, 0x65, + 0x74, 0x53, 0x65, 0x71, 0x75, 0x65, 0x6e, 0x63, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x12, 0x16, 0x0a, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, + 0x09, 0x52, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x22, 0x18, 0x0a, 0x16, 0x52, 0x65, 0x73, + 0x65, 0x74, 0x53, 0x65, 0x71, 0x75, 0x65, 0x6e, 0x63, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x22, 0xdd, 0x01, 0x0a, 0x15, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x54, 0x68, 0x72, + 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x19, 0x0a, + 0x08, 0x61, 0x70, 0x70, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x07, 0x61, 0x70, 0x70, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x63, 0x6f, 0x70, + 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x63, 0x6f, 0x70, 0x65, 0x12, 0x36, + 0x0a, 0x17, 0x73, 0x6b, 0x69, 0x70, 0x5f, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x68, + 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, + 0x15, 0x73, 0x6b, 0x69, 0x70, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x65, 0x61, 0x72, + 0x74, 0x62, 0x65, 0x61, 0x74, 0x73, 0x12, 0x27, 0x0a, 0x10, 0x6f, 0x6b, 0x5f, 0x69, 0x66, 0x5f, + 0x6e, 0x6f, 0x74, 0x5f, 0x65, 0x78, 0x69, 0x73, 0x74, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, + 0x52, 0x0d, 0x6f, 0x6b, 0x49, 0x66, 0x4e, 0x6f, 0x74, 0x45, 0x78, 0x69, 0x73, 0x74, 0x73, 0x12, + 0x32, 0x0a, 0x15, 0x6d, 0x75, 0x6c, 0x74, 0x69, 0x5f, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, + 0x5f, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x64, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x13, + 0x6d, 0x75, 0x6c, 0x74, 0x69, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x45, 0x6e, 0x61, 0x62, + 0x6c, 0x65, 0x64, 0x22, 0x9f, 0x06, 0x0a, 0x16, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x54, 0x68, 0x72, + 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1f, + 0x0a, 0x0b, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x5f, 0x63, 0x6f, 0x64, 0x65, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x05, 0x52, 0x0a, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x43, 0x6f, 0x64, 0x65, 0x12, + 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x01, 0x52, 0x05, + 0x76, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x74, 0x68, 0x72, 0x65, 0x73, 0x68, 0x6f, + 0x6c, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x01, 0x52, 0x09, 0x74, 0x68, 0x72, 0x65, 0x73, 0x68, + 0x6f, 0x6c, 0x64, 0x12, 0x14, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x04, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x65, 0x12, 0x29, 0x0a, 0x10, 0x72, 0x65, 0x63, 0x65, 0x6e, 0x74, 0x6c, 0x79, 0x5f, + 0x63, 0x68, 0x65, 0x63, 0x6b, 0x65, 0x64, 0x18, 0x06, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0f, 0x72, + 0x65, 0x63, 0x65, 0x6e, 0x74, 0x6c, 0x79, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x65, 0x64, 0x12, 0x50, + 0x0a, 0x07, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x18, 0x07, 0x20, 0x03, 0x28, 0x0b, 0x32, + 0x36, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, + 0x61, 0x74, 0x61, 0x2e, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x54, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, + 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x4d, 0x65, 0x74, 0x72, 0x69, + 0x63, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x07, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, + 0x12, 0x19, 0x0a, 0x08, 0x61, 0x70, 0x70, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x08, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x07, 0x61, 0x70, 0x70, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x73, + 0x75, 0x6d, 0x6d, 0x61, 0x72, 0x79, 0x18, 0x09, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x73, 0x75, + 0x6d, 0x6d, 0x61, 0x72, 0x79, 0x12, 0x52, 0x0a, 0x0d, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x5f, 0x63, 0x6f, 0x64, 0x65, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x2d, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x54, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x72, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x52, 0x05, - 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x1b, 0x0a, 0x19, 0x47, 0x65, 0x74, - 0x54, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0xb6, 0x10, 0x0a, 0x1a, 0x47, 0x65, 0x74, 0x54, 0x68, - 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, - 0x61, 0x6c, 0x69, 0x61, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x74, 0x61, 0x62, - 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, - 0x70, 0x61, 0x63, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, - 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x03, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, 0x1b, 0x0a, 0x09, 0x69, 0x73, - 0x5f, 0x6c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, 0x69, - 0x73, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x17, 0x0a, 0x07, 0x69, 0x73, 0x5f, 0x6f, 0x70, - 0x65, 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x06, 0x69, 0x73, 0x4f, 0x70, 0x65, 0x6e, - 0x12, 0x1d, 0x0a, 0x0a, 0x69, 0x73, 0x5f, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x64, 0x18, 0x06, - 0x20, 0x01, 0x28, 0x08, 0x52, 0x09, 0x69, 0x73, 0x45, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x64, 0x12, - 0x1d, 0x0a, 0x0a, 0x69, 0x73, 0x5f, 0x64, 0x6f, 0x72, 0x6d, 0x61, 0x6e, 0x74, 0x18, 0x07, 0x20, - 0x01, 0x28, 0x08, 0x52, 0x09, 0x69, 0x73, 0x44, 0x6f, 0x72, 0x6d, 0x61, 0x6e, 0x74, 0x12, 0x28, - 0x0a, 0x10, 0x6c, 0x61, 0x67, 0x5f, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x5f, 0x71, 0x75, 0x65, - 0x72, 0x79, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x6c, 0x61, 0x67, 0x4d, 0x65, 0x74, - 0x72, 0x69, 0x63, 0x51, 0x75, 0x65, 0x72, 0x79, 0x12, 0x2e, 0x0a, 0x13, 0x63, 0x75, 0x73, 0x74, - 0x6f, 0x6d, 0x5f, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x5f, 0x71, 0x75, 0x65, 0x72, 0x79, 0x18, - 0x09, 0x20, 0x01, 0x28, 0x09, 0x52, 0x11, 0x63, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x4d, 0x65, 0x74, - 0x72, 0x69, 0x63, 0x51, 0x75, 0x65, 0x72, 0x79, 0x12, 0x2b, 0x0a, 0x11, 0x64, 0x65, 0x66, 0x61, - 0x75, 0x6c, 0x74, 0x5f, 0x74, 0x68, 0x72, 0x65, 0x73, 0x68, 0x6f, 0x6c, 0x64, 0x18, 0x0a, 0x20, - 0x01, 0x28, 0x01, 0x52, 0x10, 0x64, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x54, 0x68, 0x72, 0x65, - 0x73, 0x68, 0x6f, 0x6c, 0x64, 0x12, 0x3c, 0x0a, 0x1b, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x5f, - 0x6e, 0x61, 0x6d, 0x65, 0x5f, 0x75, 0x73, 0x65, 0x64, 0x5f, 0x61, 0x73, 0x5f, 0x64, 0x65, 0x66, - 0x61, 0x75, 0x6c, 0x74, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x09, 0x52, 0x17, 0x6d, 0x65, 0x74, 0x72, - 0x69, 0x63, 0x4e, 0x61, 0x6d, 0x65, 0x55, 0x73, 0x65, 0x64, 0x41, 0x73, 0x44, 0x65, 0x66, 0x61, - 0x75, 0x6c, 0x74, 0x12, 0x73, 0x0a, 0x12, 0x61, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, - 0x64, 0x5f, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x18, 0x0c, 0x20, 0x03, 0x28, 0x0b, 0x32, - 0x44, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, - 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x54, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x72, - 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x41, - 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x64, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, - 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x11, 0x61, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, - 0x64, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x12, 0x70, 0x0a, 0x11, 0x6d, 0x65, 0x74, 0x72, - 0x69, 0x63, 0x5f, 0x74, 0x68, 0x72, 0x65, 0x73, 0x68, 0x6f, 0x6c, 0x64, 0x73, 0x18, 0x0d, 0x20, - 0x03, 0x28, 0x0b, 0x32, 0x43, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, - 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x54, 0x68, 0x72, 0x6f, 0x74, - 0x74, 0x6c, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x2e, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x54, 0x68, 0x72, 0x65, 0x73, 0x68, 0x6f, - 0x6c, 0x64, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x10, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, - 0x54, 0x68, 0x72, 0x65, 0x73, 0x68, 0x6f, 0x6c, 0x64, 0x73, 0x12, 0x67, 0x0a, 0x0e, 0x6d, 0x65, - 0x74, 0x72, 0x69, 0x63, 0x73, 0x5f, 0x68, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x18, 0x0e, 0x20, 0x03, - 0x28, 0x0b, 0x32, 0x40, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, - 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x54, 0x68, 0x72, 0x6f, 0x74, 0x74, - 0x6c, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x2e, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x45, - 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0d, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x48, 0x65, 0x61, - 0x6c, 0x74, 0x68, 0x12, 0x67, 0x0a, 0x0e, 0x74, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x64, - 0x5f, 0x61, 0x70, 0x70, 0x73, 0x18, 0x0f, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x40, 0x2e, 0x74, 0x61, - 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, - 0x47, 0x65, 0x74, 0x54, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, - 0x75, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x54, 0x68, 0x72, 0x6f, 0x74, - 0x74, 0x6c, 0x65, 0x64, 0x41, 0x70, 0x70, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0d, 0x74, - 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x64, 0x41, 0x70, 0x70, 0x73, 0x12, 0x74, 0x0a, 0x13, - 0x61, 0x70, 0x70, 0x5f, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x65, 0x64, 0x5f, 0x6d, 0x65, 0x74, 0x72, - 0x69, 0x63, 0x73, 0x18, 0x10, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x44, 0x2e, 0x74, 0x61, 0x62, 0x6c, - 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, - 0x74, 0x54, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, - 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x41, 0x70, 0x70, 0x43, 0x68, 0x65, 0x63, - 0x6b, 0x65, 0x64, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, - 0x11, 0x61, 0x70, 0x70, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x65, 0x64, 0x4d, 0x65, 0x74, 0x72, 0x69, - 0x63, 0x73, 0x12, 0x29, 0x0a, 0x10, 0x72, 0x65, 0x63, 0x65, 0x6e, 0x74, 0x6c, 0x79, 0x5f, 0x63, - 0x68, 0x65, 0x63, 0x6b, 0x65, 0x64, 0x18, 0x11, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0f, 0x72, 0x65, - 0x63, 0x65, 0x6e, 0x74, 0x6c, 0x79, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x65, 0x64, 0x12, 0x5e, 0x0a, - 0x0b, 0x72, 0x65, 0x63, 0x65, 0x6e, 0x74, 0x5f, 0x61, 0x70, 0x70, 0x73, 0x18, 0x12, 0x20, 0x03, - 0x28, 0x0b, 0x32, 0x3d, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, - 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x54, 0x68, 0x72, 0x6f, 0x74, 0x74, - 0x6c, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x2e, 0x52, 0x65, 0x63, 0x65, 0x6e, 0x74, 0x41, 0x70, 0x70, 0x73, 0x45, 0x6e, 0x74, 0x72, - 0x79, 0x52, 0x0a, 0x72, 0x65, 0x63, 0x65, 0x6e, 0x74, 0x41, 0x70, 0x70, 0x73, 0x1a, 0x3a, 0x0a, - 0x0c, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x14, 0x0a, - 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x01, 0x52, 0x05, 0x76, 0x61, - 0x6c, 0x75, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x1a, 0x80, 0x01, 0x0a, 0x16, 0x41, 0x67, - 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x64, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x45, - 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x50, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, - 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3a, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, - 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x54, 0x68, 0x72, - 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x52, 0x65, 0x73, 0x75, 0x6c, - 0x74, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x43, 0x0a, 0x15, - 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x54, 0x68, 0x72, 0x65, 0x73, 0x68, 0x6f, 0x6c, 0x64, 0x73, - 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x01, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, - 0x01, 0x1a, 0x81, 0x01, 0x0a, 0x0c, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x48, 0x65, 0x61, 0x6c, - 0x74, 0x68, 0x12, 0x34, 0x0a, 0x0f, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x68, 0x65, 0x61, 0x6c, 0x74, - 0x68, 0x79, 0x5f, 0x61, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x76, 0x74, - 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x52, 0x0d, 0x6c, 0x61, 0x73, 0x74, 0x48, - 0x65, 0x61, 0x6c, 0x74, 0x68, 0x79, 0x41, 0x74, 0x12, 0x3b, 0x0a, 0x1a, 0x73, 0x65, 0x63, 0x6f, - 0x6e, 0x64, 0x73, 0x5f, 0x73, 0x69, 0x6e, 0x63, 0x65, 0x5f, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x68, - 0x65, 0x61, 0x6c, 0x74, 0x68, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x17, 0x73, 0x65, - 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x53, 0x69, 0x6e, 0x63, 0x65, 0x4c, 0x61, 0x73, 0x74, 0x48, 0x65, - 0x61, 0x6c, 0x74, 0x68, 0x79, 0x1a, 0x7c, 0x0a, 0x12, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, - 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, - 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x50, 0x0a, - 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3a, 0x2e, 0x74, - 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, - 0x2e, 0x47, 0x65, 0x74, 0x54, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x72, 0x53, 0x74, 0x61, - 0x74, 0x75, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x4d, 0x65, 0x74, 0x72, - 0x69, 0x63, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, - 0x02, 0x38, 0x01, 0x1a, 0x5c, 0x0a, 0x12, 0x54, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x64, - 0x41, 0x70, 0x70, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x30, 0x0a, 0x05, 0x76, - 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x74, 0x6f, 0x70, - 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x64, 0x41, - 0x70, 0x70, 0x52, 0x75, 0x6c, 0x65, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, - 0x01, 0x1a, 0x44, 0x0a, 0x16, 0x41, 0x70, 0x70, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x65, 0x64, 0x4d, - 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, - 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, - 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, - 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0xad, 0x01, 0x0a, 0x09, 0x52, 0x65, 0x63, 0x65, - 0x6e, 0x74, 0x41, 0x70, 0x70, 0x12, 0x2b, 0x0a, 0x0a, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x65, 0x64, - 0x5f, 0x61, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x76, 0x74, 0x74, 0x69, - 0x6d, 0x65, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x52, 0x09, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x65, 0x64, - 0x41, 0x74, 0x12, 0x1f, 0x0a, 0x0b, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x5f, 0x63, 0x6f, 0x64, - 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0a, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x43, - 0x6f, 0x64, 0x65, 0x12, 0x52, 0x0a, 0x0d, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x5f, - 0x63, 0x6f, 0x64, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x2d, 0x2e, 0x74, 0x61, 0x62, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x43, 0x6f, 0x64, 0x65, 0x52, 0x0c, 0x72, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x43, 0x6f, 0x64, 0x65, 0x1a, 0x8b, 0x02, 0x0a, 0x06, 0x4d, 0x65, + 0x74, 0x72, 0x69, 0x63, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x73, 0x74, 0x61, 0x74, + 0x75, 0x73, 0x5f, 0x63, 0x6f, 0x64, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0a, 0x73, + 0x74, 0x61, 0x74, 0x75, 0x73, 0x43, 0x6f, 0x64, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, + 0x75, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x01, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x12, + 0x1c, 0x0a, 0x09, 0x74, 0x68, 0x72, 0x65, 0x73, 0x68, 0x6f, 0x6c, 0x64, 0x18, 0x04, 0x20, 0x01, + 0x28, 0x01, 0x52, 0x09, 0x74, 0x68, 0x72, 0x65, 0x73, 0x68, 0x6f, 0x6c, 0x64, 0x12, 0x14, 0x0a, + 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x65, 0x72, + 0x72, 0x6f, 0x72, 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x06, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x14, 0x0a, + 0x05, 0x73, 0x63, 0x6f, 0x70, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x63, + 0x6f, 0x70, 0x65, 0x12, 0x52, 0x0a, 0x0d, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x5f, + 0x63, 0x6f, 0x64, 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x2d, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x54, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x43, 0x6f, 0x64, 0x65, 0x52, 0x0c, 0x72, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x43, 0x6f, 0x64, 0x65, 0x1a, 0x76, 0x0a, 0x0f, 0x52, 0x65, 0x63, 0x65, 0x6e, - 0x74, 0x41, 0x70, 0x70, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, - 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x4d, 0x0a, 0x05, - 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x37, 0x2e, 0x74, 0x61, + 0x6e, 0x73, 0x65, 0x43, 0x6f, 0x64, 0x65, 0x1a, 0x6c, 0x0a, 0x0c, 0x4d, 0x65, 0x74, 0x72, 0x69, + 0x63, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x46, 0x0a, 0x05, 0x76, 0x61, 0x6c, + 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x30, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, + 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, 0x68, 0x65, + 0x63, 0x6b, 0x54, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x2e, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, + 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x1b, 0x0a, 0x19, 0x47, 0x65, 0x74, 0x54, 0x68, 0x72, 0x6f, + 0x74, 0x74, 0x6c, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x22, 0xb6, 0x10, 0x0a, 0x1a, 0x47, 0x65, 0x74, 0x54, 0x68, 0x72, 0x6f, 0x74, 0x74, + 0x6c, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x61, 0x6c, 0x69, 0x61, + 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, + 0x6c, 0x69, 0x61, 0x73, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, + 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, 0x1b, 0x0a, 0x09, 0x69, 0x73, 0x5f, 0x6c, 0x65, 0x61, + 0x64, 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, 0x69, 0x73, 0x4c, 0x65, 0x61, + 0x64, 0x65, 0x72, 0x12, 0x17, 0x0a, 0x07, 0x69, 0x73, 0x5f, 0x6f, 0x70, 0x65, 0x6e, 0x18, 0x05, + 0x20, 0x01, 0x28, 0x08, 0x52, 0x06, 0x69, 0x73, 0x4f, 0x70, 0x65, 0x6e, 0x12, 0x1d, 0x0a, 0x0a, + 0x69, 0x73, 0x5f, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x64, 0x18, 0x06, 0x20, 0x01, 0x28, 0x08, + 0x52, 0x09, 0x69, 0x73, 0x45, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x69, + 0x73, 0x5f, 0x64, 0x6f, 0x72, 0x6d, 0x61, 0x6e, 0x74, 0x18, 0x07, 0x20, 0x01, 0x28, 0x08, 0x52, + 0x09, 0x69, 0x73, 0x44, 0x6f, 0x72, 0x6d, 0x61, 0x6e, 0x74, 0x12, 0x28, 0x0a, 0x10, 0x6c, 0x61, + 0x67, 0x5f, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x5f, 0x71, 0x75, 0x65, 0x72, 0x79, 0x18, 0x08, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x6c, 0x61, 0x67, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x51, + 0x75, 0x65, 0x72, 0x79, 0x12, 0x2e, 0x0a, 0x13, 0x63, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x5f, 0x6d, + 0x65, 0x74, 0x72, 0x69, 0x63, 0x5f, 0x71, 0x75, 0x65, 0x72, 0x79, 0x18, 0x09, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x11, 0x63, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x51, + 0x75, 0x65, 0x72, 0x79, 0x12, 0x2b, 0x0a, 0x11, 0x64, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x5f, + 0x74, 0x68, 0x72, 0x65, 0x73, 0x68, 0x6f, 0x6c, 0x64, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x01, 0x52, + 0x10, 0x64, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x54, 0x68, 0x72, 0x65, 0x73, 0x68, 0x6f, 0x6c, + 0x64, 0x12, 0x3c, 0x0a, 0x1b, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x5f, 0x6e, 0x61, 0x6d, 0x65, + 0x5f, 0x75, 0x73, 0x65, 0x64, 0x5f, 0x61, 0x73, 0x5f, 0x64, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, + 0x18, 0x0b, 0x20, 0x01, 0x28, 0x09, 0x52, 0x17, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x4e, 0x61, + 0x6d, 0x65, 0x55, 0x73, 0x65, 0x64, 0x41, 0x73, 0x44, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x12, + 0x73, 0x0a, 0x12, 0x61, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x64, 0x5f, 0x6d, 0x65, + 0x74, 0x72, 0x69, 0x63, 0x73, 0x18, 0x0c, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x44, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x54, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, - 0x75, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x52, 0x65, 0x63, 0x65, 0x6e, - 0x74, 0x41, 0x70, 0x70, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, - 0xaa, 0x01, 0x0a, 0x11, 0x43, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x54, 0x61, 0x67, 0x73, 0x52, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x42, 0x0a, 0x04, 0x74, 0x61, 0x67, 0x73, 0x18, 0x01, 0x20, - 0x03, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, - 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x54, 0x61, - 0x67, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x54, 0x61, 0x67, 0x73, 0x45, 0x6e, - 0x74, 0x72, 0x79, 0x52, 0x04, 0x74, 0x61, 0x67, 0x73, 0x12, 0x18, 0x0a, 0x07, 0x72, 0x65, 0x70, - 0x6c, 0x61, 0x63, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x72, 0x65, 0x70, 0x6c, - 0x61, 0x63, 0x65, 0x1a, 0x37, 0x0a, 0x09, 0x54, 0x61, 0x67, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, + 0x75, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x41, 0x67, 0x67, 0x72, 0x65, + 0x67, 0x61, 0x74, 0x65, 0x64, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x45, 0x6e, 0x74, 0x72, + 0x79, 0x52, 0x11, 0x61, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x64, 0x4d, 0x65, 0x74, + 0x72, 0x69, 0x63, 0x73, 0x12, 0x70, 0x0a, 0x11, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x5f, 0x74, + 0x68, 0x72, 0x65, 0x73, 0x68, 0x6f, 0x6c, 0x64, 0x73, 0x18, 0x0d, 0x20, 0x03, 0x28, 0x0b, 0x32, + 0x43, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, + 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x54, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x72, + 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x4d, + 0x65, 0x74, 0x72, 0x69, 0x63, 0x54, 0x68, 0x72, 0x65, 0x73, 0x68, 0x6f, 0x6c, 0x64, 0x73, 0x45, + 0x6e, 0x74, 0x72, 0x79, 0x52, 0x10, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x54, 0x68, 0x72, 0x65, + 0x73, 0x68, 0x6f, 0x6c, 0x64, 0x73, 0x12, 0x67, 0x0a, 0x0e, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, + 0x73, 0x5f, 0x68, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x18, 0x0e, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x40, + 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, + 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x54, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x72, 0x53, + 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x4d, 0x65, + 0x74, 0x72, 0x69, 0x63, 0x73, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x45, 0x6e, 0x74, 0x72, 0x79, + 0x52, 0x0d, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x12, + 0x67, 0x0a, 0x0e, 0x74, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x64, 0x5f, 0x61, 0x70, 0x70, + 0x73, 0x18, 0x0f, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x40, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, + 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x54, + 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x54, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x64, + 0x41, 0x70, 0x70, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0d, 0x74, 0x68, 0x72, 0x6f, 0x74, + 0x74, 0x6c, 0x65, 0x64, 0x41, 0x70, 0x70, 0x73, 0x12, 0x74, 0x0a, 0x13, 0x61, 0x70, 0x70, 0x5f, + 0x63, 0x68, 0x65, 0x63, 0x6b, 0x65, 0x64, 0x5f, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x18, + 0x10, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x44, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, + 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x54, 0x68, 0x72, + 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x41, 0x70, 0x70, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x65, 0x64, 0x4d, + 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x11, 0x61, 0x70, 0x70, + 0x43, 0x68, 0x65, 0x63, 0x6b, 0x65, 0x64, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x12, 0x29, + 0x0a, 0x10, 0x72, 0x65, 0x63, 0x65, 0x6e, 0x74, 0x6c, 0x79, 0x5f, 0x63, 0x68, 0x65, 0x63, 0x6b, + 0x65, 0x64, 0x18, 0x11, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0f, 0x72, 0x65, 0x63, 0x65, 0x6e, 0x74, + 0x6c, 0x79, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x65, 0x64, 0x12, 0x5e, 0x0a, 0x0b, 0x72, 0x65, 0x63, + 0x65, 0x6e, 0x74, 0x5f, 0x61, 0x70, 0x70, 0x73, 0x18, 0x12, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3d, + 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, + 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x54, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x72, 0x53, + 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x52, 0x65, + 0x63, 0x65, 0x6e, 0x74, 0x41, 0x70, 0x70, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0a, 0x72, + 0x65, 0x63, 0x65, 0x6e, 0x74, 0x41, 0x70, 0x70, 0x73, 0x1a, 0x3a, 0x0a, 0x0c, 0x4d, 0x65, 0x74, + 0x72, 0x69, 0x63, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, + 0x75, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x01, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x12, + 0x14, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, + 0x65, 0x72, 0x72, 0x6f, 0x72, 0x1a, 0x80, 0x01, 0x0a, 0x16, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, + 0x61, 0x74, 0x65, 0x64, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, - 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x92, 0x01, 0x0a, - 0x12, 0x43, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x54, 0x61, 0x67, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x12, 0x43, 0x0a, 0x04, 0x74, 0x61, 0x67, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, - 0x0b, 0x32, 0x2f, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, - 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x54, 0x61, 0x67, 0x73, - 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x54, 0x61, 0x67, 0x73, 0x45, 0x6e, 0x74, - 0x72, 0x79, 0x52, 0x04, 0x74, 0x61, 0x67, 0x73, 0x1a, 0x37, 0x0a, 0x09, 0x54, 0x61, 0x67, 0x73, + 0x65, 0x79, 0x12, 0x50, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x3a, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, + 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x54, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, + 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x2e, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x52, 0x05, 0x76, + 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x43, 0x0a, 0x15, 0x4d, 0x65, 0x74, 0x72, + 0x69, 0x63, 0x54, 0x68, 0x72, 0x65, 0x73, 0x68, 0x6f, 0x6c, 0x64, 0x73, 0x45, 0x6e, 0x74, 0x72, + 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, + 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x01, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x81, 0x01, + 0x0a, 0x0c, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x12, 0x34, + 0x0a, 0x0f, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x68, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x79, 0x5f, 0x61, + 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, + 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x52, 0x0d, 0x6c, 0x61, 0x73, 0x74, 0x48, 0x65, 0x61, 0x6c, 0x74, + 0x68, 0x79, 0x41, 0x74, 0x12, 0x3b, 0x0a, 0x1a, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x5f, + 0x73, 0x69, 0x6e, 0x63, 0x65, 0x5f, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x68, 0x65, 0x61, 0x6c, 0x74, + 0x68, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x17, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, + 0x73, 0x53, 0x69, 0x6e, 0x63, 0x65, 0x4c, 0x61, 0x73, 0x74, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, + 0x79, 0x1a, 0x7c, 0x0a, 0x12, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x48, 0x65, 0x61, 0x6c, + 0x74, 0x68, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x50, 0x0a, 0x05, 0x76, 0x61, 0x6c, + 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3a, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, + 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, + 0x54, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x48, 0x65, + 0x61, 0x6c, 0x74, 0x68, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, + 0x5c, 0x0a, 0x12, 0x54, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x64, 0x41, 0x70, 0x70, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, - 0x01, 0x2a, 0x3e, 0x0a, 0x19, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x53, 0x65, 0x6c, 0x65, 0x63, - 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x72, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x12, 0x07, - 0x0a, 0x03, 0x41, 0x4e, 0x59, 0x10, 0x00, 0x12, 0x0b, 0x0a, 0x07, 0x49, 0x4e, 0x4f, 0x52, 0x44, - 0x45, 0x52, 0x10, 0x01, 0x12, 0x0b, 0x0a, 0x07, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, - 0x03, 0x2a, 0x83, 0x01, 0x0a, 0x1a, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x54, 0x68, 0x72, 0x6f, 0x74, - 0x74, 0x6c, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x43, 0x6f, 0x64, 0x65, - 0x12, 0x0d, 0x0a, 0x09, 0x55, 0x4e, 0x44, 0x45, 0x46, 0x49, 0x4e, 0x45, 0x44, 0x10, 0x00, 0x12, - 0x06, 0x0a, 0x02, 0x4f, 0x4b, 0x10, 0x01, 0x12, 0x16, 0x0a, 0x12, 0x54, 0x48, 0x52, 0x45, 0x53, - 0x48, 0x4f, 0x4c, 0x44, 0x5f, 0x45, 0x58, 0x43, 0x45, 0x45, 0x44, 0x45, 0x44, 0x10, 0x02, 0x12, - 0x0e, 0x0a, 0x0a, 0x41, 0x50, 0x50, 0x5f, 0x44, 0x45, 0x4e, 0x49, 0x45, 0x44, 0x10, 0x03, 0x12, - 0x12, 0x0a, 0x0e, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x5f, 0x4d, 0x45, 0x54, 0x52, 0x49, - 0x43, 0x10, 0x04, 0x12, 0x12, 0x0a, 0x0e, 0x49, 0x4e, 0x54, 0x45, 0x52, 0x4e, 0x41, 0x4c, 0x5f, - 0x45, 0x52, 0x52, 0x4f, 0x52, 0x10, 0x05, 0x42, 0x30, 0x5a, 0x2e, 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, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, - 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x33, + 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x30, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, + 0x61, 0x2e, 0x54, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x64, 0x41, 0x70, 0x70, 0x52, 0x75, + 0x6c, 0x65, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x44, 0x0a, + 0x16, 0x41, 0x70, 0x70, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x65, 0x64, 0x4d, 0x65, 0x74, 0x72, 0x69, + 0x63, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, + 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, + 0x02, 0x38, 0x01, 0x1a, 0xad, 0x01, 0x0a, 0x09, 0x52, 0x65, 0x63, 0x65, 0x6e, 0x74, 0x41, 0x70, + 0x70, 0x12, 0x2b, 0x0a, 0x0a, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x65, 0x64, 0x5f, 0x61, 0x74, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x54, + 0x69, 0x6d, 0x65, 0x52, 0x09, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x65, 0x64, 0x41, 0x74, 0x12, 0x1f, + 0x0a, 0x0b, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x5f, 0x63, 0x6f, 0x64, 0x65, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x05, 0x52, 0x0a, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x43, 0x6f, 0x64, 0x65, 0x12, + 0x52, 0x0a, 0x0d, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x5f, 0x63, 0x6f, 0x64, 0x65, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x2d, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, + 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, 0x68, 0x65, 0x63, 0x6b, + 0x54, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x43, 0x6f, 0x64, 0x65, 0x52, 0x0c, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x43, + 0x6f, 0x64, 0x65, 0x1a, 0x76, 0x0a, 0x0f, 0x52, 0x65, 0x63, 0x65, 0x6e, 0x74, 0x41, 0x70, 0x70, + 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x4d, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, + 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x37, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, + 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x54, + 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x52, 0x65, 0x63, 0x65, 0x6e, 0x74, 0x41, 0x70, 0x70, + 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xaa, 0x01, 0x0a, 0x11, + 0x43, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x54, 0x61, 0x67, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x12, 0x42, 0x0a, 0x04, 0x74, 0x61, 0x67, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, + 0x2e, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, + 0x61, 0x74, 0x61, 0x2e, 0x43, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x54, 0x61, 0x67, 0x73, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x54, 0x61, 0x67, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, + 0x04, 0x74, 0x61, 0x67, 0x73, 0x12, 0x18, 0x0a, 0x07, 0x72, 0x65, 0x70, 0x6c, 0x61, 0x63, 0x65, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x72, 0x65, 0x70, 0x6c, 0x61, 0x63, 0x65, 0x1a, + 0x37, 0x0a, 0x09, 0x54, 0x61, 0x67, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, + 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, + 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, + 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x92, 0x01, 0x0a, 0x12, 0x43, 0x68, 0x61, + 0x6e, 0x67, 0x65, 0x54, 0x61, 0x67, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, + 0x43, 0x0a, 0x04, 0x74, 0x61, 0x67, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2f, 0x2e, + 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, + 0x61, 0x2e, 0x43, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x54, 0x61, 0x67, 0x73, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x54, 0x61, 0x67, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x04, + 0x74, 0x61, 0x67, 0x73, 0x1a, 0x37, 0x0a, 0x09, 0x54, 0x61, 0x67, 0x73, 0x45, 0x6e, 0x74, 0x72, + 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, + 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x2a, 0x3e, 0x0a, + 0x19, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x50, 0x72, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x12, 0x07, 0x0a, 0x03, 0x41, 0x4e, + 0x59, 0x10, 0x00, 0x12, 0x0b, 0x0a, 0x07, 0x49, 0x4e, 0x4f, 0x52, 0x44, 0x45, 0x52, 0x10, 0x01, + 0x12, 0x0b, 0x0a, 0x07, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0x03, 0x2a, 0x83, 0x01, + 0x0a, 0x1a, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x54, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x72, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x43, 0x6f, 0x64, 0x65, 0x12, 0x0d, 0x0a, 0x09, + 0x55, 0x4e, 0x44, 0x45, 0x46, 0x49, 0x4e, 0x45, 0x44, 0x10, 0x00, 0x12, 0x06, 0x0a, 0x02, 0x4f, + 0x4b, 0x10, 0x01, 0x12, 0x16, 0x0a, 0x12, 0x54, 0x48, 0x52, 0x45, 0x53, 0x48, 0x4f, 0x4c, 0x44, + 0x5f, 0x45, 0x58, 0x43, 0x45, 0x45, 0x44, 0x45, 0x44, 0x10, 0x02, 0x12, 0x0e, 0x0a, 0x0a, 0x41, + 0x50, 0x50, 0x5f, 0x44, 0x45, 0x4e, 0x49, 0x45, 0x44, 0x10, 0x03, 0x12, 0x12, 0x0a, 0x0e, 0x55, + 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x5f, 0x4d, 0x45, 0x54, 0x52, 0x49, 0x43, 0x10, 0x04, 0x12, + 0x12, 0x0a, 0x0e, 0x49, 0x4e, 0x54, 0x45, 0x52, 0x4e, 0x41, 0x4c, 0x5f, 0x45, 0x52, 0x52, 0x4f, + 0x52, 0x10, 0x05, 0x42, 0x30, 0x5a, 0x2e, 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, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, + 0x72, 0x64, 0x61, 0x74, 0x61, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -9287,7 +9346,7 @@ func file_tabletmanagerdata_proto_rawDescGZIP() []byte { } var file_tabletmanagerdata_proto_enumTypes = make([]protoimpl.EnumInfo, 2) -var file_tabletmanagerdata_proto_msgTypes = make([]protoimpl.MessageInfo, 165) +var file_tabletmanagerdata_proto_msgTypes = make([]protoimpl.MessageInfo, 166) var file_tabletmanagerdata_proto_goTypes = []any{ (TabletSelectionPreference)(0), // 0: tabletmanagerdata.TabletSelectionPreference (CheckThrottlerResponseCode)(0), // 1: tabletmanagerdata.CheckThrottlerResponseCode @@ -9423,163 +9482,166 @@ var file_tabletmanagerdata_proto_goTypes = []any{ (*VDiffReportOptions)(nil), // 131: tabletmanagerdata.VDiffReportOptions (*VDiffCoreOptions)(nil), // 132: tabletmanagerdata.VDiffCoreOptions (*VDiffOptions)(nil), // 133: tabletmanagerdata.VDiffOptions - (*UpdateVReplicationWorkflowRequest)(nil), // 134: tabletmanagerdata.UpdateVReplicationWorkflowRequest - (*UpdateVReplicationWorkflowResponse)(nil), // 135: tabletmanagerdata.UpdateVReplicationWorkflowResponse - (*UpdateVReplicationWorkflowsRequest)(nil), // 136: tabletmanagerdata.UpdateVReplicationWorkflowsRequest - (*UpdateVReplicationWorkflowsResponse)(nil), // 137: tabletmanagerdata.UpdateVReplicationWorkflowsResponse - (*ResetSequencesRequest)(nil), // 138: tabletmanagerdata.ResetSequencesRequest - (*ResetSequencesResponse)(nil), // 139: tabletmanagerdata.ResetSequencesResponse - (*CheckThrottlerRequest)(nil), // 140: tabletmanagerdata.CheckThrottlerRequest - (*CheckThrottlerResponse)(nil), // 141: tabletmanagerdata.CheckThrottlerResponse - (*GetThrottlerStatusRequest)(nil), // 142: tabletmanagerdata.GetThrottlerStatusRequest - (*GetThrottlerStatusResponse)(nil), // 143: tabletmanagerdata.GetThrottlerStatusResponse - (*ChangeTagsRequest)(nil), // 144: tabletmanagerdata.ChangeTagsRequest - (*ChangeTagsResponse)(nil), // 145: tabletmanagerdata.ChangeTagsResponse - nil, // 146: tabletmanagerdata.UserPermission.PrivilegesEntry - nil, // 147: tabletmanagerdata.DbPermission.PrivilegesEntry - nil, // 148: tabletmanagerdata.ExecuteHookRequest.ExtraEnvEntry - nil, // 149: tabletmanagerdata.GetGlobalStatusVarsResponse.StatusValuesEntry - nil, // 150: tabletmanagerdata.DeleteTableDataRequest.TableFiltersEntry - (*ReadVReplicationWorkflowResponse_Stream)(nil), // 151: tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream - nil, // 152: tabletmanagerdata.ReadVReplicationWorkflowResponse.ConfigOverridesEntry - nil, // 153: tabletmanagerdata.UpdateVReplicationWorkflowRequest.ConfigOverridesEntry - (*CheckThrottlerResponse_Metric)(nil), // 154: tabletmanagerdata.CheckThrottlerResponse.Metric - nil, // 155: tabletmanagerdata.CheckThrottlerResponse.MetricsEntry - (*GetThrottlerStatusResponse_MetricResult)(nil), // 156: tabletmanagerdata.GetThrottlerStatusResponse.MetricResult - nil, // 157: tabletmanagerdata.GetThrottlerStatusResponse.AggregatedMetricsEntry - nil, // 158: tabletmanagerdata.GetThrottlerStatusResponse.MetricThresholdsEntry - (*GetThrottlerStatusResponse_MetricHealth)(nil), // 159: tabletmanagerdata.GetThrottlerStatusResponse.MetricHealth - nil, // 160: tabletmanagerdata.GetThrottlerStatusResponse.MetricsHealthEntry - nil, // 161: tabletmanagerdata.GetThrottlerStatusResponse.ThrottledAppsEntry - nil, // 162: tabletmanagerdata.GetThrottlerStatusResponse.AppCheckedMetricsEntry - (*GetThrottlerStatusResponse_RecentApp)(nil), // 163: tabletmanagerdata.GetThrottlerStatusResponse.RecentApp - nil, // 164: tabletmanagerdata.GetThrottlerStatusResponse.RecentAppsEntry - nil, // 165: tabletmanagerdata.ChangeTagsRequest.TagsEntry - nil, // 166: tabletmanagerdata.ChangeTagsResponse.TagsEntry - (*query.Field)(nil), // 167: query.Field - (topodata.TabletType)(0), // 168: topodata.TabletType - (*vtrpc.CallerID)(nil), // 169: vtrpc.CallerID - (*query.QueryResult)(nil), // 170: query.QueryResult - (*query.TransactionMetadata)(nil), // 171: query.TransactionMetadata - (*mysqlctl.HostMetricsResponse)(nil), // 172: mysqlctl.HostMetricsResponse - (*replicationdata.Status)(nil), // 173: replicationdata.Status - (*replicationdata.PrimaryStatus)(nil), // 174: replicationdata.PrimaryStatus - (*topodata.TabletAlias)(nil), // 175: topodata.TabletAlias - (*replicationdata.FullStatus)(nil), // 176: replicationdata.FullStatus - (replicationdata.StopReplicationMode)(0), // 177: replicationdata.StopReplicationMode - (*replicationdata.StopReplicationStatus)(nil), // 178: replicationdata.StopReplicationStatus - (*logutil.Event)(nil), // 179: logutil.Event - (*vttime.Time)(nil), // 180: vttime.Time - (*binlogdata.BinlogSource)(nil), // 181: binlogdata.BinlogSource - (binlogdata.VReplicationWorkflowType)(0), // 182: binlogdata.VReplicationWorkflowType - (binlogdata.VReplicationWorkflowSubType)(0), // 183: binlogdata.VReplicationWorkflowSubType - (binlogdata.VReplicationWorkflowState)(0), // 184: binlogdata.VReplicationWorkflowState - (binlogdata.OnDDLAction)(0), // 185: binlogdata.OnDDLAction - (*topodata.ThrottledAppRule)(nil), // 186: topodata.ThrottledAppRule + (*VDiffTableLastPK)(nil), // 134: tabletmanagerdata.VDiffTableLastPK + (*UpdateVReplicationWorkflowRequest)(nil), // 135: tabletmanagerdata.UpdateVReplicationWorkflowRequest + (*UpdateVReplicationWorkflowResponse)(nil), // 136: tabletmanagerdata.UpdateVReplicationWorkflowResponse + (*UpdateVReplicationWorkflowsRequest)(nil), // 137: tabletmanagerdata.UpdateVReplicationWorkflowsRequest + (*UpdateVReplicationWorkflowsResponse)(nil), // 138: tabletmanagerdata.UpdateVReplicationWorkflowsResponse + (*ResetSequencesRequest)(nil), // 139: tabletmanagerdata.ResetSequencesRequest + (*ResetSequencesResponse)(nil), // 140: tabletmanagerdata.ResetSequencesResponse + (*CheckThrottlerRequest)(nil), // 141: tabletmanagerdata.CheckThrottlerRequest + (*CheckThrottlerResponse)(nil), // 142: tabletmanagerdata.CheckThrottlerResponse + (*GetThrottlerStatusRequest)(nil), // 143: tabletmanagerdata.GetThrottlerStatusRequest + (*GetThrottlerStatusResponse)(nil), // 144: tabletmanagerdata.GetThrottlerStatusResponse + (*ChangeTagsRequest)(nil), // 145: tabletmanagerdata.ChangeTagsRequest + (*ChangeTagsResponse)(nil), // 146: tabletmanagerdata.ChangeTagsResponse + nil, // 147: tabletmanagerdata.UserPermission.PrivilegesEntry + nil, // 148: tabletmanagerdata.DbPermission.PrivilegesEntry + nil, // 149: tabletmanagerdata.ExecuteHookRequest.ExtraEnvEntry + nil, // 150: tabletmanagerdata.GetGlobalStatusVarsResponse.StatusValuesEntry + nil, // 151: tabletmanagerdata.DeleteTableDataRequest.TableFiltersEntry + (*ReadVReplicationWorkflowResponse_Stream)(nil), // 152: tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream + nil, // 153: tabletmanagerdata.ReadVReplicationWorkflowResponse.ConfigOverridesEntry + nil, // 154: tabletmanagerdata.UpdateVReplicationWorkflowRequest.ConfigOverridesEntry + (*CheckThrottlerResponse_Metric)(nil), // 155: tabletmanagerdata.CheckThrottlerResponse.Metric + nil, // 156: tabletmanagerdata.CheckThrottlerResponse.MetricsEntry + (*GetThrottlerStatusResponse_MetricResult)(nil), // 157: tabletmanagerdata.GetThrottlerStatusResponse.MetricResult + nil, // 158: tabletmanagerdata.GetThrottlerStatusResponse.AggregatedMetricsEntry + nil, // 159: tabletmanagerdata.GetThrottlerStatusResponse.MetricThresholdsEntry + (*GetThrottlerStatusResponse_MetricHealth)(nil), // 160: tabletmanagerdata.GetThrottlerStatusResponse.MetricHealth + nil, // 161: tabletmanagerdata.GetThrottlerStatusResponse.MetricsHealthEntry + nil, // 162: tabletmanagerdata.GetThrottlerStatusResponse.ThrottledAppsEntry + nil, // 163: tabletmanagerdata.GetThrottlerStatusResponse.AppCheckedMetricsEntry + (*GetThrottlerStatusResponse_RecentApp)(nil), // 164: tabletmanagerdata.GetThrottlerStatusResponse.RecentApp + nil, // 165: tabletmanagerdata.GetThrottlerStatusResponse.RecentAppsEntry + nil, // 166: tabletmanagerdata.ChangeTagsRequest.TagsEntry + nil, // 167: tabletmanagerdata.ChangeTagsResponse.TagsEntry + (*query.Field)(nil), // 168: query.Field + (topodata.TabletType)(0), // 169: topodata.TabletType + (*vtrpc.CallerID)(nil), // 170: vtrpc.CallerID + (*query.QueryResult)(nil), // 171: query.QueryResult + (*query.TransactionMetadata)(nil), // 172: query.TransactionMetadata + (*mysqlctl.HostMetricsResponse)(nil), // 173: mysqlctl.HostMetricsResponse + (*replicationdata.Status)(nil), // 174: replicationdata.Status + (*replicationdata.PrimaryStatus)(nil), // 175: replicationdata.PrimaryStatus + (*topodata.TabletAlias)(nil), // 176: topodata.TabletAlias + (*replicationdata.FullStatus)(nil), // 177: replicationdata.FullStatus + (replicationdata.StopReplicationMode)(0), // 178: replicationdata.StopReplicationMode + (*replicationdata.StopReplicationStatus)(nil), // 179: replicationdata.StopReplicationStatus + (*logutil.Event)(nil), // 180: logutil.Event + (*vttime.Time)(nil), // 181: vttime.Time + (*binlogdata.BinlogSource)(nil), // 182: binlogdata.BinlogSource + (binlogdata.VReplicationWorkflowType)(0), // 183: binlogdata.VReplicationWorkflowType + (binlogdata.VReplicationWorkflowSubType)(0), // 184: binlogdata.VReplicationWorkflowSubType + (binlogdata.VReplicationWorkflowState)(0), // 185: binlogdata.VReplicationWorkflowState + (binlogdata.OnDDLAction)(0), // 186: binlogdata.OnDDLAction + (*topodata.ThrottledAppRule)(nil), // 187: topodata.ThrottledAppRule } var file_tabletmanagerdata_proto_depIdxs = []int32{ - 167, // 0: tabletmanagerdata.TableDefinition.fields:type_name -> query.Field + 168, // 0: tabletmanagerdata.TableDefinition.fields:type_name -> query.Field 2, // 1: tabletmanagerdata.SchemaDefinition.table_definitions:type_name -> tabletmanagerdata.TableDefinition 3, // 2: tabletmanagerdata.SchemaChangeResult.before_schema:type_name -> tabletmanagerdata.SchemaDefinition 3, // 3: tabletmanagerdata.SchemaChangeResult.after_schema:type_name -> tabletmanagerdata.SchemaDefinition - 146, // 4: tabletmanagerdata.UserPermission.privileges:type_name -> tabletmanagerdata.UserPermission.PrivilegesEntry - 147, // 5: tabletmanagerdata.DbPermission.privileges:type_name -> tabletmanagerdata.DbPermission.PrivilegesEntry + 147, // 4: tabletmanagerdata.UserPermission.privileges:type_name -> tabletmanagerdata.UserPermission.PrivilegesEntry + 148, // 5: tabletmanagerdata.DbPermission.privileges:type_name -> tabletmanagerdata.DbPermission.PrivilegesEntry 5, // 6: tabletmanagerdata.Permissions.user_permissions:type_name -> tabletmanagerdata.UserPermission 6, // 7: tabletmanagerdata.Permissions.db_permissions:type_name -> tabletmanagerdata.DbPermission - 148, // 8: tabletmanagerdata.ExecuteHookRequest.extra_env:type_name -> tabletmanagerdata.ExecuteHookRequest.ExtraEnvEntry + 149, // 8: tabletmanagerdata.ExecuteHookRequest.extra_env:type_name -> tabletmanagerdata.ExecuteHookRequest.ExtraEnvEntry 3, // 9: tabletmanagerdata.GetSchemaResponse.schema_definition:type_name -> tabletmanagerdata.SchemaDefinition 7, // 10: tabletmanagerdata.GetPermissionsResponse.permissions:type_name -> tabletmanagerdata.Permissions - 149, // 11: tabletmanagerdata.GetGlobalStatusVarsResponse.status_values:type_name -> tabletmanagerdata.GetGlobalStatusVarsResponse.StatusValuesEntry - 168, // 12: tabletmanagerdata.ChangeTypeRequest.tablet_type:type_name -> topodata.TabletType + 150, // 11: tabletmanagerdata.GetGlobalStatusVarsResponse.status_values:type_name -> tabletmanagerdata.GetGlobalStatusVarsResponse.StatusValuesEntry + 169, // 12: tabletmanagerdata.ChangeTypeRequest.tablet_type:type_name -> topodata.TabletType 4, // 13: tabletmanagerdata.PreflightSchemaResponse.change_results:type_name -> tabletmanagerdata.SchemaChangeResult 3, // 14: tabletmanagerdata.ApplySchemaRequest.before_schema:type_name -> tabletmanagerdata.SchemaDefinition 3, // 15: tabletmanagerdata.ApplySchemaRequest.after_schema:type_name -> tabletmanagerdata.SchemaDefinition 3, // 16: tabletmanagerdata.ApplySchemaResponse.before_schema:type_name -> tabletmanagerdata.SchemaDefinition 3, // 17: tabletmanagerdata.ApplySchemaResponse.after_schema:type_name -> tabletmanagerdata.SchemaDefinition - 169, // 18: tabletmanagerdata.ExecuteQueryRequest.caller_id:type_name -> vtrpc.CallerID - 170, // 19: tabletmanagerdata.ExecuteQueryResponse.result:type_name -> query.QueryResult - 170, // 20: tabletmanagerdata.ExecuteFetchAsDbaResponse.result:type_name -> query.QueryResult - 170, // 21: tabletmanagerdata.ExecuteMultiFetchAsDbaResponse.results:type_name -> query.QueryResult - 170, // 22: tabletmanagerdata.ExecuteFetchAsAllPrivsResponse.result:type_name -> query.QueryResult - 170, // 23: tabletmanagerdata.ExecuteFetchAsAppResponse.result:type_name -> query.QueryResult - 171, // 24: tabletmanagerdata.GetUnresolvedTransactionsResponse.transactions:type_name -> query.TransactionMetadata - 171, // 25: tabletmanagerdata.ReadTransactionResponse.transaction:type_name -> query.TransactionMetadata - 172, // 26: tabletmanagerdata.MysqlHostMetricsResponse.HostMetrics:type_name -> mysqlctl.HostMetricsResponse - 173, // 27: tabletmanagerdata.ReplicationStatusResponse.status:type_name -> replicationdata.Status - 174, // 28: tabletmanagerdata.PrimaryStatusResponse.status:type_name -> replicationdata.PrimaryStatus - 170, // 29: tabletmanagerdata.VReplicationExecResponse.result:type_name -> query.QueryResult - 175, // 30: tabletmanagerdata.PopulateReparentJournalRequest.primary_alias:type_name -> topodata.TabletAlias - 175, // 31: tabletmanagerdata.InitReplicaRequest.parent:type_name -> topodata.TabletAlias - 174, // 32: tabletmanagerdata.DemotePrimaryResponse.primary_status:type_name -> replicationdata.PrimaryStatus - 176, // 33: tabletmanagerdata.FullStatusResponse.status:type_name -> replicationdata.FullStatus - 175, // 34: tabletmanagerdata.SetReplicationSourceRequest.parent:type_name -> topodata.TabletAlias - 175, // 35: tabletmanagerdata.ReplicaWasRestartedRequest.parent:type_name -> topodata.TabletAlias - 177, // 36: tabletmanagerdata.StopReplicationAndGetStatusRequest.stop_replication_mode:type_name -> replicationdata.StopReplicationMode - 178, // 37: tabletmanagerdata.StopReplicationAndGetStatusResponse.status:type_name -> replicationdata.StopReplicationStatus - 179, // 38: tabletmanagerdata.BackupResponse.event:type_name -> logutil.Event - 180, // 39: tabletmanagerdata.RestoreFromBackupRequest.backup_time:type_name -> vttime.Time - 180, // 40: tabletmanagerdata.RestoreFromBackupRequest.restore_to_timestamp:type_name -> vttime.Time - 179, // 41: tabletmanagerdata.RestoreFromBackupResponse.event:type_name -> logutil.Event - 181, // 42: tabletmanagerdata.CreateVReplicationWorkflowRequest.binlog_source:type_name -> binlogdata.BinlogSource - 168, // 43: tabletmanagerdata.CreateVReplicationWorkflowRequest.tablet_types:type_name -> topodata.TabletType + 170, // 18: tabletmanagerdata.ExecuteQueryRequest.caller_id:type_name -> vtrpc.CallerID + 171, // 19: tabletmanagerdata.ExecuteQueryResponse.result:type_name -> query.QueryResult + 171, // 20: tabletmanagerdata.ExecuteFetchAsDbaResponse.result:type_name -> query.QueryResult + 171, // 21: tabletmanagerdata.ExecuteMultiFetchAsDbaResponse.results:type_name -> query.QueryResult + 171, // 22: tabletmanagerdata.ExecuteFetchAsAllPrivsResponse.result:type_name -> query.QueryResult + 171, // 23: tabletmanagerdata.ExecuteFetchAsAppResponse.result:type_name -> query.QueryResult + 172, // 24: tabletmanagerdata.GetUnresolvedTransactionsResponse.transactions:type_name -> query.TransactionMetadata + 172, // 25: tabletmanagerdata.ReadTransactionResponse.transaction:type_name -> query.TransactionMetadata + 173, // 26: tabletmanagerdata.MysqlHostMetricsResponse.HostMetrics:type_name -> mysqlctl.HostMetricsResponse + 174, // 27: tabletmanagerdata.ReplicationStatusResponse.status:type_name -> replicationdata.Status + 175, // 28: tabletmanagerdata.PrimaryStatusResponse.status:type_name -> replicationdata.PrimaryStatus + 171, // 29: tabletmanagerdata.VReplicationExecResponse.result:type_name -> query.QueryResult + 176, // 30: tabletmanagerdata.PopulateReparentJournalRequest.primary_alias:type_name -> topodata.TabletAlias + 176, // 31: tabletmanagerdata.InitReplicaRequest.parent:type_name -> topodata.TabletAlias + 175, // 32: tabletmanagerdata.DemotePrimaryResponse.primary_status:type_name -> replicationdata.PrimaryStatus + 177, // 33: tabletmanagerdata.FullStatusResponse.status:type_name -> replicationdata.FullStatus + 176, // 34: tabletmanagerdata.SetReplicationSourceRequest.parent:type_name -> topodata.TabletAlias + 176, // 35: tabletmanagerdata.ReplicaWasRestartedRequest.parent:type_name -> topodata.TabletAlias + 178, // 36: tabletmanagerdata.StopReplicationAndGetStatusRequest.stop_replication_mode:type_name -> replicationdata.StopReplicationMode + 179, // 37: tabletmanagerdata.StopReplicationAndGetStatusResponse.status:type_name -> replicationdata.StopReplicationStatus + 180, // 38: tabletmanagerdata.BackupResponse.event:type_name -> logutil.Event + 181, // 39: tabletmanagerdata.RestoreFromBackupRequest.backup_time:type_name -> vttime.Time + 181, // 40: tabletmanagerdata.RestoreFromBackupRequest.restore_to_timestamp:type_name -> vttime.Time + 180, // 41: tabletmanagerdata.RestoreFromBackupResponse.event:type_name -> logutil.Event + 182, // 42: tabletmanagerdata.CreateVReplicationWorkflowRequest.binlog_source:type_name -> binlogdata.BinlogSource + 169, // 43: tabletmanagerdata.CreateVReplicationWorkflowRequest.tablet_types:type_name -> topodata.TabletType 0, // 44: tabletmanagerdata.CreateVReplicationWorkflowRequest.tablet_selection_preference:type_name -> tabletmanagerdata.TabletSelectionPreference - 182, // 45: tabletmanagerdata.CreateVReplicationWorkflowRequest.workflow_type:type_name -> binlogdata.VReplicationWorkflowType - 183, // 46: tabletmanagerdata.CreateVReplicationWorkflowRequest.workflow_sub_type:type_name -> binlogdata.VReplicationWorkflowSubType - 170, // 47: tabletmanagerdata.CreateVReplicationWorkflowResponse.result:type_name -> query.QueryResult - 150, // 48: tabletmanagerdata.DeleteTableDataRequest.table_filters:type_name -> tabletmanagerdata.DeleteTableDataRequest.TableFiltersEntry - 170, // 49: tabletmanagerdata.DeleteVReplicationWorkflowResponse.result:type_name -> query.QueryResult - 184, // 50: tabletmanagerdata.ReadVReplicationWorkflowsRequest.include_states:type_name -> binlogdata.VReplicationWorkflowState - 184, // 51: tabletmanagerdata.ReadVReplicationWorkflowsRequest.exclude_states:type_name -> binlogdata.VReplicationWorkflowState + 183, // 45: tabletmanagerdata.CreateVReplicationWorkflowRequest.workflow_type:type_name -> binlogdata.VReplicationWorkflowType + 184, // 46: tabletmanagerdata.CreateVReplicationWorkflowRequest.workflow_sub_type:type_name -> binlogdata.VReplicationWorkflowSubType + 171, // 47: tabletmanagerdata.CreateVReplicationWorkflowResponse.result:type_name -> query.QueryResult + 151, // 48: tabletmanagerdata.DeleteTableDataRequest.table_filters:type_name -> tabletmanagerdata.DeleteTableDataRequest.TableFiltersEntry + 171, // 49: tabletmanagerdata.DeleteVReplicationWorkflowResponse.result:type_name -> query.QueryResult + 185, // 50: tabletmanagerdata.ReadVReplicationWorkflowsRequest.include_states:type_name -> binlogdata.VReplicationWorkflowState + 185, // 51: tabletmanagerdata.ReadVReplicationWorkflowsRequest.exclude_states:type_name -> binlogdata.VReplicationWorkflowState 125, // 52: tabletmanagerdata.ReadVReplicationWorkflowsResponse.workflows:type_name -> tabletmanagerdata.ReadVReplicationWorkflowResponse - 168, // 53: tabletmanagerdata.ReadVReplicationWorkflowResponse.tablet_types:type_name -> topodata.TabletType + 169, // 53: tabletmanagerdata.ReadVReplicationWorkflowResponse.tablet_types:type_name -> topodata.TabletType 0, // 54: tabletmanagerdata.ReadVReplicationWorkflowResponse.tablet_selection_preference:type_name -> tabletmanagerdata.TabletSelectionPreference - 182, // 55: tabletmanagerdata.ReadVReplicationWorkflowResponse.workflow_type:type_name -> binlogdata.VReplicationWorkflowType - 183, // 56: tabletmanagerdata.ReadVReplicationWorkflowResponse.workflow_sub_type:type_name -> binlogdata.VReplicationWorkflowSubType - 151, // 57: tabletmanagerdata.ReadVReplicationWorkflowResponse.streams:type_name -> tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream - 152, // 58: tabletmanagerdata.ReadVReplicationWorkflowResponse.config_overrides:type_name -> tabletmanagerdata.ReadVReplicationWorkflowResponse.ConfigOverridesEntry + 183, // 55: tabletmanagerdata.ReadVReplicationWorkflowResponse.workflow_type:type_name -> binlogdata.VReplicationWorkflowType + 184, // 56: tabletmanagerdata.ReadVReplicationWorkflowResponse.workflow_sub_type:type_name -> binlogdata.VReplicationWorkflowSubType + 152, // 57: tabletmanagerdata.ReadVReplicationWorkflowResponse.streams:type_name -> tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream + 153, // 58: tabletmanagerdata.ReadVReplicationWorkflowResponse.config_overrides:type_name -> tabletmanagerdata.ReadVReplicationWorkflowResponse.ConfigOverridesEntry 133, // 59: tabletmanagerdata.VDiffRequest.options:type_name -> tabletmanagerdata.VDiffOptions - 170, // 60: tabletmanagerdata.VDiffResponse.output:type_name -> query.QueryResult + 171, // 60: tabletmanagerdata.VDiffResponse.output:type_name -> query.QueryResult 130, // 61: tabletmanagerdata.VDiffOptions.picker_options:type_name -> tabletmanagerdata.VDiffPickerOptions 132, // 62: tabletmanagerdata.VDiffOptions.core_options:type_name -> tabletmanagerdata.VDiffCoreOptions 131, // 63: tabletmanagerdata.VDiffOptions.report_options:type_name -> tabletmanagerdata.VDiffReportOptions - 168, // 64: tabletmanagerdata.UpdateVReplicationWorkflowRequest.tablet_types:type_name -> topodata.TabletType - 0, // 65: tabletmanagerdata.UpdateVReplicationWorkflowRequest.tablet_selection_preference:type_name -> tabletmanagerdata.TabletSelectionPreference - 185, // 66: tabletmanagerdata.UpdateVReplicationWorkflowRequest.on_ddl:type_name -> binlogdata.OnDDLAction - 184, // 67: tabletmanagerdata.UpdateVReplicationWorkflowRequest.state:type_name -> binlogdata.VReplicationWorkflowState - 153, // 68: tabletmanagerdata.UpdateVReplicationWorkflowRequest.config_overrides:type_name -> tabletmanagerdata.UpdateVReplicationWorkflowRequest.ConfigOverridesEntry - 170, // 69: tabletmanagerdata.UpdateVReplicationWorkflowResponse.result:type_name -> query.QueryResult - 184, // 70: tabletmanagerdata.UpdateVReplicationWorkflowsRequest.state:type_name -> binlogdata.VReplicationWorkflowState - 170, // 71: tabletmanagerdata.UpdateVReplicationWorkflowsResponse.result:type_name -> query.QueryResult - 155, // 72: tabletmanagerdata.CheckThrottlerResponse.metrics:type_name -> tabletmanagerdata.CheckThrottlerResponse.MetricsEntry - 1, // 73: tabletmanagerdata.CheckThrottlerResponse.response_code:type_name -> tabletmanagerdata.CheckThrottlerResponseCode - 157, // 74: tabletmanagerdata.GetThrottlerStatusResponse.aggregated_metrics:type_name -> tabletmanagerdata.GetThrottlerStatusResponse.AggregatedMetricsEntry - 158, // 75: tabletmanagerdata.GetThrottlerStatusResponse.metric_thresholds:type_name -> tabletmanagerdata.GetThrottlerStatusResponse.MetricThresholdsEntry - 160, // 76: tabletmanagerdata.GetThrottlerStatusResponse.metrics_health:type_name -> tabletmanagerdata.GetThrottlerStatusResponse.MetricsHealthEntry - 161, // 77: tabletmanagerdata.GetThrottlerStatusResponse.throttled_apps:type_name -> tabletmanagerdata.GetThrottlerStatusResponse.ThrottledAppsEntry - 162, // 78: tabletmanagerdata.GetThrottlerStatusResponse.app_checked_metrics:type_name -> tabletmanagerdata.GetThrottlerStatusResponse.AppCheckedMetricsEntry - 164, // 79: tabletmanagerdata.GetThrottlerStatusResponse.recent_apps:type_name -> tabletmanagerdata.GetThrottlerStatusResponse.RecentAppsEntry - 165, // 80: tabletmanagerdata.ChangeTagsRequest.tags:type_name -> tabletmanagerdata.ChangeTagsRequest.TagsEntry - 166, // 81: tabletmanagerdata.ChangeTagsResponse.tags:type_name -> tabletmanagerdata.ChangeTagsResponse.TagsEntry - 181, // 82: tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.bls:type_name -> binlogdata.BinlogSource - 180, // 83: tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.time_updated:type_name -> vttime.Time - 180, // 84: tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.transaction_timestamp:type_name -> vttime.Time - 184, // 85: tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.state:type_name -> binlogdata.VReplicationWorkflowState - 180, // 86: tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.time_heartbeat:type_name -> vttime.Time - 180, // 87: tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.time_throttled:type_name -> vttime.Time - 1, // 88: tabletmanagerdata.CheckThrottlerResponse.Metric.response_code:type_name -> tabletmanagerdata.CheckThrottlerResponseCode - 154, // 89: tabletmanagerdata.CheckThrottlerResponse.MetricsEntry.value:type_name -> tabletmanagerdata.CheckThrottlerResponse.Metric - 156, // 90: tabletmanagerdata.GetThrottlerStatusResponse.AggregatedMetricsEntry.value:type_name -> tabletmanagerdata.GetThrottlerStatusResponse.MetricResult - 180, // 91: tabletmanagerdata.GetThrottlerStatusResponse.MetricHealth.last_healthy_at:type_name -> vttime.Time - 159, // 92: tabletmanagerdata.GetThrottlerStatusResponse.MetricsHealthEntry.value:type_name -> tabletmanagerdata.GetThrottlerStatusResponse.MetricHealth - 186, // 93: tabletmanagerdata.GetThrottlerStatusResponse.ThrottledAppsEntry.value:type_name -> topodata.ThrottledAppRule - 180, // 94: tabletmanagerdata.GetThrottlerStatusResponse.RecentApp.checked_at:type_name -> vttime.Time - 1, // 95: tabletmanagerdata.GetThrottlerStatusResponse.RecentApp.response_code:type_name -> tabletmanagerdata.CheckThrottlerResponseCode - 163, // 96: tabletmanagerdata.GetThrottlerStatusResponse.RecentAppsEntry.value:type_name -> tabletmanagerdata.GetThrottlerStatusResponse.RecentApp - 97, // [97:97] is the sub-list for method output_type - 97, // [97:97] is the sub-list for method input_type - 97, // [97:97] is the sub-list for extension type_name - 97, // [97:97] is the sub-list for extension extendee - 0, // [0:97] is the sub-list for field type_name + 171, // 64: tabletmanagerdata.VDiffTableLastPK.source:type_name -> query.QueryResult + 171, // 65: tabletmanagerdata.VDiffTableLastPK.target:type_name -> query.QueryResult + 169, // 66: tabletmanagerdata.UpdateVReplicationWorkflowRequest.tablet_types:type_name -> topodata.TabletType + 0, // 67: tabletmanagerdata.UpdateVReplicationWorkflowRequest.tablet_selection_preference:type_name -> tabletmanagerdata.TabletSelectionPreference + 186, // 68: tabletmanagerdata.UpdateVReplicationWorkflowRequest.on_ddl:type_name -> binlogdata.OnDDLAction + 185, // 69: tabletmanagerdata.UpdateVReplicationWorkflowRequest.state:type_name -> binlogdata.VReplicationWorkflowState + 154, // 70: tabletmanagerdata.UpdateVReplicationWorkflowRequest.config_overrides:type_name -> tabletmanagerdata.UpdateVReplicationWorkflowRequest.ConfigOverridesEntry + 171, // 71: tabletmanagerdata.UpdateVReplicationWorkflowResponse.result:type_name -> query.QueryResult + 185, // 72: tabletmanagerdata.UpdateVReplicationWorkflowsRequest.state:type_name -> binlogdata.VReplicationWorkflowState + 171, // 73: tabletmanagerdata.UpdateVReplicationWorkflowsResponse.result:type_name -> query.QueryResult + 156, // 74: tabletmanagerdata.CheckThrottlerResponse.metrics:type_name -> tabletmanagerdata.CheckThrottlerResponse.MetricsEntry + 1, // 75: tabletmanagerdata.CheckThrottlerResponse.response_code:type_name -> tabletmanagerdata.CheckThrottlerResponseCode + 158, // 76: tabletmanagerdata.GetThrottlerStatusResponse.aggregated_metrics:type_name -> tabletmanagerdata.GetThrottlerStatusResponse.AggregatedMetricsEntry + 159, // 77: tabletmanagerdata.GetThrottlerStatusResponse.metric_thresholds:type_name -> tabletmanagerdata.GetThrottlerStatusResponse.MetricThresholdsEntry + 161, // 78: tabletmanagerdata.GetThrottlerStatusResponse.metrics_health:type_name -> tabletmanagerdata.GetThrottlerStatusResponse.MetricsHealthEntry + 162, // 79: tabletmanagerdata.GetThrottlerStatusResponse.throttled_apps:type_name -> tabletmanagerdata.GetThrottlerStatusResponse.ThrottledAppsEntry + 163, // 80: tabletmanagerdata.GetThrottlerStatusResponse.app_checked_metrics:type_name -> tabletmanagerdata.GetThrottlerStatusResponse.AppCheckedMetricsEntry + 165, // 81: tabletmanagerdata.GetThrottlerStatusResponse.recent_apps:type_name -> tabletmanagerdata.GetThrottlerStatusResponse.RecentAppsEntry + 166, // 82: tabletmanagerdata.ChangeTagsRequest.tags:type_name -> tabletmanagerdata.ChangeTagsRequest.TagsEntry + 167, // 83: tabletmanagerdata.ChangeTagsResponse.tags:type_name -> tabletmanagerdata.ChangeTagsResponse.TagsEntry + 182, // 84: tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.bls:type_name -> binlogdata.BinlogSource + 181, // 85: tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.time_updated:type_name -> vttime.Time + 181, // 86: tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.transaction_timestamp:type_name -> vttime.Time + 185, // 87: tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.state:type_name -> binlogdata.VReplicationWorkflowState + 181, // 88: tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.time_heartbeat:type_name -> vttime.Time + 181, // 89: tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.time_throttled:type_name -> vttime.Time + 1, // 90: tabletmanagerdata.CheckThrottlerResponse.Metric.response_code:type_name -> tabletmanagerdata.CheckThrottlerResponseCode + 155, // 91: tabletmanagerdata.CheckThrottlerResponse.MetricsEntry.value:type_name -> tabletmanagerdata.CheckThrottlerResponse.Metric + 157, // 92: tabletmanagerdata.GetThrottlerStatusResponse.AggregatedMetricsEntry.value:type_name -> tabletmanagerdata.GetThrottlerStatusResponse.MetricResult + 181, // 93: tabletmanagerdata.GetThrottlerStatusResponse.MetricHealth.last_healthy_at:type_name -> vttime.Time + 160, // 94: tabletmanagerdata.GetThrottlerStatusResponse.MetricsHealthEntry.value:type_name -> tabletmanagerdata.GetThrottlerStatusResponse.MetricHealth + 187, // 95: tabletmanagerdata.GetThrottlerStatusResponse.ThrottledAppsEntry.value:type_name -> topodata.ThrottledAppRule + 181, // 96: tabletmanagerdata.GetThrottlerStatusResponse.RecentApp.checked_at:type_name -> vttime.Time + 1, // 97: tabletmanagerdata.GetThrottlerStatusResponse.RecentApp.response_code:type_name -> tabletmanagerdata.CheckThrottlerResponseCode + 164, // 98: tabletmanagerdata.GetThrottlerStatusResponse.RecentAppsEntry.value:type_name -> tabletmanagerdata.GetThrottlerStatusResponse.RecentApp + 99, // [99:99] is the sub-list for method output_type + 99, // [99:99] is the sub-list for method input_type + 99, // [99:99] is the sub-list for extension type_name + 99, // [99:99] is the sub-list for extension extendee + 0, // [0:99] is the sub-list for field type_name } func init() { file_tabletmanagerdata_proto_init() } @@ -9589,15 +9651,15 @@ func file_tabletmanagerdata_proto_init() { } file_tabletmanagerdata_proto_msgTypes[108].OneofWrappers = []any{} file_tabletmanagerdata_proto_msgTypes[130].OneofWrappers = []any{} - file_tabletmanagerdata_proto_msgTypes[132].OneofWrappers = []any{} - file_tabletmanagerdata_proto_msgTypes[134].OneofWrappers = []any{} + file_tabletmanagerdata_proto_msgTypes[133].OneofWrappers = []any{} + file_tabletmanagerdata_proto_msgTypes[135].OneofWrappers = []any{} type x struct{} out := protoimpl.TypeBuilder{ File: protoimpl.DescBuilder{ GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_tabletmanagerdata_proto_rawDesc, NumEnums: 2, - NumMessages: 165, + NumMessages: 166, NumExtensions: 0, NumServices: 0, }, diff --git a/go/vt/proto/tabletmanagerdata/tabletmanagerdata_vtproto.pb.go b/go/vt/proto/tabletmanagerdata/tabletmanagerdata_vtproto.pb.go index 9f6b0df851b..9e41c7ed338 100644 --- a/go/vt/proto/tabletmanagerdata/tabletmanagerdata_vtproto.pb.go +++ b/go/vt/proto/tabletmanagerdata/tabletmanagerdata_vtproto.pb.go @@ -2579,6 +2579,24 @@ func (m *VDiffOptions) CloneMessageVT() proto.Message { return m.CloneVT() } +func (m *VDiffTableLastPK) CloneVT() *VDiffTableLastPK { + if m == nil { + return (*VDiffTableLastPK)(nil) + } + r := new(VDiffTableLastPK) + r.Source = m.Source.CloneVT() + r.Target = m.Target.CloneVT() + if len(m.unknownFields) > 0 { + r.unknownFields = make([]byte, len(m.unknownFields)) + copy(r.unknownFields, m.unknownFields) + } + return r +} + +func (m *VDiffTableLastPK) CloneMessageVT() proto.Message { + return m.CloneVT() +} + func (m *UpdateVReplicationWorkflowRequest) CloneVT() *UpdateVReplicationWorkflowRequest { if m == nil { return (*UpdateVReplicationWorkflowRequest)(nil) @@ -9350,6 +9368,59 @@ func (m *VDiffOptions) MarshalToSizedBufferVT(dAtA []byte) (int, error) { return len(dAtA) - i, nil } +func (m *VDiffTableLastPK) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *VDiffTableLastPK) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *VDiffTableLastPK) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Target != nil { + size, err := m.Target.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Source != nil { + size, err := m.Source.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + func (m *UpdateVReplicationWorkflowRequest) MarshalVT() (dAtA []byte, err error) { if m == nil { return nil, nil @@ -12772,6 +12843,24 @@ func (m *VDiffOptions) SizeVT() (n int) { return n } +func (m *VDiffTableLastPK) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Source != nil { + l = m.Source.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Target != nil { + l = m.Target.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + func (m *UpdateVReplicationWorkflowRequest) SizeVT() (n int) { if m == nil { return 0 @@ -27261,6 +27350,129 @@ func (m *VDiffOptions) UnmarshalVT(dAtA []byte) error { } return nil } +func (m *VDiffTableLastPK) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: VDiffTableLastPK: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: VDiffTableLastPK: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Source", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Source == nil { + m.Source = &query.QueryResult{} + } + if err := m.Source.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Target", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Target == nil { + m.Target = &query.QueryResult{} + } + if err := m.Target.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func (m *UpdateVReplicationWorkflowRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 diff --git a/go/vt/vttablet/tabletmanager/vdiff/table_differ.go b/go/vt/vttablet/tabletmanager/vdiff/table_differ.go index f509f8fe734..359f18a3b3f 100644 --- a/go/vt/vttablet/tabletmanager/vdiff/table_differ.go +++ b/go/vt/vttablet/tabletmanager/vdiff/table_differ.go @@ -196,7 +196,7 @@ func (td *tableDiffer) stopTargetVReplicationStreams(ctx context.Context, dbClie return fmt.Errorf("stream %d has not started on tablet %v", id, td.wd.ct.vde.thisTablet.Alias) } - sourceBytes, err := row[source].ToBytes() + sourceBytes, err := row["source"].ToBytes() if err != nil { return err } @@ -522,8 +522,8 @@ func (td *tableDiffer) diff(ctx context.Context, coreOpts *tabletmanagerdatapb.V } dr.TableName = td.table.Name - sourceExecutor := newPrimitiveExecutor(ctx, td.sourcePrimitive, source) - targetExecutor := newPrimitiveExecutor(ctx, td.targetPrimitive, target) + sourceExecutor := newPrimitiveExecutor(ctx, td.sourcePrimitive, "source") + targetExecutor := newPrimitiveExecutor(ctx, td.targetPrimitive, "target") var sourceRow, lastProcessedRow, targetRow []sqltypes.Value advanceSource := true advanceTarget := true @@ -736,42 +736,20 @@ func (td *tableDiffer) updateTableProgress(dbClient binlogplayer.DBClient, dr *D return err } } else { - var lastSourcePK, lastTargetPK []byte - lastPK := make(map[string]string, 2) - lastTargetPK, err = td.lastPKFromRow(lastRow, td.tablePlan.pkCols) - if err != nil { - return err - } - lastPK[target] = string(lastTargetPK) - if len(td.tablePlan.sourcePkCols) == 0 || slices.Equal(td.tablePlan.sourcePkCols, td.tablePlan.pkCols) { - lastPK[source] = string(lastTargetPK) - } else { - lastSourcePK, err = td.lastPKFromRow(lastRow, td.tablePlan.sourcePkCols) - if err != nil { - return err - } - lastPK[source] = string(lastSourcePK) - } + lastPK := td.lastPKFromRow(lastRow) if td.wd.opts.CoreOptions.MaxDiffSeconds > 0 { // Update the in-memory lastPK as well so that we can restart the table // diff if needed. - lastSourcePKPB, lastTargetPKPB := &querypb.QueryResult{}, &querypb.QueryResult{} - if err := prototext.Unmarshal(lastSourcePK, lastSourcePKPB); err != nil { - return err - } - td.lastSourcePK = lastSourcePKPB - if err := prototext.Unmarshal(lastTargetPK, lastTargetPKPB); err != nil { - return err - } - td.lastTargetPK = lastTargetPKPB + td.lastSourcePK, td.lastTargetPK = lastPK.Source, lastPK.Target } - lastPKJS, err := json.Marshal(lastPK) + lastPKTxt, err := prototext.Marshal(lastPK) if err != nil { return err } + log.Errorf("DEBUG: lastpk for table %s to DB: %s", td.table.Name, string(lastPKTxt)) query, err = sqlparser.ParseAndBind(sqlUpdateTableProgress, sqltypes.Int64BindVariable(dr.ProcessedRows), - sqltypes.StringBindVariable(string(lastPKJS)), + sqltypes.StringBindVariable(string(lastPKTxt)), sqltypes.StringBindVariable(string(rpt)), sqltypes.Int64BindVariable(td.wd.ct.id), sqltypes.StringBindVariable(td.table.Name), @@ -847,19 +825,31 @@ func updateTableMismatch(dbClient binlogplayer.DBClient, vdiffID int64, table st return nil } -func (td *tableDiffer) lastPKFromRow(row []sqltypes.Value, pkCols []int) ([]byte, error) { - pkColCnt := len(pkCols) - pkFields := make([]*querypb.Field, pkColCnt) - pkVals := make([]sqltypes.Value, pkColCnt) - for i, colIndex := range pkCols { - pkFields[i] = td.tablePlan.table.Fields[colIndex] - pkVals[i] = row[colIndex] - } - buf, err := prototext.Marshal(&querypb.QueryResult{ - Fields: pkFields, - Rows: []*querypb.Row{sqltypes.RowToProto3(pkVals)}, - }) - return buf, err +func (td *tableDiffer) lastPKFromRow(row []sqltypes.Value) *tabletmanagerdatapb.VDiffTableLastPK { + var source, target *querypb.QueryResult + buildQR := func(pkCols []int) *querypb.QueryResult { + pkColCnt := len(pkCols) + pkFields := make([]*querypb.Field, pkColCnt) + pkVals := make([]sqltypes.Value, pkColCnt) + for i, colIndex := range td.tablePlan.pkCols { + pkFields[i] = td.tablePlan.table.Fields[colIndex] + pkVals[i] = row[colIndex] + } + return &querypb.QueryResult{ + Fields: pkFields, + Rows: []*querypb.Row{sqltypes.RowToProto3(pkVals)}, + } + } + target = buildQR(td.tablePlan.pkCols) + if len(td.tablePlan.sourcePkCols) == 0 || slices.Equal(td.tablePlan.sourcePkCols, td.tablePlan.pkCols) { + source = target + } else { + source = buildQR(td.tablePlan.sourcePkCols) + } + return &tabletmanagerdatapb.VDiffTableLastPK{ + Source: source, + Target: target, + } } // If SourceTimeZone is defined in the BinlogSource (_vt.vreplication.source), the diff --git a/go/vt/vttablet/tabletmanager/vdiff/workflow_differ.go b/go/vt/vttablet/tabletmanager/vdiff/workflow_differ.go index 34fd3cfad19..fef0bf7a1d3 100644 --- a/go/vt/vttablet/tabletmanager/vdiff/workflow_differ.go +++ b/go/vt/vttablet/tabletmanager/vdiff/workflow_differ.go @@ -18,7 +18,6 @@ package vdiff import ( "context" - "encoding/json" "errors" "fmt" "reflect" @@ -43,16 +42,10 @@ import ( "vitess.io/vitess/go/vt/vttablet/tabletmanager/vreplication" binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" - querypb "vitess.io/vitess/go/vt/proto/query" tabletmanagerdatapb "vitess.io/vitess/go/vt/proto/tabletmanagerdata" vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc" ) -const ( - source = "source" - target = "target" -) - // workflowDiffer has metadata and state for the vdiff of a single workflow on this tablet // only one vdiff can be running for a workflow at any time. type workflowDiffer struct { @@ -383,8 +376,10 @@ func (wd *workflowDiffer) buildPlan(dbClient binlogplayer.DBClient, filter *binl if err != nil { return err } - td.lastSourcePK = lastPK[source] - td.lastTargetPK = lastPK[target] + if lastPK != nil { + td.lastSourcePK = lastPK.Source + td.lastTargetPK = lastPK.Target + } wd.tableDiffers[table.Name] = td if _, err := td.buildTablePlan(dbClient, wd.ct.vde.dbName, wd.collationEnv); err != nil { return err @@ -404,7 +399,7 @@ func (wd *workflowDiffer) buildPlan(dbClient binlogplayer.DBClient, filter *binl } // getTableLastPK gets the lastPK protobuf message for a given vdiff table. -func (wd *workflowDiffer) getTableLastPK(dbClient binlogplayer.DBClient, tableName string) (map[string]*querypb.QueryResult, error) { +func (wd *workflowDiffer) getTableLastPK(dbClient binlogplayer.DBClient, tableName string) (*tabletmanagerdatapb.VDiffTableLastPK, error) { query, err := sqlparser.ParseAndBind(sqlGetVDiffTable, sqltypes.Int64BindVariable(wd.ct.id), sqltypes.StringBindVariable(tableName), @@ -421,19 +416,13 @@ func (wd *workflowDiffer) getTableLastPK(dbClient binlogplayer.DBClient, tableNa if lastpk, err = qr.Named().Row().ToBytes("lastpk"); err != nil { return nil, err } + log.Errorf("DEBUG: lastpk for table %s from DB: %s", tableName, string(lastpk)) if len(lastpk) != 0 { - lastPK := make(map[string]string, 2) - lastPKResults := make(map[string]*querypb.QueryResult, 2) - if err := json.Unmarshal(lastpk, &lastPK); err != nil { - return nil, vterrors.Wrapf(err, "failed to unmarshal lastpk JSON for table %s", tableName) - } - for k, v := range lastPK { - lastPKResults[k] = &querypb.QueryResult{} - if err := prototext.Unmarshal([]byte(v), lastPKResults[k]); err != nil { - return nil, vterrors.Wrapf(err, "failed to unmarshal lastpk QueryResult for table %s", tableName) - } + lastPK := &tabletmanagerdatapb.VDiffTableLastPK{} + if err := prototext.Unmarshal(lastpk, lastPK); err != nil { + return nil, vterrors.Wrapf(err, "failed to unmarshal lastpk for table %s", tableName) } - return lastPKResults, nil + return lastPK, nil } } return nil, nil diff --git a/proto/tabletmanagerdata.proto b/proto/tabletmanagerdata.proto index bb20e712e7f..374a51798ec 100644 --- a/proto/tabletmanagerdata.proto +++ b/proto/tabletmanagerdata.proto @@ -760,6 +760,11 @@ message VDiffOptions { VDiffReportOptions report_options = 3; } +message VDiffTableLastPK { + query.QueryResult source = 1; + query.QueryResult target = 2; +} + // UpdateVReplicationWorkflowRequest is used to update an existing VReplication // workflow. Note that the following fields MUST have an explicit value provided diff --git a/web/vtadmin/src/proto/vtadmin.d.ts b/web/vtadmin/src/proto/vtadmin.d.ts index adb02034c92..bf36ddc7c42 100644 --- a/web/vtadmin/src/proto/vtadmin.d.ts +++ b/web/vtadmin/src/proto/vtadmin.d.ts @@ -34138,6 +34138,109 @@ export namespace tabletmanagerdata { public static getTypeUrl(typeUrlPrefix?: string): string; } + /** Properties of a VDiffTableLastPK. */ + interface IVDiffTableLastPK { + + /** VDiffTableLastPK source */ + source?: (query.IQueryResult|null); + + /** VDiffTableLastPK target */ + target?: (query.IQueryResult|null); + } + + /** Represents a VDiffTableLastPK. */ + class VDiffTableLastPK implements IVDiffTableLastPK { + + /** + * Constructs a new VDiffTableLastPK. + * @param [properties] Properties to set + */ + constructor(properties?: tabletmanagerdata.IVDiffTableLastPK); + + /** VDiffTableLastPK source. */ + public source?: (query.IQueryResult|null); + + /** VDiffTableLastPK target. */ + public target?: (query.IQueryResult|null); + + /** + * Creates a new VDiffTableLastPK instance using the specified properties. + * @param [properties] Properties to set + * @returns VDiffTableLastPK instance + */ + public static create(properties?: tabletmanagerdata.IVDiffTableLastPK): tabletmanagerdata.VDiffTableLastPK; + + /** + * Encodes the specified VDiffTableLastPK message. Does not implicitly {@link tabletmanagerdata.VDiffTableLastPK.verify|verify} messages. + * @param message VDiffTableLastPK message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encode(message: tabletmanagerdata.IVDiffTableLastPK, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Encodes the specified VDiffTableLastPK message, length delimited. Does not implicitly {@link tabletmanagerdata.VDiffTableLastPK.verify|verify} messages. + * @param message VDiffTableLastPK message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encodeDelimited(message: tabletmanagerdata.IVDiffTableLastPK, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Decodes a VDiffTableLastPK message from the specified reader or buffer. + * @param reader Reader or buffer to decode from + * @param [length] Message length if known beforehand + * @returns VDiffTableLastPK + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): tabletmanagerdata.VDiffTableLastPK; + + /** + * Decodes a VDiffTableLastPK message from the specified reader or buffer, length delimited. + * @param reader Reader or buffer to decode from + * @returns VDiffTableLastPK + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): tabletmanagerdata.VDiffTableLastPK; + + /** + * Verifies a VDiffTableLastPK message. + * @param message Plain object to verify + * @returns `null` if valid, otherwise the reason why it is not + */ + public static verify(message: { [k: string]: any }): (string|null); + + /** + * Creates a VDiffTableLastPK message from a plain object. Also converts values to their respective internal types. + * @param object Plain object + * @returns VDiffTableLastPK + */ + public static fromObject(object: { [k: string]: any }): tabletmanagerdata.VDiffTableLastPK; + + /** + * Creates a plain object from a VDiffTableLastPK message. Also converts values to other types if specified. + * @param message VDiffTableLastPK + * @param [options] Conversion options + * @returns Plain object + */ + public static toObject(message: tabletmanagerdata.VDiffTableLastPK, options?: $protobuf.IConversionOptions): { [k: string]: any }; + + /** + * Converts this VDiffTableLastPK to JSON. + * @returns JSON object + */ + public toJSON(): { [k: string]: any }; + + /** + * Gets the default type url for VDiffTableLastPK + * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns The default type url + */ + public static getTypeUrl(typeUrlPrefix?: string): string; + } + /** Properties of an UpdateVReplicationWorkflowRequest. */ interface IUpdateVReplicationWorkflowRequest { diff --git a/web/vtadmin/src/proto/vtadmin.js b/web/vtadmin/src/proto/vtadmin.js index d71d3e08fe5..d223576d6fd 100644 --- a/web/vtadmin/src/proto/vtadmin.js +++ b/web/vtadmin/src/proto/vtadmin.js @@ -78899,6 +78899,243 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { return VDiffOptions; })(); + tabletmanagerdata.VDiffTableLastPK = (function() { + + /** + * Properties of a VDiffTableLastPK. + * @memberof tabletmanagerdata + * @interface IVDiffTableLastPK + * @property {query.IQueryResult|null} [source] VDiffTableLastPK source + * @property {query.IQueryResult|null} [target] VDiffTableLastPK target + */ + + /** + * Constructs a new VDiffTableLastPK. + * @memberof tabletmanagerdata + * @classdesc Represents a VDiffTableLastPK. + * @implements IVDiffTableLastPK + * @constructor + * @param {tabletmanagerdata.IVDiffTableLastPK=} [properties] Properties to set + */ + function VDiffTableLastPK(properties) { + if (properties) + for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) + if (properties[keys[i]] != null) + this[keys[i]] = properties[keys[i]]; + } + + /** + * VDiffTableLastPK source. + * @member {query.IQueryResult|null|undefined} source + * @memberof tabletmanagerdata.VDiffTableLastPK + * @instance + */ + VDiffTableLastPK.prototype.source = null; + + /** + * VDiffTableLastPK target. + * @member {query.IQueryResult|null|undefined} target + * @memberof tabletmanagerdata.VDiffTableLastPK + * @instance + */ + VDiffTableLastPK.prototype.target = null; + + /** + * Creates a new VDiffTableLastPK instance using the specified properties. + * @function create + * @memberof tabletmanagerdata.VDiffTableLastPK + * @static + * @param {tabletmanagerdata.IVDiffTableLastPK=} [properties] Properties to set + * @returns {tabletmanagerdata.VDiffTableLastPK} VDiffTableLastPK instance + */ + VDiffTableLastPK.create = function create(properties) { + return new VDiffTableLastPK(properties); + }; + + /** + * Encodes the specified VDiffTableLastPK message. Does not implicitly {@link tabletmanagerdata.VDiffTableLastPK.verify|verify} messages. + * @function encode + * @memberof tabletmanagerdata.VDiffTableLastPK + * @static + * @param {tabletmanagerdata.IVDiffTableLastPK} message VDiffTableLastPK message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + VDiffTableLastPK.encode = function encode(message, writer) { + if (!writer) + writer = $Writer.create(); + if (message.source != null && Object.hasOwnProperty.call(message, "source")) + $root.query.QueryResult.encode(message.source, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.target != null && Object.hasOwnProperty.call(message, "target")) + $root.query.QueryResult.encode(message.target, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); + return writer; + }; + + /** + * Encodes the specified VDiffTableLastPK message, length delimited. Does not implicitly {@link tabletmanagerdata.VDiffTableLastPK.verify|verify} messages. + * @function encodeDelimited + * @memberof tabletmanagerdata.VDiffTableLastPK + * @static + * @param {tabletmanagerdata.IVDiffTableLastPK} message VDiffTableLastPK message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + VDiffTableLastPK.encodeDelimited = function encodeDelimited(message, writer) { + return this.encode(message, writer).ldelim(); + }; + + /** + * Decodes a VDiffTableLastPK message from the specified reader or buffer. + * @function decode + * @memberof tabletmanagerdata.VDiffTableLastPK + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @param {number} [length] Message length if known beforehand + * @returns {tabletmanagerdata.VDiffTableLastPK} VDiffTableLastPK + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + VDiffTableLastPK.decode = function decode(reader, length) { + if (!(reader instanceof $Reader)) + reader = $Reader.create(reader); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.tabletmanagerdata.VDiffTableLastPK(); + while (reader.pos < end) { + let tag = reader.uint32(); + switch (tag >>> 3) { + case 1: { + message.source = $root.query.QueryResult.decode(reader, reader.uint32()); + break; + } + case 2: { + message.target = $root.query.QueryResult.decode(reader, reader.uint32()); + break; + } + default: + reader.skipType(tag & 7); + break; + } + } + return message; + }; + + /** + * Decodes a VDiffTableLastPK message from the specified reader or buffer, length delimited. + * @function decodeDelimited + * @memberof tabletmanagerdata.VDiffTableLastPK + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @returns {tabletmanagerdata.VDiffTableLastPK} VDiffTableLastPK + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + VDiffTableLastPK.decodeDelimited = function decodeDelimited(reader) { + if (!(reader instanceof $Reader)) + reader = new $Reader(reader); + return this.decode(reader, reader.uint32()); + }; + + /** + * Verifies a VDiffTableLastPK message. + * @function verify + * @memberof tabletmanagerdata.VDiffTableLastPK + * @static + * @param {Object.} message Plain object to verify + * @returns {string|null} `null` if valid, otherwise the reason why it is not + */ + VDiffTableLastPK.verify = function verify(message) { + if (typeof message !== "object" || message === null) + return "object expected"; + if (message.source != null && message.hasOwnProperty("source")) { + let error = $root.query.QueryResult.verify(message.source); + if (error) + return "source." + error; + } + if (message.target != null && message.hasOwnProperty("target")) { + let error = $root.query.QueryResult.verify(message.target); + if (error) + return "target." + error; + } + return null; + }; + + /** + * Creates a VDiffTableLastPK message from a plain object. Also converts values to their respective internal types. + * @function fromObject + * @memberof tabletmanagerdata.VDiffTableLastPK + * @static + * @param {Object.} object Plain object + * @returns {tabletmanagerdata.VDiffTableLastPK} VDiffTableLastPK + */ + VDiffTableLastPK.fromObject = function fromObject(object) { + if (object instanceof $root.tabletmanagerdata.VDiffTableLastPK) + return object; + let message = new $root.tabletmanagerdata.VDiffTableLastPK(); + if (object.source != null) { + if (typeof object.source !== "object") + throw TypeError(".tabletmanagerdata.VDiffTableLastPK.source: object expected"); + message.source = $root.query.QueryResult.fromObject(object.source); + } + if (object.target != null) { + if (typeof object.target !== "object") + throw TypeError(".tabletmanagerdata.VDiffTableLastPK.target: object expected"); + message.target = $root.query.QueryResult.fromObject(object.target); + } + return message; + }; + + /** + * Creates a plain object from a VDiffTableLastPK message. Also converts values to other types if specified. + * @function toObject + * @memberof tabletmanagerdata.VDiffTableLastPK + * @static + * @param {tabletmanagerdata.VDiffTableLastPK} message VDiffTableLastPK + * @param {$protobuf.IConversionOptions} [options] Conversion options + * @returns {Object.} Plain object + */ + VDiffTableLastPK.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.defaults) { + object.source = null; + object.target = null; + } + if (message.source != null && message.hasOwnProperty("source")) + object.source = $root.query.QueryResult.toObject(message.source, options); + if (message.target != null && message.hasOwnProperty("target")) + object.target = $root.query.QueryResult.toObject(message.target, options); + return object; + }; + + /** + * Converts this VDiffTableLastPK to JSON. + * @function toJSON + * @memberof tabletmanagerdata.VDiffTableLastPK + * @instance + * @returns {Object.} JSON object + */ + VDiffTableLastPK.prototype.toJSON = function toJSON() { + return this.constructor.toObject(this, $protobuf.util.toJSONOptions); + }; + + /** + * Gets the default type url for VDiffTableLastPK + * @function getTypeUrl + * @memberof tabletmanagerdata.VDiffTableLastPK + * @static + * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns {string} The default type url + */ + VDiffTableLastPK.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + if (typeUrlPrefix === undefined) { + typeUrlPrefix = "type.googleapis.com"; + } + return typeUrlPrefix + "/tabletmanagerdata.VDiffTableLastPK"; + }; + + return VDiffTableLastPK; + })(); + tabletmanagerdata.UpdateVReplicationWorkflowRequest = (function() { /** From 02b7f06c3839f286b73654ae5923056b92a98328 Mon Sep 17 00:00:00 2001 From: Matt Lord Date: Thu, 9 Jan 2025 10:40:10 -0500 Subject: [PATCH 5/7] Add PKE handling for source as well Signed-off-by: Matt Lord --- .../tabletmanager/vdiff/table_differ.go | 35 +++++++++++++------ .../tabletmanager/vdiff/workflow_differ.go | 1 - 2 files changed, 24 insertions(+), 12 deletions(-) diff --git a/go/vt/vttablet/tabletmanager/vdiff/table_differ.go b/go/vt/vttablet/tabletmanager/vdiff/table_differ.go index 359f18a3b3f..a98af2d54fa 100644 --- a/go/vt/vttablet/tabletmanager/vdiff/table_differ.go +++ b/go/vt/vttablet/tabletmanager/vdiff/table_differ.go @@ -20,7 +20,6 @@ import ( "context" "encoding/json" "fmt" - "slices" "strings" "sync" "time" @@ -36,6 +35,7 @@ import ( "vitess.io/vitess/go/vt/concurrency" "vitess.io/vitess/go/vt/discovery" "vitess.io/vitess/go/vt/log" + "vitess.io/vitess/go/vt/mysqlctl" "vitess.io/vitess/go/vt/sqlparser" "vitess.io/vitess/go/vt/topo" "vitess.io/vitess/go/vt/topo/topoproto" @@ -746,7 +746,6 @@ func (td *tableDiffer) updateTableProgress(dbClient binlogplayer.DBClient, dr *D if err != nil { return err } - log.Errorf("DEBUG: lastpk for table %s to DB: %s", td.table.Name, string(lastPKTxt)) query, err = sqlparser.ParseAndBind(sqlUpdateTableProgress, sqltypes.Int64BindVariable(dr.ProcessedRows), sqltypes.StringBindVariable(string(lastPKTxt)), @@ -826,7 +825,6 @@ func updateTableMismatch(dbClient binlogplayer.DBClient, vdiffID int64, table st } func (td *tableDiffer) lastPKFromRow(row []sqltypes.Value) *tabletmanagerdatapb.VDiffTableLastPK { - var source, target *querypb.QueryResult buildQR := func(pkCols []int) *querypb.QueryResult { pkColCnt := len(pkCols) pkFields := make([]*querypb.Field, pkColCnt) @@ -840,15 +838,9 @@ func (td *tableDiffer) lastPKFromRow(row []sqltypes.Value) *tabletmanagerdatapb. Rows: []*querypb.Row{sqltypes.RowToProto3(pkVals)}, } } - target = buildQR(td.tablePlan.pkCols) - if len(td.tablePlan.sourcePkCols) == 0 || slices.Equal(td.tablePlan.sourcePkCols, td.tablePlan.pkCols) { - source = target - } else { - source = buildQR(td.tablePlan.sourcePkCols) - } return &tabletmanagerdatapb.VDiffTableLastPK{ - Source: source, - Target: target, + Source: buildQR(td.tablePlan.sourcePkCols), + Target: buildQR(td.tablePlan.pkCols), } } @@ -905,6 +897,9 @@ func (td *tableDiffer) getSourcePKCols() error { ctx, cancel := context.WithTimeout(td.wd.ct.vde.ctx, topo.RemoteOperationTimeout*3) defer cancel() // We use the first sourceShard as all of them should have the same schema. + if len(td.wd.ct.sources) == 0 { + return fmt.Errorf("no source shards found") + } sourceShardName := maps.Keys(td.wd.ct.sources)[0] sourceTS, err := td.wd.getSourceTopoServer() if err != nil { @@ -928,6 +923,24 @@ func (td *tableDiffer) getSourcePKCols() error { return err } sourceTable := sourceSchema.TableDefinitions[0] + if len(sourceTable.PrimaryKeyColumns) == 0 { + // We use the columns from a PKE if there is one. + executeFetch := func(query string, maxrows int, wantfields bool) (*sqltypes.Result, error) { + res, err := td.wd.ct.tmc.ExecuteFetchAsApp(ctx, sourceTablet.Tablet, false, &tabletmanagerdatapb.ExecuteFetchAsAppRequest{ + Query: []byte(query), + MaxRows: 1, + }) + if err != nil { + return nil, err + } + return sqltypes.Proto3ToResult(res), nil + } + pkeCols, _, err := mysqlctl.GetPrimaryKeyEquivalentColumns(ctx, executeFetch, td.wd.ct.sourceKeyspace, td.table.Name) + if err != nil { + return err + } + sourceTable.PrimaryKeyColumns = pkeCols + } sourcePKColumns := make(map[string]struct{}, len(sourceTable.PrimaryKeyColumns)) td.tablePlan.sourcePkCols = make([]int, 0, len(sourceTable.PrimaryKeyColumns)) for _, pkc := range sourceTable.PrimaryKeyColumns { diff --git a/go/vt/vttablet/tabletmanager/vdiff/workflow_differ.go b/go/vt/vttablet/tabletmanager/vdiff/workflow_differ.go index fef0bf7a1d3..3315607dc9e 100644 --- a/go/vt/vttablet/tabletmanager/vdiff/workflow_differ.go +++ b/go/vt/vttablet/tabletmanager/vdiff/workflow_differ.go @@ -416,7 +416,6 @@ func (wd *workflowDiffer) getTableLastPK(dbClient binlogplayer.DBClient, tableNa if lastpk, err = qr.Named().Row().ToBytes("lastpk"); err != nil { return nil, err } - log.Errorf("DEBUG: lastpk for table %s from DB: %s", tableName, string(lastpk)) if len(lastpk) != 0 { lastPK := &tabletmanagerdatapb.VDiffTableLastPK{} if err := prototext.Unmarshal(lastpk, lastPK); err != nil { From f7bd4b840d114b1657c229daa00938fbfdfc518f Mon Sep 17 00:00:00 2001 From: Matt Lord Date: Thu, 9 Jan 2025 12:42:26 -0500 Subject: [PATCH 6/7] Fix source PKE handling Signed-off-by: Matt Lord --- go/vt/vttablet/tabletmanager/vdiff/table_differ.go | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/go/vt/vttablet/tabletmanager/vdiff/table_differ.go b/go/vt/vttablet/tabletmanager/vdiff/table_differ.go index a98af2d54fa..486a6d63f14 100644 --- a/go/vt/vttablet/tabletmanager/vdiff/table_differ.go +++ b/go/vt/vttablet/tabletmanager/vdiff/table_differ.go @@ -829,7 +829,7 @@ func (td *tableDiffer) lastPKFromRow(row []sqltypes.Value) *tabletmanagerdatapb. pkColCnt := len(pkCols) pkFields := make([]*querypb.Field, pkColCnt) pkVals := make([]sqltypes.Value, pkColCnt) - for i, colIndex := range td.tablePlan.pkCols { + for i, colIndex := range pkCols { pkFields[i] = td.tablePlan.table.Fields[colIndex] pkVals[i] = row[colIndex] } @@ -935,11 +935,16 @@ func (td *tableDiffer) getSourcePKCols() error { } return sqltypes.Proto3ToResult(res), nil } - pkeCols, _, err := mysqlctl.GetPrimaryKeyEquivalentColumns(ctx, executeFetch, td.wd.ct.sourceKeyspace, td.table.Name) + pkeCols, _, err := mysqlctl.GetPrimaryKeyEquivalentColumns(ctx, executeFetch, sourceTablet.DbName(), td.table.Name) if err != nil { return err } - sourceTable.PrimaryKeyColumns = pkeCols + if len(pkeCols) > 0 { + sourceTable.PrimaryKeyColumns = pkeCols + } else { + // We use every column together as a substitute PK. + sourceTable.PrimaryKeyColumns = append(sourceTable.PrimaryKeyColumns, td.table.Columns...) + } } sourcePKColumns := make(map[string]struct{}, len(sourceTable.PrimaryKeyColumns)) td.tablePlan.sourcePkCols = make([]int, 0, len(sourceTable.PrimaryKeyColumns)) From afa6ffb846ef94bddba669216fca6c0dc99ef5ab Mon Sep 17 00:00:00 2001 From: Matt Lord Date: Thu, 9 Jan 2025 16:24:18 -0500 Subject: [PATCH 7/7] Fix unit tests Signed-off-by: Matt Lord --- .../tabletmanager/vdiff/engine_test.go | 8 +- .../tabletmanager/vdiff/framework_test.go | 10 + .../vdiff/workflow_differ_test.go | 367 ++++++++++-------- 3 files changed, 213 insertions(+), 172 deletions(-) diff --git a/go/vt/vttablet/tabletmanager/vdiff/engine_test.go b/go/vt/vttablet/tabletmanager/vdiff/engine_test.go index 61d713a8e58..8849bcfea93 100644 --- a/go/vt/vttablet/tabletmanager/vdiff/engine_test.go +++ b/go/vt/vttablet/tabletmanager/vdiff/engine_test.go @@ -139,7 +139,7 @@ func TestVDiff(t *testing.T) { "lastpk|mismatch|report", "varbinary|int64|json", ), - `fields:{name:"c1" type:INT64 table:"t1" org_table:"t1" database:"vt_customer" org_name:"c1" column_length:20 charset:63 flags:53251} rows:{lengths:1 values:"1"}|0|{}`, + `source:{fields:{name:"c1" type:INT64 table:"t1" org_table:"t1" database:"vt_customer" org_name:"c1" column_length:20 charset:63 flags:53251} rows:{lengths:1 values:"1"}} target:{fields:{name:"c1" type:INT64 table:"t1" org_table:"t1" database:"vt_customer" org_name:"c1" column_length:20 charset:63 flags:53251} rows:{lengths:1 values:"1"}}|0|{}`, ), nil) vdenv.dbClient.ExpectRequest(fmt.Sprintf("select column_name as column_name, collation_name as collation_name from information_schema.columns where table_schema='%s' and table_name='t1' and column_name in ('c1')", vdiffDBName), sqltypes.MakeTestResult(sqltypes.MakeTestFields( "collation_name", @@ -159,7 +159,7 @@ func TestVDiff(t *testing.T) { "lastpk|mismatch|report", "varbinary|int64|json", ), - `fields:{name:"c1" type:INT64 table:"t1" org_table:"t1" database:"vt_customer" org_name:"c1" column_length:20 charset:63 flags:53251} rows:{lengths:1 values:"1"}|0|{"TableName": "t1", "MatchingRows": 1, "ProcessedRows": 1, "MismatchedRows": 0, "ExtraRowsSource": 0, "ExtraRowsTarget": 0}`, + `source:{fields:{name:"c1" type:INT64 table:"t1" org_table:"t1" database:"vt_customer" org_name:"c1" column_length:20 charset:63 flags:53251} rows:{lengths:1 values:"1"}} target:{fields:{name:"c1" type:INT64 table:"t1" org_table:"t1" database:"vt_customer" org_name:"c1" column_length:20 charset:63 flags:53251} rows:{lengths:1 values:"1"}}|0|{}`, ), nil) vdenv.dbClient.ExpectRequest("update _vt.vdiff_table set table_rows = 1 where vdiff_id = 1 and table_name = 't1'", singleRowAffected, nil) @@ -169,7 +169,7 @@ func TestVDiff(t *testing.T) { "lastpk|mismatch|report", "varbinary|int64|json", ), - `fields:{name:"c1" type:INT64 table:"t1" org_table:"t1" database:"vt_customer" org_name:"c1" column_length:20 charset:63 flags:53251} rows:{lengths:1 values:"1"}|0|{"TableName": "t1", "MatchingRows": 1, "ProcessedRows": 1, "MismatchedRows": 0, "ExtraRowsSource": 0, "ExtraRowsTarget": 0}`, + `source:{fields:{name:"c1" type:INT64 table:"t1" org_table:"t1" database:"vt_customer" org_name:"c1" column_length:20 charset:63 flags:53251} rows:{lengths:1 values:"1"}} target:{fields:{name:"c1" type:INT64 table:"t1" org_table:"t1" database:"vt_customer" org_name:"c1" column_length:20 charset:63 flags:53251} rows:{lengths:1 values:"1"}}|0|{}`, ), nil) vdenv.dbClient.ExpectRequest("update _vt.vdiff_table set state = 'started' where vdiff_id = 1 and table_name = 't1'", singleRowAffected, nil) vdenv.dbClient.ExpectRequest(`insert into _vt.vdiff_log(vdiff_id, message) values (1, 'started: table \'t1\'')`, singleRowAffected, nil) @@ -185,7 +185,7 @@ func TestVDiff(t *testing.T) { "lastpk|mismatch|report", "varbinary|int64|json", ), - `fields:{name:"c1" type:INT64 table:"t1" org_table:"t1" database:"vt_customer" org_name:"c1" column_length:20 charset:63 flags:53251} rows:{lengths:1 values:"1"}|0|{}`, + `source:{fields:{name:"c1" type:INT64 table:"t1" org_table:"t1" database:"vt_customer" org_name:"c1" column_length:20 charset:63 flags:53251} rows:{lengths:1 values:"1"}} target:{fields:{name:"c1" type:INT64 table:"t1" org_table:"t1" database:"vt_customer" org_name:"c1" column_length:20 charset:63 flags:53251} rows:{lengths:1 values:"1"}}|0|{}`, ), nil) vdenv.dbClient.ExpectRequest(`update _vt.vdiff_table set rows_compared = 0, report = '{"TableName":"t1","ProcessedRows":0,"MatchingRows":0,"MismatchedRows":0,"ExtraRowsSource":0,"ExtraRowsTarget":0}' where vdiff_id = 1 and table_name = 't1'`, singleRowAffected, nil) vdenv.dbClient.ExpectRequest(`update _vt.vdiff_table set state = 'completed', rows_compared = 0, report = '{"TableName":"t1","ProcessedRows":0,"MatchingRows":0,"MismatchedRows":0,"ExtraRowsSource":0,"ExtraRowsTarget":0}' where vdiff_id = 1 and table_name = 't1'`, singleRowAffected, nil) diff --git a/go/vt/vttablet/tabletmanager/vdiff/framework_test.go b/go/vt/vttablet/tabletmanager/vdiff/framework_test.go index 7d4cdb78c20..563741fcd23 100644 --- a/go/vt/vttablet/tabletmanager/vdiff/framework_test.go +++ b/go/vt/vttablet/tabletmanager/vdiff/framework_test.go @@ -678,5 +678,15 @@ func (tvde *testVDiffEnv) createController(t *testing.T, id int) *controller { tvde.dbClient.ExpectRequest(fmt.Sprintf("select * from _vt.vdiff where id = %d", id), noResults, nil) ct, err := newController(context.Background(), controllerQR.Named().Row(), tvde.dbClientFactory, tstenv.TopoServ, tvde.vde, tvde.opts) require.NoError(t, err) + ct.sources = map[string]*migrationSource{ + tstenv.ShardName: { + vrID: 1, + shardStreamer: &shardStreamer{ + tablet: tvde.vde.thisTablet, + shard: tstenv.ShardName, + }, + }, + } + ct.sourceKeyspace = tstenv.KeyspaceName return ct } diff --git a/go/vt/vttablet/tabletmanager/vdiff/workflow_differ_test.go b/go/vt/vttablet/tabletmanager/vdiff/workflow_differ_test.go index c51b6fdf281..547341e192e 100644 --- a/go/vt/vttablet/tabletmanager/vdiff/workflow_differ_test.go +++ b/go/vt/vttablet/tabletmanager/vdiff/workflow_differ_test.go @@ -51,6 +51,16 @@ func TestBuildPlanSuccess(t *testing.T) { vdiffenv.dbClient.ExpectRequest("select * from _vt.vdiff where id = 1", noResults, nil) ct, err := newController(context.Background(), controllerQR.Named().Row(), vdiffenv.dbClientFactory, tstenv.TopoServ, vdiffenv.vde, vdiffenv.opts) require.NoError(t, err) + ct.sources = map[string]*migrationSource{ + tstenv.ShardName: { + vrID: 1, + shardStreamer: &shardStreamer{ + tablet: vdenv.vde.thisTablet, + shard: tstenv.ShardName, + }, + }, + } + ct.sourceKeyspace = tstenv.KeyspaceName testcases := []struct { input *binlogdatapb.Rule @@ -63,14 +73,15 @@ func TestBuildPlanSuccess(t *testing.T) { }, table: "t1", tablePlan: &tablePlan{ - dbName: vdiffDBName, - table: testSchema.TableDefinitions[tableDefMap["t1"]], - sourceQuery: "select c1, c2 from t1 order by c1 asc", - targetQuery: "select c1, c2 from t1 order by c1 asc", - compareCols: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c1"}, {1, collations.MySQL8().LookupByName(sqltypes.NULL.String()), false, "c2"}}, - comparePKs: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c1"}}, - pkCols: []int{0}, - selectPks: []int{0}, + dbName: vdiffDBName, + table: testSchema.TableDefinitions[tableDefMap["t1"]], + sourceQuery: "select c1, c2 from t1 order by c1 asc", + targetQuery: "select c1, c2 from t1 order by c1 asc", + compareCols: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c1"}, {1, collations.MySQL8().LookupByName(sqltypes.NULL.String()), false, "c2"}}, + comparePKs: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c1"}}, + pkCols: []int{0}, + sourcePkCols: []int{0}, + selectPks: []int{0}, orderBy: sqlparser.OrderBy{&sqlparser.Order{ Expr: &sqlparser.ColName{Name: sqlparser.NewIdentifierCI("c1")}, Direction: sqlparser.AscOrder, @@ -83,14 +94,15 @@ func TestBuildPlanSuccess(t *testing.T) { }, table: "t1", tablePlan: &tablePlan{ - dbName: vdiffDBName, - table: testSchema.TableDefinitions[tableDefMap["t1"]], - sourceQuery: "select c1, c2 from t1 where in_keyrange('-80') order by c1 asc", - targetQuery: "select c1, c2 from t1 order by c1 asc", - compareCols: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c1"}, {1, collations.MySQL8().LookupByName(sqltypes.NULL.String()), false, "c2"}}, - comparePKs: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c1"}}, - pkCols: []int{0}, - selectPks: []int{0}, + dbName: vdiffDBName, + table: testSchema.TableDefinitions[tableDefMap["t1"]], + sourceQuery: "select c1, c2 from t1 where in_keyrange('-80') order by c1 asc", + targetQuery: "select c1, c2 from t1 order by c1 asc", + compareCols: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c1"}, {1, collations.MySQL8().LookupByName(sqltypes.NULL.String()), false, "c2"}}, + comparePKs: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c1"}}, + pkCols: []int{0}, + sourcePkCols: []int{0}, + selectPks: []int{0}, orderBy: sqlparser.OrderBy{&sqlparser.Order{ Expr: &sqlparser.ColName{Name: sqlparser.NewIdentifierCI("c1")}, Direction: sqlparser.AscOrder, @@ -103,14 +115,15 @@ func TestBuildPlanSuccess(t *testing.T) { }, table: "t1", tablePlan: &tablePlan{ - dbName: vdiffDBName, - table: testSchema.TableDefinitions[tableDefMap["t1"]], - sourceQuery: "select c1, c2 from t1 order by c1 asc", - targetQuery: "select c1, c2 from t1 order by c1 asc", - compareCols: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c1"}, {1, collations.MySQL8().LookupByName(sqltypes.NULL.String()), false, "c2"}}, - comparePKs: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c1"}}, - pkCols: []int{0}, - selectPks: []int{0}, + dbName: vdiffDBName, + table: testSchema.TableDefinitions[tableDefMap["t1"]], + sourceQuery: "select c1, c2 from t1 order by c1 asc", + targetQuery: "select c1, c2 from t1 order by c1 asc", + compareCols: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c1"}, {1, collations.MySQL8().LookupByName(sqltypes.NULL.String()), false, "c2"}}, + comparePKs: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c1"}}, + pkCols: []int{0}, + sourcePkCols: []int{0}, + selectPks: []int{0}, orderBy: sqlparser.OrderBy{&sqlparser.Order{ Expr: &sqlparser.ColName{Name: sqlparser.NewIdentifierCI("c1")}, Direction: sqlparser.AscOrder, @@ -123,14 +136,15 @@ func TestBuildPlanSuccess(t *testing.T) { }, table: "t1", tablePlan: &tablePlan{ - dbName: vdiffDBName, - table: testSchema.TableDefinitions[tableDefMap["t1"]], - sourceQuery: "select c2, c1 from t1 order by c1 asc", - targetQuery: "select c2, c1 from t1 order by c1 asc", - compareCols: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), false, "c2"}, {1, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c1"}}, - comparePKs: []compareColInfo{{1, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c1"}}, - pkCols: []int{1}, - selectPks: []int{1}, + dbName: vdiffDBName, + table: testSchema.TableDefinitions[tableDefMap["t1"]], + sourceQuery: "select c2, c1 from t1 order by c1 asc", + targetQuery: "select c2, c1 from t1 order by c1 asc", + compareCols: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), false, "c2"}, {1, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c1"}}, + comparePKs: []compareColInfo{{1, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c1"}}, + pkCols: []int{1}, + sourcePkCols: []int{0}, + selectPks: []int{1}, orderBy: sqlparser.OrderBy{&sqlparser.Order{ Expr: &sqlparser.ColName{Name: sqlparser.NewIdentifierCI("c1")}, Direction: sqlparser.AscOrder, @@ -143,14 +157,15 @@ func TestBuildPlanSuccess(t *testing.T) { }, table: "t1", tablePlan: &tablePlan{ - dbName: vdiffDBName, - table: testSchema.TableDefinitions[tableDefMap["t1"]], - sourceQuery: "select c0 as c1, c2 from t2 order by c1 asc", - targetQuery: "select c1, c2 from t1 order by c1 asc", - compareCols: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c1"}, {1, collations.MySQL8().LookupByName(sqltypes.NULL.String()), false, "c2"}}, - comparePKs: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c1"}}, - pkCols: []int{0}, - selectPks: []int{0}, + dbName: vdiffDBName, + table: testSchema.TableDefinitions[tableDefMap["t1"]], + sourceQuery: "select c0 as c1, c2 from t2 order by c1 asc", + targetQuery: "select c1, c2 from t1 order by c1 asc", + compareCols: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c1"}, {1, collations.MySQL8().LookupByName(sqltypes.NULL.String()), false, "c2"}}, + comparePKs: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c1"}}, + pkCols: []int{0}, + sourcePkCols: []int{0}, + selectPks: []int{0}, orderBy: sqlparser.OrderBy{&sqlparser.Order{ Expr: &sqlparser.ColName{Name: sqlparser.NewIdentifierCI("c1")}, Direction: sqlparser.AscOrder, @@ -164,14 +179,15 @@ func TestBuildPlanSuccess(t *testing.T) { }, table: "nonpktext", tablePlan: &tablePlan{ - dbName: vdiffDBName, - table: testSchema.TableDefinitions[tableDefMap["nonpktext"]], - sourceQuery: "select c1, textcol from nonpktext order by c1 asc", - targetQuery: "select c1, textcol from nonpktext order by c1 asc", - compareCols: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c1"}, {1, collations.MySQL8().LookupByName(sqltypes.NULL.String()), false, "textcol"}}, - comparePKs: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c1"}}, - pkCols: []int{0}, - selectPks: []int{0}, + dbName: vdiffDBName, + table: testSchema.TableDefinitions[tableDefMap["nonpktext"]], + sourceQuery: "select c1, textcol from nonpktext order by c1 asc", + targetQuery: "select c1, textcol from nonpktext order by c1 asc", + compareCols: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c1"}, {1, collations.MySQL8().LookupByName(sqltypes.NULL.String()), false, "textcol"}}, + comparePKs: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c1"}}, + pkCols: []int{0}, + sourcePkCols: []int{0}, + selectPks: []int{0}, orderBy: sqlparser.OrderBy{&sqlparser.Order{ Expr: &sqlparser.ColName{Name: sqlparser.NewIdentifierCI("c1")}, Direction: sqlparser.AscOrder, @@ -185,14 +201,15 @@ func TestBuildPlanSuccess(t *testing.T) { }, table: "nonpktext", tablePlan: &tablePlan{ - dbName: vdiffDBName, - table: testSchema.TableDefinitions[tableDefMap["nonpktext"]], - sourceQuery: "select textcol, c1 from nonpktext order by c1 asc", - targetQuery: "select textcol, c1 from nonpktext order by c1 asc", - compareCols: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), false, "textcol"}, {1, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c1"}}, - comparePKs: []compareColInfo{{1, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c1"}}, - pkCols: []int{1}, - selectPks: []int{1}, + dbName: vdiffDBName, + table: testSchema.TableDefinitions[tableDefMap["nonpktext"]], + sourceQuery: "select textcol, c1 from nonpktext order by c1 asc", + targetQuery: "select textcol, c1 from nonpktext order by c1 asc", + compareCols: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), false, "textcol"}, {1, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c1"}}, + comparePKs: []compareColInfo{{1, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c1"}}, + pkCols: []int{1}, + sourcePkCols: []int{0}, + selectPks: []int{1}, orderBy: sqlparser.OrderBy{&sqlparser.Order{ Expr: &sqlparser.ColName{Name: sqlparser.NewIdentifierCI("c1")}, Direction: sqlparser.AscOrder, @@ -206,14 +223,15 @@ func TestBuildPlanSuccess(t *testing.T) { }, table: "pktext", tablePlan: &tablePlan{ - dbName: vdiffDBName, - table: testSchema.TableDefinitions[tableDefMap["pktext"]], - sourceQuery: "select textcol, c2 from pktext order by textcol asc", - targetQuery: "select textcol, c2 from pktext order by textcol asc", - compareCols: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "textcol"}, {1, collations.MySQL8().LookupByName(sqltypes.NULL.String()), false, "c2"}}, - comparePKs: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "textcol"}}, - pkCols: []int{0}, - selectPks: []int{0}, + dbName: vdiffDBName, + table: testSchema.TableDefinitions[tableDefMap["pktext"]], + sourceQuery: "select textcol, c2 from pktext order by textcol asc", + targetQuery: "select textcol, c2 from pktext order by textcol asc", + compareCols: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "textcol"}, {1, collations.MySQL8().LookupByName(sqltypes.NULL.String()), false, "c2"}}, + comparePKs: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "textcol"}}, + pkCols: []int{0}, + sourcePkCols: []int{}, + selectPks: []int{0}, orderBy: sqlparser.OrderBy{&sqlparser.Order{ Expr: &sqlparser.ColName{Name: sqlparser.NewIdentifierCI("textcol")}, Direction: sqlparser.AscOrder, @@ -227,14 +245,15 @@ func TestBuildPlanSuccess(t *testing.T) { }, table: "pktext", tablePlan: &tablePlan{ - dbName: vdiffDBName, - table: testSchema.TableDefinitions[tableDefMap["pktext"]], - sourceQuery: "select c2, textcol from pktext order by textcol asc", - targetQuery: "select c2, textcol from pktext order by textcol asc", - compareCols: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), false, "c2"}, {1, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "textcol"}}, - comparePKs: []compareColInfo{{1, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "textcol"}}, - pkCols: []int{1}, - selectPks: []int{1}, + dbName: vdiffDBName, + table: testSchema.TableDefinitions[tableDefMap["pktext"]], + sourceQuery: "select c2, textcol from pktext order by textcol asc", + targetQuery: "select c2, textcol from pktext order by textcol asc", + compareCols: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), false, "c2"}, {1, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "textcol"}}, + comparePKs: []compareColInfo{{1, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "textcol"}}, + pkCols: []int{1}, + sourcePkCols: []int{}, + selectPks: []int{1}, orderBy: sqlparser.OrderBy{&sqlparser.Order{ Expr: &sqlparser.ColName{Name: sqlparser.NewIdentifierCI("textcol")}, Direction: sqlparser.AscOrder, @@ -248,14 +267,15 @@ func TestBuildPlanSuccess(t *testing.T) { }, table: "nopk", tablePlan: &tablePlan{ - dbName: vdiffDBName, - table: testSchema.TableDefinitions[tableDefMap["nopk"]], - sourceQuery: "select c1, c2, c3 from nopk order by c1 asc, c2 asc, c3 asc", - targetQuery: "select c1, c2, c3 from nopk order by c1 asc, c2 asc, c3 asc", - compareCols: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c1"}, {1, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c2"}, {2, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c3"}}, - comparePKs: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c1"}, {1, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c2"}, {2, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c3"}}, - pkCols: []int{0, 1, 2}, - selectPks: []int{0, 1, 2}, + dbName: vdiffDBName, + table: testSchema.TableDefinitions[tableDefMap["nopk"]], + sourceQuery: "select c1, c2, c3 from nopk order by c1 asc, c2 asc, c3 asc", + targetQuery: "select c1, c2, c3 from nopk order by c1 asc, c2 asc, c3 asc", + compareCols: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c1"}, {1, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c2"}, {2, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c3"}}, + comparePKs: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c1"}, {1, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c2"}, {2, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c3"}}, + pkCols: []int{0, 1, 2}, + sourcePkCols: []int{0}, + selectPks: []int{0, 1, 2}, orderBy: sqlparser.OrderBy{ &sqlparser.Order{ Expr: &sqlparser.ColName{Name: sqlparser.NewIdentifierCI("c1")}, @@ -279,14 +299,15 @@ func TestBuildPlanSuccess(t *testing.T) { }, table: "nopkwithpke", tablePlan: &tablePlan{ - dbName: vdiffDBName, - table: testSchema.TableDefinitions[tableDefMap["nopkwithpke"]], - sourceQuery: "select c1, c2, c3 from nopkwithpke order by c3 asc", - targetQuery: "select c1, c2, c3 from nopkwithpke order by c3 asc", - compareCols: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), false, "c1"}, {1, collations.MySQL8().LookupByName(sqltypes.NULL.String()), false, "c2"}, {2, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c3"}}, - comparePKs: []compareColInfo{{2, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c3"}}, - pkCols: []int{2}, - selectPks: []int{2}, + dbName: vdiffDBName, + table: testSchema.TableDefinitions[tableDefMap["nopkwithpke"]], + sourceQuery: "select c1, c2, c3 from nopkwithpke order by c3 asc", + targetQuery: "select c1, c2, c3 from nopkwithpke order by c3 asc", + compareCols: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), false, "c1"}, {1, collations.MySQL8().LookupByName(sqltypes.NULL.String()), false, "c2"}, {2, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c3"}}, + comparePKs: []compareColInfo{{2, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c3"}}, + pkCols: []int{2}, + sourcePkCols: []int{}, + selectPks: []int{2}, orderBy: sqlparser.OrderBy{ &sqlparser.Order{ Expr: &sqlparser.ColName{Name: sqlparser.NewIdentifierCI("c3")}, @@ -302,14 +323,15 @@ func TestBuildPlanSuccess(t *testing.T) { }, table: "pktext", tablePlan: &tablePlan{ - dbName: vdiffDBName, - table: testSchema.TableDefinitions[tableDefMap["pktext"]], - sourceQuery: "select c2, a + b as textcol from pktext order by textcol asc", - targetQuery: "select c2, textcol from pktext order by textcol asc", - compareCols: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), false, "c2"}, {1, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "textcol"}}, - comparePKs: []compareColInfo{{1, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "textcol"}}, - pkCols: []int{1}, - selectPks: []int{1}, + dbName: vdiffDBName, + table: testSchema.TableDefinitions[tableDefMap["pktext"]], + sourceQuery: "select c2, a + b as textcol from pktext order by textcol asc", + targetQuery: "select c2, textcol from pktext order by textcol asc", + compareCols: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), false, "c2"}, {1, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "textcol"}}, + comparePKs: []compareColInfo{{1, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "textcol"}}, + pkCols: []int{1}, + sourcePkCols: []int{}, + selectPks: []int{1}, orderBy: sqlparser.OrderBy{&sqlparser.Order{ Expr: &sqlparser.ColName{Name: sqlparser.NewIdentifierCI("textcol")}, Direction: sqlparser.AscOrder, @@ -322,14 +344,15 @@ func TestBuildPlanSuccess(t *testing.T) { }, table: "multipk", tablePlan: &tablePlan{ - dbName: vdiffDBName, - table: testSchema.TableDefinitions[tableDefMap["multipk"]], - sourceQuery: "select c1, c2 from multipk order by c1 asc, c2 asc", - targetQuery: "select c1, c2 from multipk order by c1 asc, c2 asc", - compareCols: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c1"}, {1, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c2"}}, - comparePKs: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c1"}, {1, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c2"}}, - pkCols: []int{0, 1}, - selectPks: []int{0, 1}, + dbName: vdiffDBName, + table: testSchema.TableDefinitions[tableDefMap["multipk"]], + sourceQuery: "select c1, c2 from multipk order by c1 asc, c2 asc", + targetQuery: "select c1, c2 from multipk order by c1 asc, c2 asc", + compareCols: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c1"}, {1, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c2"}}, + comparePKs: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c1"}, {1, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c2"}}, + pkCols: []int{0, 1}, + sourcePkCols: []int{0}, + selectPks: []int{0, 1}, orderBy: sqlparser.OrderBy{ &sqlparser.Order{ Expr: &sqlparser.ColName{Name: sqlparser.NewIdentifierCI("c1")}, @@ -349,14 +372,15 @@ func TestBuildPlanSuccess(t *testing.T) { }, table: "t1", tablePlan: &tablePlan{ - dbName: vdiffDBName, - table: testSchema.TableDefinitions[tableDefMap["t1"]], - sourceQuery: "select c1, c2 from t1 where in_keyrange('-80') order by c1 asc", - targetQuery: "select c1, c2 from t1 order by c1 asc", - compareCols: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c1"}, {1, collations.MySQL8().LookupByName(sqltypes.NULL.String()), false, "c2"}}, - comparePKs: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c1"}}, - pkCols: []int{0}, - selectPks: []int{0}, + dbName: vdiffDBName, + table: testSchema.TableDefinitions[tableDefMap["t1"]], + sourceQuery: "select c1, c2 from t1 where in_keyrange('-80') order by c1 asc", + targetQuery: "select c1, c2 from t1 order by c1 asc", + compareCols: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c1"}, {1, collations.MySQL8().LookupByName(sqltypes.NULL.String()), false, "c2"}}, + comparePKs: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c1"}}, + pkCols: []int{0}, + sourcePkCols: []int{0}, + selectPks: []int{0}, orderBy: sqlparser.OrderBy{&sqlparser.Order{ Expr: &sqlparser.ColName{Name: sqlparser.NewIdentifierCI("c1")}, Direction: sqlparser.AscOrder, @@ -370,14 +394,15 @@ func TestBuildPlanSuccess(t *testing.T) { }, table: "t1", tablePlan: &tablePlan{ - dbName: vdiffDBName, - table: testSchema.TableDefinitions[tableDefMap["t1"]], - sourceQuery: "select c1, c2 from t1 where c2 = 2 and in_keyrange('-80') order by c1 asc", - targetQuery: "select c1, c2 from t1 where c2 = 2 order by c1 asc", - compareCols: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c1"}, {1, collations.MySQL8().LookupByName(sqltypes.NULL.String()), false, "c2"}}, - comparePKs: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c1"}}, - pkCols: []int{0}, - selectPks: []int{0}, + dbName: vdiffDBName, + table: testSchema.TableDefinitions[tableDefMap["t1"]], + sourceQuery: "select c1, c2 from t1 where c2 = 2 and in_keyrange('-80') order by c1 asc", + targetQuery: "select c1, c2 from t1 where c2 = 2 order by c1 asc", + compareCols: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c1"}, {1, collations.MySQL8().LookupByName(sqltypes.NULL.String()), false, "c2"}}, + comparePKs: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c1"}}, + pkCols: []int{0}, + sourcePkCols: []int{0}, + selectPks: []int{0}, orderBy: sqlparser.OrderBy{&sqlparser.Order{ Expr: &sqlparser.ColName{Name: sqlparser.NewIdentifierCI("c1")}, Direction: sqlparser.AscOrder, @@ -391,14 +416,15 @@ func TestBuildPlanSuccess(t *testing.T) { }, table: "t1", tablePlan: &tablePlan{ - dbName: vdiffDBName, - table: testSchema.TableDefinitions[tableDefMap["t1"]], - sourceQuery: "select c1, c2 from t1 where in_keyrange('-80') and c2 = 2 order by c1 asc", - targetQuery: "select c1, c2 from t1 where c2 = 2 order by c1 asc", - compareCols: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c1"}, {1, collations.MySQL8().LookupByName(sqltypes.NULL.String()), false, "c2"}}, - comparePKs: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c1"}}, - pkCols: []int{0}, - selectPks: []int{0}, + dbName: vdiffDBName, + table: testSchema.TableDefinitions[tableDefMap["t1"]], + sourceQuery: "select c1, c2 from t1 where in_keyrange('-80') and c2 = 2 order by c1 asc", + targetQuery: "select c1, c2 from t1 where c2 = 2 order by c1 asc", + compareCols: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c1"}, {1, collations.MySQL8().LookupByName(sqltypes.NULL.String()), false, "c2"}}, + comparePKs: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c1"}}, + pkCols: []int{0}, + sourcePkCols: []int{0}, + selectPks: []int{0}, orderBy: sqlparser.OrderBy{&sqlparser.Order{ Expr: &sqlparser.ColName{Name: sqlparser.NewIdentifierCI("c1")}, Direction: sqlparser.AscOrder, @@ -412,14 +438,15 @@ func TestBuildPlanSuccess(t *testing.T) { }, table: "t1", tablePlan: &tablePlan{ - dbName: vdiffDBName, - table: testSchema.TableDefinitions[tableDefMap["t1"]], - sourceQuery: "select c1, c2 from t1 where c2 = 2 and c1 = 1 and in_keyrange('-80') order by c1 asc", - targetQuery: "select c1, c2 from t1 where c2 = 2 and c1 = 1 order by c1 asc", - compareCols: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c1"}, {1, collations.MySQL8().LookupByName(sqltypes.NULL.String()), false, "c2"}}, - comparePKs: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c1"}}, - pkCols: []int{0}, - selectPks: []int{0}, + dbName: vdiffDBName, + table: testSchema.TableDefinitions[tableDefMap["t1"]], + sourceQuery: "select c1, c2 from t1 where c2 = 2 and c1 = 1 and in_keyrange('-80') order by c1 asc", + targetQuery: "select c1, c2 from t1 where c2 = 2 and c1 = 1 order by c1 asc", + compareCols: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c1"}, {1, collations.MySQL8().LookupByName(sqltypes.NULL.String()), false, "c2"}}, + comparePKs: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c1"}}, + pkCols: []int{0}, + sourcePkCols: []int{0}, + selectPks: []int{0}, orderBy: sqlparser.OrderBy{&sqlparser.Order{ Expr: &sqlparser.ColName{Name: sqlparser.NewIdentifierCI("c1")}, Direction: sqlparser.AscOrder, @@ -433,14 +460,15 @@ func TestBuildPlanSuccess(t *testing.T) { }, table: "t1", tablePlan: &tablePlan{ - dbName: vdiffDBName, - table: testSchema.TableDefinitions[tableDefMap["t1"]], - sourceQuery: "select c1, c2 from t1 where c2 = 2 and in_keyrange('-80') order by c1 asc", - targetQuery: "select c1, c2 from t1 where c2 = 2 order by c1 asc", - compareCols: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c1"}, {1, collations.MySQL8().LookupByName(sqltypes.NULL.String()), false, "c2"}}, - comparePKs: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c1"}}, - pkCols: []int{0}, - selectPks: []int{0}, + dbName: vdiffDBName, + table: testSchema.TableDefinitions[tableDefMap["t1"]], + sourceQuery: "select c1, c2 from t1 where c2 = 2 and in_keyrange('-80') order by c1 asc", + targetQuery: "select c1, c2 from t1 where c2 = 2 order by c1 asc", + compareCols: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c1"}, {1, collations.MySQL8().LookupByName(sqltypes.NULL.String()), false, "c2"}}, + comparePKs: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c1"}}, + pkCols: []int{0}, + sourcePkCols: []int{0}, + selectPks: []int{0}, orderBy: sqlparser.OrderBy{&sqlparser.Order{ Expr: &sqlparser.ColName{Name: sqlparser.NewIdentifierCI("c1")}, Direction: sqlparser.AscOrder, @@ -454,14 +482,15 @@ func TestBuildPlanSuccess(t *testing.T) { }, table: "t1", tablePlan: &tablePlan{ - dbName: vdiffDBName, - table: testSchema.TableDefinitions[tableDefMap["t1"]], - sourceQuery: "select c1, c2 from t1 group by c1 order by c1 asc", - targetQuery: "select c1, c2 from t1 order by c1 asc", - compareCols: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c1"}, {1, collations.MySQL8().LookupByName(sqltypes.NULL.String()), false, "c2"}}, - comparePKs: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c1"}}, - pkCols: []int{0}, - selectPks: []int{0}, + dbName: vdiffDBName, + table: testSchema.TableDefinitions[tableDefMap["t1"]], + sourceQuery: "select c1, c2 from t1 group by c1 order by c1 asc", + targetQuery: "select c1, c2 from t1 order by c1 asc", + compareCols: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c1"}, {1, collations.MySQL8().LookupByName(sqltypes.NULL.String()), false, "c2"}}, + comparePKs: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c1"}}, + pkCols: []int{0}, + sourcePkCols: []int{0}, + selectPks: []int{0}, orderBy: sqlparser.OrderBy{&sqlparser.Order{ Expr: &sqlparser.ColName{Name: sqlparser.NewIdentifierCI("c1")}, Direction: sqlparser.AscOrder, @@ -475,14 +504,15 @@ func TestBuildPlanSuccess(t *testing.T) { }, table: "aggr", tablePlan: &tablePlan{ - dbName: vdiffDBName, - table: testSchema.TableDefinitions[tableDefMap["aggr"]], - sourceQuery: "select c1, c2, count(*) as c3, sum(c4) as c4 from t1 group by c1 order by c1 asc", - targetQuery: "select c1, c2, c3, c4 from aggr order by c1 asc", - compareCols: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c1"}, {1, collations.MySQL8().LookupByName(sqltypes.NULL.String()), false, "c2"}, {2, collations.MySQL8().LookupByName(sqltypes.NULL.String()), false, "c3"}, {3, collations.MySQL8().LookupByName(sqltypes.NULL.String()), false, "c4"}}, - comparePKs: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c1"}}, - pkCols: []int{0}, - selectPks: []int{0}, + dbName: vdiffDBName, + table: testSchema.TableDefinitions[tableDefMap["aggr"]], + sourceQuery: "select c1, c2, count(*) as c3, sum(c4) as c4 from t1 group by c1 order by c1 asc", + targetQuery: "select c1, c2, c3, c4 from aggr order by c1 asc", + compareCols: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c1"}, {1, collations.MySQL8().LookupByName(sqltypes.NULL.String()), false, "c2"}, {2, collations.MySQL8().LookupByName(sqltypes.NULL.String()), false, "c3"}, {3, collations.MySQL8().LookupByName(sqltypes.NULL.String()), false, "c4"}}, + comparePKs: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "c1"}}, + pkCols: []int{0}, + sourcePkCols: []int{0}, + selectPks: []int{0}, orderBy: sqlparser.OrderBy{&sqlparser.Order{ Expr: &sqlparser.ColName{Name: sqlparser.NewIdentifierCI("c1")}, Direction: sqlparser.AscOrder, @@ -500,14 +530,15 @@ func TestBuildPlanSuccess(t *testing.T) { sourceTimeZone: "US/Pacific", table: "datze", tablePlan: &tablePlan{ - dbName: vdiffDBName, - table: testSchema.TableDefinitions[tableDefMap["datze"]], - sourceQuery: "select id, dt from datze order by id asc", - targetQuery: "select id, convert_tz(dt, 'UTC', 'US/Pacific') as dt from datze order by id asc", - compareCols: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "id"}, {1, collations.MySQL8().LookupByName(sqltypes.NULL.String()), false, "dt"}}, - comparePKs: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "id"}}, - pkCols: []int{0}, - selectPks: []int{0}, + dbName: vdiffDBName, + table: testSchema.TableDefinitions[tableDefMap["datze"]], + sourceQuery: "select id, dt from datze order by id asc", + targetQuery: "select id, convert_tz(dt, 'UTC', 'US/Pacific') as dt from datze order by id asc", + compareCols: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "id"}, {1, collations.MySQL8().LookupByName(sqltypes.NULL.String()), false, "dt"}}, + comparePKs: []compareColInfo{{0, collations.MySQL8().LookupByName(sqltypes.NULL.String()), true, "id"}}, + pkCols: []int{0}, + sourcePkCols: []int{}, + selectPks: []int{0}, orderBy: sqlparser.OrderBy{&sqlparser.Order{ Expr: &sqlparser.ColName{Name: sqlparser.NewIdentifierCI("id")}, Direction: sqlparser.AscOrder,