From a99994c57dd78538d4be03bb9beee56f9047a157 Mon Sep 17 00:00:00 2001 From: "vitess-bot[bot]" <108069721+vitess-bot[bot]@users.noreply.github.com> Date: Wed, 3 Apr 2024 13:55:00 -0400 Subject: [PATCH 1/2] Cherry-pick 0e2f1751d470da9d17f6c1c7bfb381ba6676117d with conflicts --- .../command/vreplication/workflow/state.go | 1 - .../command/vreplication/workflow/update.go | 2 +- .../vreplication/multi_tenant_test.go | 464 ++++++++++++++++++ .../resharding_workflows_v2_test.go | 31 +- .../tabletmanagerdata/tabletmanagerdata.pb.go | 329 ++++++++++++- .../tabletmanagerdata_vtproto.pb.go | 52 -- go/vt/vtctl/vtctl.go | 1 + go/vt/vtctl/workflow/server.go | 2 - .../tabletmanager/rpc_vreplication.go | 30 +- proto/tabletmanagerdata.proto | 30 +- web/vtadmin/src/proto/vtadmin.d.ts | 6 - web/vtadmin/src/proto/vtadmin.js | 39 -- 12 files changed, 873 insertions(+), 114 deletions(-) create mode 100644 go/test/endtoend/vreplication/multi_tenant_test.go diff --git a/go/cmd/vtctldclient/command/vreplication/workflow/state.go b/go/cmd/vtctldclient/command/vreplication/workflow/state.go index a5555d78b0e..89e75312ab2 100644 --- a/go/cmd/vtctldclient/command/vreplication/workflow/state.go +++ b/go/cmd/vtctldclient/command/vreplication/workflow/state.go @@ -82,7 +82,6 @@ func commandUpdateState(cmd *cobra.Command, args []string) error { TabletTypes: []topodatapb.TabletType{topodatapb.TabletType(textutil.SimulatedNullInt)}, OnDdl: binlogdatapb.OnDDLAction(textutil.SimulatedNullInt), State: state, - Shards: baseOptions.Shards, }, } diff --git a/go/cmd/vtctldclient/command/vreplication/workflow/update.go b/go/cmd/vtctldclient/command/vreplication/workflow/update.go index 52df87acc8b..3d06ad3e64e 100644 --- a/go/cmd/vtctldclient/command/vreplication/workflow/update.go +++ b/go/cmd/vtctldclient/command/vreplication/workflow/update.go @@ -111,7 +111,7 @@ func commandUpdate(cmd *cobra.Command, args []string) error { TabletTypes: updateOptions.TabletTypes, TabletSelectionPreference: tsp, OnDdl: binlogdatapb.OnDDLAction(onddl), - Shards: baseOptions.Shards, + State: binlogdatapb.VReplicationWorkflowState(textutil.SimulatedNullInt), // We don't allow changing this in the client command }, } diff --git a/go/test/endtoend/vreplication/multi_tenant_test.go b/go/test/endtoend/vreplication/multi_tenant_test.go new file mode 100644 index 00000000000..3130e498f5c --- /dev/null +++ b/go/test/endtoend/vreplication/multi_tenant_test.go @@ -0,0 +1,464 @@ +/* +Copyright 2024 The Vitess Authors. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +/* +Consists of two tests. Both tests are for multi-tenant migration scenarios. + +1. TestMultiTenantSimple: migrates a single tenant to a target keyspace. + +2. TestMultiTenantComplex: migrates multiple tenants to a single target keyspace, with concurrent migrations. + +The tests use the MoveTables workflow to migrate the tenants. They are designed to simulate a real-world multi-tenant +migration scenario, where each tenant is in a separate database. +*/ + +package vreplication + +import ( + "encoding/json" + "fmt" + "math/rand/v2" + "strconv" + "sync" + "sync/atomic" + "testing" + "time" + + "github.com/stretchr/testify/require" + + "vitess.io/vitess/go/test/endtoend/cluster" + "vitess.io/vitess/go/vt/log" + binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" + vschemapb "vitess.io/vitess/go/vt/proto/vschema" +) + +type tenantMigrationStatus int + +const ( + tenantMigrationStatusNotMigrated tenantMigrationStatus = iota + tenantMigrationStatusMigrating + tenantMigrationStatusMigrated + + sourceKeyspaceTemplate = "s%d" + sourceAliasKeyspaceTemplate = "a%d" + targetKeyspaceName = "mt" + + numTenants = 10 + numInitialRowsPerTenant = 10 + numAdditionalRowsPerTenant = 10 + baseInitialTabletId = 1000 + tabletIdStep = 100 + maxRandomDelaySeconds = 5 + waitTimeout = 10 * time.Minute +) + +var ( + // channels to coordinate the migration workflow + chNotSetup, chNotCreated, chInProgress, chSwitched, chCompleted chan int64 + // counters to keep track of the number of tenants in each state + numSetup, numInProgress, numSwitched, numCompleted atomic.Int64 +) + +// multiTenantMigration manages the migration of multiple tenants to a single target keyspace. +// A singleton object of this type is created for the test case. +type multiTenantMigration struct { + t *testing.T + mu sync.Mutex + tenantMigrationStatus map[int64]tenantMigrationStatus // current migration status for each tenant + activeMoveTables map[int64]*VtctldMoveTables // the internal MoveTables object for each tenant + + targetKeyspace string + tables string + tenantIdColumnName string // the name of the column in each table that holds the tenant ID + + lastIDs map[int64]int64 // the last primary key inserted for each tenant +} + +const ( + mtSchema = "create table t1(id int, tenant_id int, primary key(id, tenant_id)) Engine=InnoDB" + mtVSchema = ` +{ + "multi_tenant_spec": { + "tenant_id_column_name": "tenant_id", + "tenant_id_column_type": "INT64" + }, + "tables": { + "t1": {} + } +} +` + stSchema = mtSchema + stVSchema = ` +{ + "tables": { + "t1": {} + } +} +` +) + +// TestMultiTenantSimple tests a single tenant migration. The aim here is to test all the steps of the migration process +// including keyspace routing rules, addition of tenant filters to the forward and reverse vreplication streams, and +// verifying that the data is migrated correctly. +func TestMultiTenantSimple(t *testing.T) { + setSidecarDBName("_vt") + // Don't create RDONLY tablets to reduce number of tablets created to reduce resource requirements for the test. + origDefaultRdonly := defaultRdonly + defer func() { + defaultRdonly = origDefaultRdonly + }() + defaultRdonly = 0 + vc = setupMinimalCluster(t) + defer vc.TearDown() + + targetKeyspace := "mt" + _, err := vc.AddKeyspace(t, []*Cell{vc.Cells["zone1"]}, targetKeyspace, "0", mtVSchema, mtSchema, 1, 0, 200, nil) + require.NoError(t, err) + + tenantId := int64(1) + sourceKeyspace := getSourceKeyspace(tenantId) + sourceAliasKeyspace := getSourceAliasKeyspace(tenantId) + _, err = vc.AddKeyspace(t, []*Cell{vc.Cells["zone1"]}, sourceKeyspace, "0", stVSchema, stSchema, 1, 0, getInitialTabletIdForTenant(tenantId), nil) + require.NoError(t, err) + + targetPrimary := vc.getPrimaryTablet(t, targetKeyspace, "0") + sourcePrimary := vc.getPrimaryTablet(t, sourceKeyspace, "0") + primaries := map[string]*cluster.VttabletProcess{ + "target": targetPrimary, + "source": sourcePrimary, + } + + vtgateConn, closeConn := getVTGateConn() + defer closeConn() + numRows := 10 + lastIndex := int64(0) + insertRows := func(lastIndex int64, keyspace string) int64 { + for i := 1; i <= numRows; i++ { + execQueryWithRetry(t, vtgateConn, + fmt.Sprintf("insert into %s.t1(id, tenant_id) values(%d, %d)", keyspace, int64(i)+lastIndex, tenantId), queryTimeout) + } + return int64(numRows) + lastIndex + } + lastIndex = insertRows(lastIndex, sourceKeyspace) + + mt := newVtctldMoveTables(&moveTablesWorkflow{ + workflowInfo: &workflowInfo{ + vc: vc, + workflowName: fmt.Sprintf("wf%d", tenantId), + targetKeyspace: targetKeyspace, + }, + sourceKeyspace: sourceKeyspace, + createFlags: []string{ + "--tenant-id", strconv.FormatInt(tenantId, 10), + "--source-keyspace-alias", sourceAliasKeyspace, + }, + }) + + preSwitchRules := &vschemapb.KeyspaceRoutingRules{ + Rules: []*vschemapb.KeyspaceRoutingRule{ + {FromKeyspace: "a1", ToKeyspace: "s1"}, + {FromKeyspace: "s1", ToKeyspace: "s1"}, + }, + } + postSwitchRules := &vschemapb.KeyspaceRoutingRules{ + Rules: []*vschemapb.KeyspaceRoutingRule{ + {FromKeyspace: "a1", ToKeyspace: "mt"}, + {FromKeyspace: "s1", ToKeyspace: "mt"}, + }, + } + rulesMap := map[string]*vschemapb.KeyspaceRoutingRules{ + "pre": preSwitchRules, + "post": postSwitchRules, + } + require.Zero(t, len(getKeyspaceRoutingRules(t, vc).Rules)) + mt.Create() + validateKeyspaceRoutingRules(t, vc, primaries, rulesMap, false) + // Note: we cannot insert into the target keyspace since that is never routed to the source keyspace. + for _, ks := range []string{sourceKeyspace, sourceAliasKeyspace} { + lastIndex = insertRows(lastIndex, ks) + } + waitForWorkflowState(t, vc, fmt.Sprintf("%s.%s", targetKeyspace, mt.workflowName), binlogdatapb.VReplicationWorkflowState_Running.String()) + mt.SwitchReadsAndWrites() + validateKeyspaceRoutingRules(t, vc, primaries, rulesMap, true) + // Note: here we have already switched and we can insert into the target keyspace and it should get reverse + // replicated to the source keyspace. The source and alias are also routed to the target keyspace at this point. + for _, ks := range []string{sourceKeyspace, sourceAliasKeyspace, targetKeyspace} { + lastIndex = insertRows(lastIndex, ks) + } + mt.Complete() + require.Zero(t, len(getKeyspaceRoutingRules(t, vc).Rules)) + actualRowsInserted := getRowCount(t, vtgateConn, fmt.Sprintf("%s.%s", targetKeyspace, "t1")) + log.Infof("Migration completed, total rows in target: %d", actualRowsInserted) + require.Equal(t, lastIndex, int64(actualRowsInserted)) +} + +// If switched queries with source/alias qualifiers should execute on target, else on source. Confirm that +// the routing rules are as expected and that the query executes on the expected tablet. +func validateKeyspaceRoutingRules(t *testing.T, vc *VitessCluster, primaries map[string]*cluster.VttabletProcess, rulesMap map[string]*vschemapb.KeyspaceRoutingRules, switched bool) { + currentRules := getKeyspaceRoutingRules(t, vc) + vtgateConn, closeConn := getVTGateConn() + defer closeConn() + queryTemplate := "select count(*) from %s.t1" + matchQuery := "select count(*) from t1" + + validateQueryRoute := func(qualifier, dest string) { + query := fmt.Sprintf(queryTemplate, qualifier) + assertQueryExecutesOnTablet(t, vtgateConn, primaries[dest], "", query, matchQuery) + log.Infof("query %s executed on %s", query, dest) + } + + if switched { + require.ElementsMatch(t, rulesMap["post"].Rules, currentRules.Rules) + validateQueryRoute("mt", "target") + validateQueryRoute("s1", "target") + validateQueryRoute("a1", "target") + } else { + require.ElementsMatch(t, rulesMap["pre"].Rules, currentRules.Rules) + // Note that with multi-tenant migration, we cannot redirect the target keyspace since + // there are multiple source keyspaces and the target has the aggregate of all the tenants. + validateQueryRoute("mt", "target") + validateQueryRoute("s1", "source") + validateQueryRoute("a1", "source") + } +} + +func getSourceKeyspace(tenantId int64) string { + return fmt.Sprintf(sourceKeyspaceTemplate, tenantId) +} + +func getSourceAliasKeyspace(tenantId int64) string { + return fmt.Sprintf(sourceAliasKeyspaceTemplate, tenantId) +} + +func (mtm *multiTenantMigration) insertSomeData(t *testing.T, tenantId int64, keyspace string, numRows int64) { + vtgateConn, closeConn := getVTGateConn() + defer closeConn() + idx := mtm.getLastID(tenantId) + for i := idx + 1; i <= idx+numRows; i++ { + execQueryWithRetry(t, vtgateConn, + fmt.Sprintf("insert into %s.t1(id, tenant_id) values(%d, %d)", keyspace, i, tenantId), queryTimeout) + } + mtm.setLastID(tenantId, idx+numRows) +} + +func getKeyspaceRoutingRules(t *testing.T, vc *VitessCluster) *vschemapb.KeyspaceRoutingRules { + output, err := vc.VtctldClient.ExecuteCommandWithOutput("GetKeyspaceRoutingRules") + require.NoError(t, err) + rules := &vschemapb.KeyspaceRoutingRules{} + err = json.Unmarshal([]byte(output), rules) + require.NoError(t, err) + return rules +} + +// TestMultiTenant tests a multi-tenant migration scenario where each tenant is in a separate database. +// It uses MoveTables to migrate all tenants to the same target keyspace. The test creates a separate source keyspace +// for each tenant. It then steps through the migration process for each tenant, and verifies that the data is migrated +// correctly. The migration steps are done concurrently and randomly to simulate an actual multi-tenant migration. +func TestMultiTenantComplex(t *testing.T) { + setSidecarDBName("_vt") + // Don't create RDONLY tablets to reduce number of tablets created to reduce resource requirements for the test. + origDefaultRdonly := defaultRdonly + defer func() { + defaultRdonly = origDefaultRdonly + }() + defaultRdonly = 0 + vc = setupMinimalCluster(t) + defer vc.TearDown() + + mtm := newMultiTenantMigration(t) + numTenantsMigrated := 0 + mtm.run() // Start the migration process for all tenants. + timer := time.NewTimer(waitTimeout) + for numTenantsMigrated < numTenants { + select { + case tenantId := <-chCompleted: + mtm.setTenantMigrationStatus(tenantId, tenantMigrationStatusMigrated) + numTenantsMigrated++ + timer.Reset(waitTimeout) + case <-timer.C: + require.FailNow(t, "Timed out waiting for all tenants to complete") + } + } + vtgateConn, closeConn := getVTGateConn() + defer closeConn() + t.Run("Verify all rows have been migrated", func(t *testing.T) { + numAdditionalInsertSets := 2 // during the SwitchTraffic stop + totalRowsInsertedPerTenant := numInitialRowsPerTenant + numAdditionalRowsPerTenant*numAdditionalInsertSets + totalRowsInserted := totalRowsInsertedPerTenant * numTenants + totalActualRowsInserted := getRowCount(t, vtgateConn, fmt.Sprintf("%s.%s", mtm.targetKeyspace, "t1")) + require.Equal(t, totalRowsInserted, totalActualRowsInserted) + log.Infof("Migration completed, total rows inserted in target: %d", totalActualRowsInserted) + }) +} + +func newMultiTenantMigration(t *testing.T) *multiTenantMigration { + _, err := vc.AddKeyspace(t, []*Cell{vc.Cells["zone1"]}, targetKeyspaceName, "0", mtVSchema, mtSchema, 1, 0, 200, nil) + require.NoError(t, err) + mtm := &multiTenantMigration{ + t: t, + tenantMigrationStatus: make(map[int64]tenantMigrationStatus), + activeMoveTables: make(map[int64]*VtctldMoveTables), + targetKeyspace: targetKeyspaceName, + tables: "t1", + tenantIdColumnName: "tenant_id", + lastIDs: make(map[int64]int64), + } + for i := 1; i <= numTenants; i++ { + mtm.setTenantMigrationStatus(int64(i), tenantMigrationStatusNotMigrated) + } + channelSize := numTenants + 1 // +1 to make sure the channels never block + for _, ch := range []*chan int64{&chNotSetup, &chNotCreated, &chInProgress, &chSwitched, &chCompleted} { + *ch = make(chan int64, channelSize) + } + return mtm +} + +func (mtm *multiTenantMigration) getTenantMigrationStatus(tenantId int64) tenantMigrationStatus { + mtm.mu.Lock() + defer mtm.mu.Unlock() + return mtm.tenantMigrationStatus[tenantId] +} + +func (mtm *multiTenantMigration) setTenantMigrationStatus(tenantId int64, status tenantMigrationStatus) { + mtm.mu.Lock() + defer mtm.mu.Unlock() + mtm.tenantMigrationStatus[tenantId] = status +} + +func (mtm *multiTenantMigration) getActiveMoveTables(tenantId int64) *VtctldMoveTables { + mtm.mu.Lock() + defer mtm.mu.Unlock() + return mtm.activeMoveTables[tenantId] +} + +func (mtm *multiTenantMigration) setActiveMoveTables(tenantId int64, mt *VtctldMoveTables) { + mtm.mu.Lock() + defer mtm.mu.Unlock() + mtm.activeMoveTables[tenantId] = mt +} + +func (mtm *multiTenantMigration) setLastID(tenantId, lastID int64) { + mtm.mu.Lock() + defer mtm.mu.Unlock() + mtm.lastIDs[tenantId] = lastID +} + +func (mtm *multiTenantMigration) getLastID(tenantId int64) int64 { + mtm.mu.Lock() + defer mtm.mu.Unlock() + return mtm.lastIDs[tenantId] +} + +func (mtm *multiTenantMigration) initTenantData(t *testing.T, tenantId int64, sourceAliasKeyspace string) { + mtm.insertSomeData(t, tenantId, getSourceKeyspace(tenantId), numInitialRowsPerTenant) +} + +func getInitialTabletIdForTenant(tenantId int64) int { + return int(baseInitialTabletId + tenantId*tabletIdStep) +} + +func (mtm *multiTenantMigration) setup(tenantId int64) { + log.Infof("Creating MoveTables for tenant %d", tenantId) + mtm.setLastID(tenantId, 0) + sourceKeyspace := getSourceKeyspace(tenantId) + sourceAliasKeyspace := getSourceAliasKeyspace(tenantId) + _, err := vc.AddKeyspace(mtm.t, []*Cell{vc.Cells["zone1"]}, sourceKeyspace, "0", stVSchema, stSchema, + 1, 0, getInitialTabletIdForTenant(tenantId), nil) + require.NoError(mtm.t, err) + mtm.initTenantData(mtm.t, tenantId, sourceAliasKeyspace) +} + +func (mtm *multiTenantMigration) start(tenantId int64) { + sourceKeyspace := getSourceKeyspace(tenantId) + sourceAliasKeyspace := getSourceAliasKeyspace(tenantId) + mtm.setTenantMigrationStatus(tenantId, tenantMigrationStatusMigrating) + mt := newVtctldMoveTables(&moveTablesWorkflow{ + workflowInfo: &workflowInfo{ + vc: vc, + workflowName: fmt.Sprintf("wf%d", tenantId), + targetKeyspace: mtm.targetKeyspace, + }, + sourceKeyspace: sourceKeyspace, + tables: mtm.tables, + createFlags: []string{ + "--tenant-id", strconv.FormatInt(tenantId, 10), + "--source-keyspace-alias", sourceAliasKeyspace, + }, + }) + mtm.setActiveMoveTables(tenantId, mt) + mt.Create() +} + +func (mtm *multiTenantMigration) switchTraffic(tenantId int64) { + t := mtm.t + sourceAliasKeyspace := getSourceAliasKeyspace(tenantId) + sourceKeyspaceName := getSourceKeyspace(tenantId) + mt := mtm.getActiveMoveTables(tenantId) + ksWorkflow := fmt.Sprintf("%s.%s", mtm.targetKeyspace, mt.workflowName) + waitForWorkflowState(t, vc, ksWorkflow, binlogdatapb.VReplicationWorkflowState_Running.String()) + // we intentionally insert first into the source alias keyspace and then the source keyspace to test routing rules for both. + mtm.insertSomeData(t, tenantId, sourceAliasKeyspace, numAdditionalRowsPerTenant) + mt.SwitchReadsAndWrites() + mtm.insertSomeData(t, tenantId, sourceKeyspaceName, numAdditionalRowsPerTenant) +} + +func (mtm *multiTenantMigration) complete(tenantId int64) { + mt := mtm.getActiveMoveTables(tenantId) + mt.Complete() + vtgateConn := vc.GetVTGateConn(mtm.t) + defer vtgateConn.Close() + waitForQueryResult(mtm.t, vtgateConn, "", + fmt.Sprintf("select count(*) from %s.t1 where tenant_id=%d", mt.targetKeyspace, tenantId), + fmt.Sprintf("[[INT64(%d)]]", mtm.getLastID(tenantId))) +} + +func randomWait() { + time.Sleep(time.Duration(rand.IntN(maxRandomDelaySeconds)) * time.Second) +} + +func (mtm *multiTenantMigration) doThis(name string, chIn, chOut chan int64, counter *atomic.Int64, f func(int64)) { + timer := time.NewTimer(waitTimeout) + for counter.Load() < numTenants { + select { + case tenantId := <-chIn: + f(tenantId) + counter.Add(1) + chOut <- tenantId + timer.Reset(waitTimeout) + case <-timer.C: + require.FailNowf(mtm.t, "Timed out: %s", name) + } + randomWait() + } +} + +// run starts the migration process for all tenants. It starts concurrent +func (mtm *multiTenantMigration) run() { + go mtm.doThis("Setup tenant keyspace/schemas", chNotSetup, chNotCreated, &numSetup, mtm.setup) + for i := int64(1); i <= numTenants; i++ { + chNotSetup <- i + } + // Wait for all tenants to be created before starting the workflows: 10 seconds per tenant to account for CI overhead. + perTenantLoadTimeout := 1 * time.Minute + require.NoError(mtm.t, waitForCondition("All tenants created", + func() bool { + return numSetup.Load() == numTenants + }, perTenantLoadTimeout*numTenants)) + + go mtm.doThis("Start Migrations", chNotCreated, chInProgress, &numInProgress, mtm.start) + go mtm.doThis("Switch Traffic", chInProgress, chSwitched, &numSwitched, mtm.switchTraffic) + go mtm.doThis("Mark Migrations Complete", chSwitched, chCompleted, &numCompleted, mtm.complete) +} diff --git a/go/test/endtoend/vreplication/resharding_workflows_v2_test.go b/go/test/endtoend/vreplication/resharding_workflows_v2_test.go index b14b649d6b7..c63fc245535 100644 --- a/go/test/endtoend/vreplication/resharding_workflows_v2_test.go +++ b/go/test/endtoend/vreplication/resharding_workflows_v2_test.go @@ -26,12 +26,14 @@ import ( "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" + "google.golang.org/protobuf/encoding/protojson" "vitess.io/vitess/go/test/endtoend/cluster" "vitess.io/vitess/go/vt/log" "vitess.io/vitess/go/vt/wrangler" binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" + vtctldatapb "vitess.io/vitess/go/vt/proto/vtctldata" ) const ( @@ -212,19 +214,42 @@ func tstWorkflowComplete(t *testing.T) error { // to primary,replica,rdonly (the only applicable types in these tests). func testWorkflowUpdate(t *testing.T) { tabletTypes := "primary,replica,rdonly" - // Test vtctlclient first + // Test vtctlclient first. _, err := vc.VtctlClient.ExecuteCommandWithOutput("workflow", "--", "--tablet-types", tabletTypes, "noexist.noexist", "update") require.Error(t, err, err) resp, err := vc.VtctlClient.ExecuteCommandWithOutput("workflow", "--", "--tablet-types", tabletTypes, ksWorkflow, "update") require.NoError(t, err) require.NotEmpty(t, resp) - // Test vtctldclient last + // Test vtctldclient last. _, err = vc.VtctldClient.ExecuteCommandWithOutput("workflow", "--keyspace", "noexist", "update", "--workflow", "noexist", "--tablet-types", tabletTypes) require.Error(t, err) + // Change the tablet-types to rdonly. + resp, err = vc.VtctldClient.ExecuteCommandWithOutput("workflow", "--keyspace", targetKs, "update", "--workflow", workflowName, "--tablet-types", "rdonly") + require.NoError(t, err, err) + // Confirm that we changed the workflow. + var ures vtctldatapb.WorkflowUpdateResponse + require.NoError(t, err) + err = protojson.Unmarshal([]byte(resp), &ures) + require.NoError(t, err) + require.Greater(t, len(ures.Details), 0) + require.True(t, ures.Details[0].Changed) + // Change tablet-types back to primary,replica,rdonly. resp, err = vc.VtctldClient.ExecuteCommandWithOutput("workflow", "--keyspace", targetKs, "update", "--workflow", workflowName, "--tablet-types", tabletTypes) require.NoError(t, err, err) - require.NotEmpty(t, resp) + // Confirm that we changed the workflow. + err = protojson.Unmarshal([]byte(resp), &ures) + require.NoError(t, err) + require.Greater(t, len(ures.Details), 0) + require.True(t, ures.Details[0].Changed) + // Execute a no-op as tablet-types is already primary,replica,rdonly. + resp, err = vc.VtctldClient.ExecuteCommandWithOutput("workflow", "--keyspace", targetKs, "update", "--workflow", workflowName, "--tablet-types", tabletTypes) + require.NoError(t, err, err) + // Confirm that we didn't change the workflow. + err = protojson.Unmarshal([]byte(resp), &ures) + require.NoError(t, err) + require.Greater(t, len(ures.Details), 0) + require.False(t, ures.Details[0].Changed) } func tstWorkflowCancel(t *testing.T) error { diff --git a/go/vt/proto/tabletmanagerdata/tabletmanagerdata.pb.go b/go/vt/proto/tabletmanagerdata/tabletmanagerdata.pb.go index 3de3681a896..4b5625c5e4c 100644 --- a/go/vt/proto/tabletmanagerdata/tabletmanagerdata.pb.go +++ b/go/vt/proto/tabletmanagerdata/tabletmanagerdata.pb.go @@ -5687,6 +5687,12 @@ func (x *VDiffOptions) GetReportOptions() *VDiffReportOptions { 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: +// cells, tablet_types, on_ddl, and state. +// TODO: leverage the optional modifier for these fields rather than using SimulatedNull +// values: https://github.com/vitessio/vitess/issues/15627 type UpdateVReplicationWorkflowRequest struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -5698,7 +5704,6 @@ type UpdateVReplicationWorkflowRequest struct { TabletSelectionPreference TabletSelectionPreference `protobuf:"varint,4,opt,name=tablet_selection_preference,json=tabletSelectionPreference,proto3,enum=tabletmanagerdata.TabletSelectionPreference" json:"tablet_selection_preference,omitempty"` OnDdl binlogdata.OnDDLAction `protobuf:"varint,5,opt,name=on_ddl,json=onDdl,proto3,enum=binlogdata.OnDDLAction" json:"on_ddl,omitempty"` State binlogdata.VReplicationWorkflowState `protobuf:"varint,6,opt,name=state,proto3,enum=binlogdata.VReplicationWorkflowState" json:"state,omitempty"` - Shards []string `protobuf:"bytes,7,rep,name=shards,proto3" json:"shards,omitempty"` } func (x *UpdateVReplicationWorkflowRequest) Reset() { @@ -5775,13 +5780,6 @@ func (x *UpdateVReplicationWorkflowRequest) GetState() binlogdata.VReplicationWo return binlogdata.VReplicationWorkflowState(0) } -func (x *UpdateVReplicationWorkflowRequest) GetShards() []string { - if x != nil { - return x.Shards - } - return nil -} - type UpdateVReplicationWorkflowResponse struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -5829,6 +5827,149 @@ func (x *UpdateVReplicationWorkflowResponse) GetResult() *query.QueryResult { return nil } +<<<<<<< HEAD +======= +// UpdateVReplicationWorkflowsRequest is used to update multiple existing VReplication +// workflows. Note that the following fields MUST have an explicit value provided +// if you do NOT wish to update the existing values to the given type's ZeroValue: +// state, message, and stop_position. +// TODO: leverage the optional modifier for these fields rather than using SimulatedNull +// values: https://github.com/vitessio/vitess/issues/15627 +type UpdateVReplicationWorkflowsRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + AllWorkflows bool `protobuf:"varint,1,opt,name=all_workflows,json=allWorkflows,proto3" json:"all_workflows,omitempty"` + IncludeWorkflows []string `protobuf:"bytes,2,rep,name=include_workflows,json=includeWorkflows,proto3" json:"include_workflows,omitempty"` + ExcludeWorkflows []string `protobuf:"bytes,3,rep,name=exclude_workflows,json=excludeWorkflows,proto3" json:"exclude_workflows,omitempty"` + State binlogdata.VReplicationWorkflowState `protobuf:"varint,4,opt,name=state,proto3,enum=binlogdata.VReplicationWorkflowState" json:"state,omitempty"` + Message string `protobuf:"bytes,5,opt,name=message,proto3" json:"message,omitempty"` + StopPosition string `protobuf:"bytes,6,opt,name=stop_position,json=stopPosition,proto3" json:"stop_position,omitempty"` +} + +func (x *UpdateVReplicationWorkflowsRequest) Reset() { + *x = UpdateVReplicationWorkflowsRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_tabletmanagerdata_proto_msgTypes[116] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *UpdateVReplicationWorkflowsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*UpdateVReplicationWorkflowsRequest) ProtoMessage() {} + +func (x *UpdateVReplicationWorkflowsRequest) ProtoReflect() protoreflect.Message { + mi := &file_tabletmanagerdata_proto_msgTypes[116] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use UpdateVReplicationWorkflowsRequest.ProtoReflect.Descriptor instead. +func (*UpdateVReplicationWorkflowsRequest) Descriptor() ([]byte, []int) { + return file_tabletmanagerdata_proto_rawDescGZIP(), []int{116} +} + +func (x *UpdateVReplicationWorkflowsRequest) GetAllWorkflows() bool { + if x != nil { + return x.AllWorkflows + } + return false +} + +func (x *UpdateVReplicationWorkflowsRequest) GetIncludeWorkflows() []string { + if x != nil { + return x.IncludeWorkflows + } + return nil +} + +func (x *UpdateVReplicationWorkflowsRequest) GetExcludeWorkflows() []string { + if x != nil { + return x.ExcludeWorkflows + } + return nil +} + +func (x *UpdateVReplicationWorkflowsRequest) GetState() binlogdata.VReplicationWorkflowState { + if x != nil { + return x.State + } + return binlogdata.VReplicationWorkflowState(0) +} + +func (x *UpdateVReplicationWorkflowsRequest) GetMessage() string { + if x != nil { + return x.Message + } + return "" +} + +func (x *UpdateVReplicationWorkflowsRequest) GetStopPosition() string { + if x != nil { + return x.StopPosition + } + return "" +} + +type UpdateVReplicationWorkflowsResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Result *query.QueryResult `protobuf:"bytes,1,opt,name=result,proto3" json:"result,omitempty"` +} + +func (x *UpdateVReplicationWorkflowsResponse) Reset() { + *x = UpdateVReplicationWorkflowsResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_tabletmanagerdata_proto_msgTypes[117] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *UpdateVReplicationWorkflowsResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*UpdateVReplicationWorkflowsResponse) ProtoMessage() {} + +func (x *UpdateVReplicationWorkflowsResponse) ProtoReflect() protoreflect.Message { + mi := &file_tabletmanagerdata_proto_msgTypes[117] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use UpdateVReplicationWorkflowsResponse.ProtoReflect.Descriptor instead. +func (*UpdateVReplicationWorkflowsResponse) Descriptor() ([]byte, []int) { + return file_tabletmanagerdata_proto_rawDescGZIP(), []int{117} +} + +func (x *UpdateVReplicationWorkflowsResponse) GetResult() *query.QueryResult { + if x != nil { + return x.Result + } + return nil +} + +>>>>>>> 0e2f1751d4 (VReplication: Fix workflow update changed handling (#15621)) type ResetSequencesRequest struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -6912,6 +7053,7 @@ var file_tabletmanagerdata_proto_rawDesc = []byte{ 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, +<<<<<<< HEAD 0x65, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x73, 0x68, 0x61, 0x72, 0x64, 0x73, 0x18, 0x07, 0x20, 0x03, 0x28, 0x09, 0x52, 0x06, 0x73, 0x68, 0x61, 0x72, 0x64, 0x73, 0x22, 0x50, 0x0a, 0x22, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, @@ -6948,6 +7090,177 @@ var file_tabletmanagerdata_proto_rawDesc = []byte{ 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, +======= + 0x65, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x65, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, + 0x67, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x72, 0x6f, 0x77, 0x73, 0x5f, 0x63, 0x6f, 0x70, 0x69, 0x65, + 0x64, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, 0x72, 0x6f, 0x77, 0x73, 0x43, 0x6f, 0x70, + 0x69, 0x65, 0x64, 0x12, 0x33, 0x0a, 0x0e, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x68, 0x65, 0x61, 0x72, + 0x74, 0x62, 0x65, 0x61, 0x74, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x76, 0x74, + 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x52, 0x0d, 0x74, 0x69, 0x6d, 0x65, 0x48, + 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x12, 0x33, 0x0a, 0x0e, 0x74, 0x69, 0x6d, 0x65, + 0x5f, 0x74, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x64, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x0c, 0x2e, 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x52, 0x0d, + 0x74, 0x69, 0x6d, 0x65, 0x54, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x64, 0x12, 0x2f, 0x0a, + 0x13, 0x63, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x5f, 0x74, 0x68, 0x72, 0x6f, 0x74, + 0x74, 0x6c, 0x65, 0x64, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x09, 0x52, 0x12, 0x63, 0x6f, 0x6d, 0x70, + 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x54, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x64, 0x22, 0xd7, + 0x01, 0x0a, 0x0c, 0x56, 0x44, 0x69, 0x66, 0x66, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, + 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x77, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x77, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x12, 0x16, 0x0a, 0x06, 0x61, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, + 0x1d, 0x0a, 0x0a, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x61, 0x72, 0x67, 0x18, 0x04, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x09, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x72, 0x67, 0x12, 0x1d, + 0x0a, 0x0a, 0x76, 0x64, 0x69, 0x66, 0x66, 0x5f, 0x75, 0x75, 0x69, 0x64, 0x18, 0x05, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x09, 0x76, 0x64, 0x69, 0x66, 0x66, 0x55, 0x75, 0x69, 0x64, 0x12, 0x39, 0x0a, + 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, + 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, + 0x74, 0x61, 0x2e, 0x56, 0x44, 0x69, 0x66, 0x66, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, + 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0x6a, 0x0a, 0x0d, 0x56, 0x44, 0x69, 0x66, + 0x66, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x02, 0x69, 0x64, 0x12, 0x2a, 0x0a, 0x06, 0x6f, 0x75, 0x74, + 0x70, 0x75, 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, 0x6f, + 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x1d, 0x0a, 0x0a, 0x76, 0x64, 0x69, 0x66, 0x66, 0x5f, 0x75, + 0x75, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x76, 0x64, 0x69, 0x66, 0x66, + 0x55, 0x75, 0x69, 0x64, 0x22, 0x79, 0x0a, 0x12, 0x56, 0x44, 0x69, 0x66, 0x66, 0x50, 0x69, 0x63, + 0x6b, 0x65, 0x72, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x21, 0x0a, 0x0c, 0x74, 0x61, + 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x54, 0x79, 0x70, 0x65, 0x73, 0x12, 0x1f, 0x0a, + 0x0b, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x63, 0x65, 0x6c, 0x6c, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x0a, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x43, 0x65, 0x6c, 0x6c, 0x12, 0x1f, + 0x0a, 0x0b, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x5f, 0x63, 0x65, 0x6c, 0x6c, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x0a, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x43, 0x65, 0x6c, 0x6c, 0x22, + 0x90, 0x01, 0x0a, 0x12, 0x56, 0x44, 0x69, 0x66, 0x66, 0x52, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x4f, + 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x19, 0x0a, 0x08, 0x6f, 0x6e, 0x6c, 0x79, 0x5f, 0x70, + 0x6b, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x6f, 0x6e, 0x6c, 0x79, 0x50, 0x6b, + 0x73, 0x12, 0x1f, 0x0a, 0x0b, 0x64, 0x65, 0x62, 0x75, 0x67, 0x5f, 0x71, 0x75, 0x65, 0x72, 0x79, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0a, 0x64, 0x65, 0x62, 0x75, 0x67, 0x51, 0x75, 0x65, + 0x72, 0x79, 0x12, 0x16, 0x0a, 0x06, 0x66, 0x6f, 0x72, 0x6d, 0x61, 0x74, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x06, 0x66, 0x6f, 0x72, 0x6d, 0x61, 0x74, 0x12, 0x26, 0x0a, 0x0f, 0x6d, 0x61, + 0x78, 0x5f, 0x73, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x04, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x0d, 0x6d, 0x61, 0x78, 0x53, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x52, 0x6f, + 0x77, 0x73, 0x22, 0xda, 0x02, 0x0a, 0x10, 0x56, 0x44, 0x69, 0x66, 0x66, 0x43, 0x6f, 0x72, 0x65, + 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x16, 0x0a, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, + 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x12, + 0x1d, 0x0a, 0x0a, 0x61, 0x75, 0x74, 0x6f, 0x5f, 0x72, 0x65, 0x74, 0x72, 0x79, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x08, 0x52, 0x09, 0x61, 0x75, 0x74, 0x6f, 0x52, 0x65, 0x74, 0x72, 0x79, 0x12, 0x19, + 0x0a, 0x08, 0x6d, 0x61, 0x78, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x07, 0x6d, 0x61, 0x78, 0x52, 0x6f, 0x77, 0x73, 0x12, 0x1a, 0x0a, 0x08, 0x63, 0x68, 0x65, + 0x63, 0x6b, 0x73, 0x75, 0x6d, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, 0x63, 0x68, 0x65, + 0x63, 0x6b, 0x73, 0x75, 0x6d, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x5f, + 0x70, 0x63, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x73, 0x61, 0x6d, 0x70, 0x6c, + 0x65, 0x50, 0x63, 0x74, 0x12, 0x27, 0x0a, 0x0f, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x5f, + 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x06, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0e, 0x74, + 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x53, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x12, 0x38, 0x0a, + 0x19, 0x6d, 0x61, 0x78, 0x5f, 0x65, 0x78, 0x74, 0x72, 0x61, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x5f, + 0x74, 0x6f, 0x5f, 0x63, 0x6f, 0x6d, 0x70, 0x61, 0x72, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x15, 0x6d, 0x61, 0x78, 0x45, 0x78, 0x74, 0x72, 0x61, 0x52, 0x6f, 0x77, 0x73, 0x54, 0x6f, + 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x72, 0x65, 0x12, 0x2c, 0x0a, 0x12, 0x75, 0x70, 0x64, 0x61, 0x74, + 0x65, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x73, 0x18, 0x08, 0x20, + 0x01, 0x28, 0x08, 0x52, 0x10, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, + 0x53, 0x74, 0x61, 0x74, 0x73, 0x12, 0x28, 0x0a, 0x10, 0x6d, 0x61, 0x78, 0x5f, 0x64, 0x69, 0x66, + 0x66, 0x5f, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x09, 0x20, 0x01, 0x28, 0x03, 0x52, + 0x0e, 0x6d, 0x61, 0x78, 0x44, 0x69, 0x66, 0x66, 0x53, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x22, + 0xf2, 0x01, 0x0a, 0x0c, 0x56, 0x44, 0x69, 0x66, 0x66, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, + 0x12, 0x4c, 0x0a, 0x0e, 0x70, 0x69, 0x63, 0x6b, 0x65, 0x72, 0x5f, 0x6f, 0x70, 0x74, 0x69, 0x6f, + 0x6e, 0x73, 0x18, 0x01, 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, 0x50, 0x69, 0x63, 0x6b, 0x65, 0x72, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, + 0x0d, 0x70, 0x69, 0x63, 0x6b, 0x65, 0x72, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x46, + 0x0a, 0x0c, 0x63, 0x6f, 0x72, 0x65, 0x5f, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, + 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x44, 0x69, 0x66, 0x66, 0x43, 0x6f, + 0x72, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x0b, 0x63, 0x6f, 0x72, 0x65, 0x4f, + 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x4c, 0x0a, 0x0e, 0x72, 0x65, 0x70, 0x6f, 0x72, 0x74, + 0x5f, 0x6f, 0x70, 0x74, 0x69, 0x6f, 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, 0xef, 0x02, 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, 0x6c, 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, 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, 0x12, 0x2e, 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, 0x52, 0x05, 0x6f, 0x6e, + 0x44, 0x64, 0x6c, 0x12, 0x3b, 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, 0x52, 0x05, 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, 0x9f, 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, 0x3b, + 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, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 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, 0x23, 0x0a, 0x0d, 0x73, 0x74, 0x6f, 0x70, 0x5f, 0x70, 0x6f, + 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x73, 0x74, + 0x6f, 0x70, 0x50, 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, 0x32, 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, 0x22, 0xc8, 0x01, 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, 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, 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, +>>>>>>> 0e2f1751d4 (VReplication: Fix workflow update changed handling (#15621)) } var ( diff --git a/go/vt/proto/tabletmanagerdata/tabletmanagerdata_vtproto.pb.go b/go/vt/proto/tabletmanagerdata/tabletmanagerdata_vtproto.pb.go index 4d5b4188e5e..c03bdb152f3 100644 --- a/go/vt/proto/tabletmanagerdata/tabletmanagerdata_vtproto.pb.go +++ b/go/vt/proto/tabletmanagerdata/tabletmanagerdata_vtproto.pb.go @@ -2159,11 +2159,6 @@ func (m *UpdateVReplicationWorkflowRequest) CloneVT() *UpdateVReplicationWorkflo copy(tmpContainer, rhs) r.TabletTypes = tmpContainer } - if rhs := m.Shards; rhs != nil { - tmpContainer := make([]string, len(rhs)) - copy(tmpContainer, rhs) - r.Shards = tmpContainer - } if len(m.unknownFields) > 0 { r.unknownFields = make([]byte, len(m.unknownFields)) copy(r.unknownFields, m.unknownFields) @@ -7435,15 +7430,6 @@ func (m *UpdateVReplicationWorkflowRequest) MarshalToSizedBufferVT(dAtA []byte) i -= len(m.unknownFields) copy(dAtA[i:], m.unknownFields) } - if len(m.Shards) > 0 { - for iNdEx := len(m.Shards) - 1; iNdEx >= 0; iNdEx-- { - i -= len(m.Shards[iNdEx]) - copy(dAtA[i:], m.Shards[iNdEx]) - i = encodeVarint(dAtA, i, uint64(len(m.Shards[iNdEx]))) - i-- - dAtA[i] = 0x3a - } - } if m.State != 0 { i = encodeVarint(dAtA, i, uint64(m.State)) i-- @@ -9587,12 +9573,6 @@ func (m *UpdateVReplicationWorkflowRequest) SizeVT() (n int) { if m.State != 0 { n += 1 + sov(uint64(m.State)) } - if len(m.Shards) > 0 { - for _, s := range m.Shards { - l = len(s) - n += 1 + l + sov(uint64(l)) - } - } n += len(m.unknownFields) return n } @@ -21105,38 +21085,6 @@ func (m *UpdateVReplicationWorkflowRequest) UnmarshalVT(dAtA []byte) error { break } } - case 7: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Shards", wireType) - } - var stringLen uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflow - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - stringLen |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLength - } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLength - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.Shards = append(m.Shards, string(dAtA[iNdEx:postIndex])) - iNdEx = postIndex default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) diff --git a/go/vt/vtctl/vtctl.go b/go/vt/vtctl/vtctl.go index 919783e2f76..324cdda0a76 100644 --- a/go/vt/vtctl/vtctl.go +++ b/go/vt/vtctl/vtctl.go @@ -3829,6 +3829,7 @@ func commandWorkflow(ctx context.Context, wr *wrangler.Wrangler, subFlags *pflag TabletTypes: tabletTypes, TabletSelectionPreference: tsp, OnDdl: binlogdatapb.OnDDLAction(onddl), + State: binlogdatapb.VReplicationWorkflowState(textutil.SimulatedNullInt), // We don't allow changing this in the client command } } results, err = wr.WorkflowAction(ctx, workflow, keyspace, action, *dryRun, rpcReq, *shards) // Only update currently uses the new RPC path diff --git a/go/vt/vtctl/workflow/server.go b/go/vt/vtctl/workflow/server.go index af0f16b2138..635d47d8bf2 100644 --- a/go/vt/vtctl/workflow/server.go +++ b/go/vt/vtctl/workflow/server.go @@ -2235,10 +2235,8 @@ func (s *Server) WorkflowUpdate(ctx context.Context, req *vtctldatapb.WorkflowUp span.Annotate("tablet_types", req.TabletRequest.TabletTypes) span.Annotate("on_ddl", req.TabletRequest.OnDdl) span.Annotate("state", req.TabletRequest.State) - span.Annotate("shards", req.TabletRequest.Shards) vx := vexec.NewVExec(req.Keyspace, req.TabletRequest.Workflow, s.ts, s.tmc, s.env.Parser()) - vx.SetShardSubset(req.TabletRequest.Shards) callback := func(ctx context.Context, tablet *topo.TabletInfo) (*querypb.QueryResult, error) { res, err := s.tmc.UpdateVReplicationWorkflow(ctx, tablet.Tablet, req.TabletRequest) if err != nil { diff --git a/go/vt/vttablet/tabletmanager/rpc_vreplication.go b/go/vt/vttablet/tabletmanager/rpc_vreplication.go index 02c2eb10056..aad18ef6a45 100644 --- a/go/vt/vttablet/tabletmanager/rpc_vreplication.go +++ b/go/vt/vttablet/tabletmanager/rpc_vreplication.go @@ -257,6 +257,7 @@ func (tm *TabletManager) UpdateVReplicationWorkflow(ctx context.Context, req *ta return &tabletmanagerdatapb.UpdateVReplicationWorkflowResponse{Result: nil}, nil } + rowsAffected := uint64(0) for _, row := range res.Named().Rows { id := row.AsInt64("id", 0) cells := strings.Split(row.AsString("cell", ""), ",") @@ -317,15 +318,42 @@ func (tm *TabletManager) UpdateVReplicationWorkflow(ctx context.Context, req *ta if err != nil { return nil, err } + rowsAffected += res.RowsAffected } return &tabletmanagerdatapb.UpdateVReplicationWorkflowResponse{ Result: &querypb.QueryResult{ - RowsAffected: uint64(len(res.Rows)), + RowsAffected: rowsAffected, }, }, nil } +<<<<<<< HEAD +======= +// UpdateVReplicationWorkflows operates in much the same way that +// UpdateVReplicationWorkflow does, but it allows you to update the +// metadata/flow control fields -- state, message, and stop_pos -- for +// multiple workflows. +// Note: today this is only used during Reshard as all of the vreplication +// streams need to be migrated from the old shards to the new ones. +func (tm *TabletManager) UpdateVReplicationWorkflows(ctx context.Context, req *tabletmanagerdatapb.UpdateVReplicationWorkflowsRequest) (*tabletmanagerdatapb.UpdateVReplicationWorkflowsResponse, error) { + query, err := tm.buildUpdateVReplicationWorkflowsQuery(req) + if err != nil { + return nil, err + } + res, err := tm.VREngine.Exec(query) + if err != nil { + return nil, err + } + + return &tabletmanagerdatapb.UpdateVReplicationWorkflowsResponse{ + Result: &querypb.QueryResult{ + RowsAffected: res.RowsAffected, + }, + }, nil +} + +>>>>>>> 0e2f1751d4 (VReplication: Fix workflow update changed handling (#15621)) // VReplicationExec executes a vreplication command. func (tm *TabletManager) VReplicationExec(ctx context.Context, query string) (*querypb.QueryResult, error) { // Replace any provided sidecar database qualifiers with the correct one. diff --git a/proto/tabletmanagerdata.proto b/proto/tabletmanagerdata.proto index e1a0e97f03e..6a925842087 100644 --- a/proto/tabletmanagerdata.proto +++ b/proto/tabletmanagerdata.proto @@ -631,6 +631,12 @@ message VDiffOptions { VDiffReportOptions report_options = 3; } +// 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: +// cells, tablet_types, on_ddl, and state. +// TODO: leverage the optional modifier for these fields rather than using SimulatedNull +// values: https://github.com/vitessio/vitess/issues/15627 message UpdateVReplicationWorkflowRequest { string workflow = 1; repeated string cells = 2; @@ -638,13 +644,35 @@ message UpdateVReplicationWorkflowRequest { TabletSelectionPreference tablet_selection_preference = 4; binlogdata.OnDDLAction on_ddl = 5; binlogdata.VReplicationWorkflowState state = 6; - repeated string shards = 7; + reserved 7; // unused, was: repeated string shards } message UpdateVReplicationWorkflowResponse { query.QueryResult result = 1; } +<<<<<<< HEAD +======= +// UpdateVReplicationWorkflowsRequest is used to update multiple existing VReplication +// workflows. Note that the following fields MUST have an explicit value provided +// if you do NOT wish to update the existing values to the given type's ZeroValue: +// state, message, and stop_position. +// TODO: leverage the optional modifier for these fields rather than using SimulatedNull +// values: https://github.com/vitessio/vitess/issues/15627 +message UpdateVReplicationWorkflowsRequest { + bool all_workflows = 1; + repeated string include_workflows = 2; + repeated string exclude_workflows = 3; + binlogdata.VReplicationWorkflowState state = 4; + string message = 5; + string stop_position = 6; +} + +message UpdateVReplicationWorkflowsResponse { + query.QueryResult result = 1; +} + +>>>>>>> 0e2f1751d4 (VReplication: Fix workflow update changed handling (#15621)) message ResetSequencesRequest { repeated string tables = 1; } diff --git a/web/vtadmin/src/proto/vtadmin.d.ts b/web/vtadmin/src/proto/vtadmin.d.ts index 53fc9df7e7f..69f7891e271 100644 --- a/web/vtadmin/src/proto/vtadmin.d.ts +++ b/web/vtadmin/src/proto/vtadmin.d.ts @@ -28927,9 +28927,6 @@ export namespace tabletmanagerdata { /** UpdateVReplicationWorkflowRequest state */ state?: (binlogdata.VReplicationWorkflowState|null); - - /** UpdateVReplicationWorkflowRequest shards */ - shards?: (string[]|null); } /** Represents an UpdateVReplicationWorkflowRequest. */ @@ -28959,9 +28956,6 @@ export namespace tabletmanagerdata { /** UpdateVReplicationWorkflowRequest state. */ public state: binlogdata.VReplicationWorkflowState; - /** UpdateVReplicationWorkflowRequest shards. */ - public shards: string[]; - /** * Creates a new UpdateVReplicationWorkflowRequest instance using the specified properties. * @param [properties] Properties to set diff --git a/web/vtadmin/src/proto/vtadmin.js b/web/vtadmin/src/proto/vtadmin.js index fc8c6a862fe..3263bc3eb51 100644 --- a/web/vtadmin/src/proto/vtadmin.js +++ b/web/vtadmin/src/proto/vtadmin.js @@ -66687,7 +66687,6 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { * @property {tabletmanagerdata.TabletSelectionPreference|null} [tablet_selection_preference] UpdateVReplicationWorkflowRequest tablet_selection_preference * @property {binlogdata.OnDDLAction|null} [on_ddl] UpdateVReplicationWorkflowRequest on_ddl * @property {binlogdata.VReplicationWorkflowState|null} [state] UpdateVReplicationWorkflowRequest state - * @property {Array.|null} [shards] UpdateVReplicationWorkflowRequest shards */ /** @@ -66701,7 +66700,6 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { function UpdateVReplicationWorkflowRequest(properties) { this.cells = []; this.tablet_types = []; - this.shards = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -66756,14 +66754,6 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { */ UpdateVReplicationWorkflowRequest.prototype.state = 0; - /** - * UpdateVReplicationWorkflowRequest shards. - * @member {Array.} shards - * @memberof tabletmanagerdata.UpdateVReplicationWorkflowRequest - * @instance - */ - UpdateVReplicationWorkflowRequest.prototype.shards = $util.emptyArray; - /** * Creates a new UpdateVReplicationWorkflowRequest instance using the specified properties. * @function create @@ -66805,9 +66795,6 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { writer.uint32(/* id 5, wireType 0 =*/40).int32(message.on_ddl); if (message.state != null && Object.hasOwnProperty.call(message, "state")) writer.uint32(/* id 6, wireType 0 =*/48).int32(message.state); - if (message.shards != null && message.shards.length) - for (let i = 0; i < message.shards.length; ++i) - writer.uint32(/* id 7, wireType 2 =*/58).string(message.shards[i]); return writer; }; @@ -66875,12 +66862,6 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { message.state = reader.int32(); break; } - case 7: { - if (!(message.shards && message.shards.length)) - message.shards = []; - message.shards.push(reader.string()); - break; - } default: reader.skipType(tag & 7); break; @@ -66979,13 +66960,6 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { case 6: break; } - if (message.shards != null && message.hasOwnProperty("shards")) { - if (!Array.isArray(message.shards)) - return "shards: array expected"; - for (let i = 0; i < message.shards.length; ++i) - if (!$util.isString(message.shards[i])) - return "shards: string[] expected"; - } return null; }; @@ -67147,13 +67121,6 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { message.state = 6; break; } - if (object.shards) { - if (!Array.isArray(object.shards)) - throw TypeError(".tabletmanagerdata.UpdateVReplicationWorkflowRequest.shards: array expected"); - message.shards = []; - for (let i = 0; i < object.shards.length; ++i) - message.shards[i] = String(object.shards[i]); - } return message; }; @@ -67173,7 +67140,6 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { if (options.arrays || options.defaults) { object.cells = []; object.tablet_types = []; - object.shards = []; } if (options.defaults) { object.workflow = ""; @@ -67199,11 +67165,6 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { object.on_ddl = options.enums === String ? $root.binlogdata.OnDDLAction[message.on_ddl] === undefined ? message.on_ddl : $root.binlogdata.OnDDLAction[message.on_ddl] : message.on_ddl; if (message.state != null && message.hasOwnProperty("state")) object.state = options.enums === String ? $root.binlogdata.VReplicationWorkflowState[message.state] === undefined ? message.state : $root.binlogdata.VReplicationWorkflowState[message.state] : message.state; - if (message.shards && message.shards.length) { - object.shards = []; - for (let j = 0; j < message.shards.length; ++j) - object.shards[j] = message.shards[j]; - } return object; }; From f9c1a837c31b898bba571b4ad189a18b3cc39791 Mon Sep 17 00:00:00 2001 From: Matt Lord Date: Wed, 3 Apr 2024 14:38:08 -0400 Subject: [PATCH 2/2] Adjust for v19 Signed-off-by: Matt Lord --- .../vreplication/multi_tenant_test.go | 464 ------------------ .../tabletmanagerdata/tabletmanagerdata.pb.go | 388 ++------------- .../tabletmanager/rpc_vreplication.go | 26 - proto/tabletmanagerdata.proto | 22 - 4 files changed, 36 insertions(+), 864 deletions(-) delete mode 100644 go/test/endtoend/vreplication/multi_tenant_test.go diff --git a/go/test/endtoend/vreplication/multi_tenant_test.go b/go/test/endtoend/vreplication/multi_tenant_test.go deleted file mode 100644 index 3130e498f5c..00000000000 --- a/go/test/endtoend/vreplication/multi_tenant_test.go +++ /dev/null @@ -1,464 +0,0 @@ -/* -Copyright 2024 The Vitess Authors. - -Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. -*/ - -/* -Consists of two tests. Both tests are for multi-tenant migration scenarios. - -1. TestMultiTenantSimple: migrates a single tenant to a target keyspace. - -2. TestMultiTenantComplex: migrates multiple tenants to a single target keyspace, with concurrent migrations. - -The tests use the MoveTables workflow to migrate the tenants. They are designed to simulate a real-world multi-tenant -migration scenario, where each tenant is in a separate database. -*/ - -package vreplication - -import ( - "encoding/json" - "fmt" - "math/rand/v2" - "strconv" - "sync" - "sync/atomic" - "testing" - "time" - - "github.com/stretchr/testify/require" - - "vitess.io/vitess/go/test/endtoend/cluster" - "vitess.io/vitess/go/vt/log" - binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" - vschemapb "vitess.io/vitess/go/vt/proto/vschema" -) - -type tenantMigrationStatus int - -const ( - tenantMigrationStatusNotMigrated tenantMigrationStatus = iota - tenantMigrationStatusMigrating - tenantMigrationStatusMigrated - - sourceKeyspaceTemplate = "s%d" - sourceAliasKeyspaceTemplate = "a%d" - targetKeyspaceName = "mt" - - numTenants = 10 - numInitialRowsPerTenant = 10 - numAdditionalRowsPerTenant = 10 - baseInitialTabletId = 1000 - tabletIdStep = 100 - maxRandomDelaySeconds = 5 - waitTimeout = 10 * time.Minute -) - -var ( - // channels to coordinate the migration workflow - chNotSetup, chNotCreated, chInProgress, chSwitched, chCompleted chan int64 - // counters to keep track of the number of tenants in each state - numSetup, numInProgress, numSwitched, numCompleted atomic.Int64 -) - -// multiTenantMigration manages the migration of multiple tenants to a single target keyspace. -// A singleton object of this type is created for the test case. -type multiTenantMigration struct { - t *testing.T - mu sync.Mutex - tenantMigrationStatus map[int64]tenantMigrationStatus // current migration status for each tenant - activeMoveTables map[int64]*VtctldMoveTables // the internal MoveTables object for each tenant - - targetKeyspace string - tables string - tenantIdColumnName string // the name of the column in each table that holds the tenant ID - - lastIDs map[int64]int64 // the last primary key inserted for each tenant -} - -const ( - mtSchema = "create table t1(id int, tenant_id int, primary key(id, tenant_id)) Engine=InnoDB" - mtVSchema = ` -{ - "multi_tenant_spec": { - "tenant_id_column_name": "tenant_id", - "tenant_id_column_type": "INT64" - }, - "tables": { - "t1": {} - } -} -` - stSchema = mtSchema - stVSchema = ` -{ - "tables": { - "t1": {} - } -} -` -) - -// TestMultiTenantSimple tests a single tenant migration. The aim here is to test all the steps of the migration process -// including keyspace routing rules, addition of tenant filters to the forward and reverse vreplication streams, and -// verifying that the data is migrated correctly. -func TestMultiTenantSimple(t *testing.T) { - setSidecarDBName("_vt") - // Don't create RDONLY tablets to reduce number of tablets created to reduce resource requirements for the test. - origDefaultRdonly := defaultRdonly - defer func() { - defaultRdonly = origDefaultRdonly - }() - defaultRdonly = 0 - vc = setupMinimalCluster(t) - defer vc.TearDown() - - targetKeyspace := "mt" - _, err := vc.AddKeyspace(t, []*Cell{vc.Cells["zone1"]}, targetKeyspace, "0", mtVSchema, mtSchema, 1, 0, 200, nil) - require.NoError(t, err) - - tenantId := int64(1) - sourceKeyspace := getSourceKeyspace(tenantId) - sourceAliasKeyspace := getSourceAliasKeyspace(tenantId) - _, err = vc.AddKeyspace(t, []*Cell{vc.Cells["zone1"]}, sourceKeyspace, "0", stVSchema, stSchema, 1, 0, getInitialTabletIdForTenant(tenantId), nil) - require.NoError(t, err) - - targetPrimary := vc.getPrimaryTablet(t, targetKeyspace, "0") - sourcePrimary := vc.getPrimaryTablet(t, sourceKeyspace, "0") - primaries := map[string]*cluster.VttabletProcess{ - "target": targetPrimary, - "source": sourcePrimary, - } - - vtgateConn, closeConn := getVTGateConn() - defer closeConn() - numRows := 10 - lastIndex := int64(0) - insertRows := func(lastIndex int64, keyspace string) int64 { - for i := 1; i <= numRows; i++ { - execQueryWithRetry(t, vtgateConn, - fmt.Sprintf("insert into %s.t1(id, tenant_id) values(%d, %d)", keyspace, int64(i)+lastIndex, tenantId), queryTimeout) - } - return int64(numRows) + lastIndex - } - lastIndex = insertRows(lastIndex, sourceKeyspace) - - mt := newVtctldMoveTables(&moveTablesWorkflow{ - workflowInfo: &workflowInfo{ - vc: vc, - workflowName: fmt.Sprintf("wf%d", tenantId), - targetKeyspace: targetKeyspace, - }, - sourceKeyspace: sourceKeyspace, - createFlags: []string{ - "--tenant-id", strconv.FormatInt(tenantId, 10), - "--source-keyspace-alias", sourceAliasKeyspace, - }, - }) - - preSwitchRules := &vschemapb.KeyspaceRoutingRules{ - Rules: []*vschemapb.KeyspaceRoutingRule{ - {FromKeyspace: "a1", ToKeyspace: "s1"}, - {FromKeyspace: "s1", ToKeyspace: "s1"}, - }, - } - postSwitchRules := &vschemapb.KeyspaceRoutingRules{ - Rules: []*vschemapb.KeyspaceRoutingRule{ - {FromKeyspace: "a1", ToKeyspace: "mt"}, - {FromKeyspace: "s1", ToKeyspace: "mt"}, - }, - } - rulesMap := map[string]*vschemapb.KeyspaceRoutingRules{ - "pre": preSwitchRules, - "post": postSwitchRules, - } - require.Zero(t, len(getKeyspaceRoutingRules(t, vc).Rules)) - mt.Create() - validateKeyspaceRoutingRules(t, vc, primaries, rulesMap, false) - // Note: we cannot insert into the target keyspace since that is never routed to the source keyspace. - for _, ks := range []string{sourceKeyspace, sourceAliasKeyspace} { - lastIndex = insertRows(lastIndex, ks) - } - waitForWorkflowState(t, vc, fmt.Sprintf("%s.%s", targetKeyspace, mt.workflowName), binlogdatapb.VReplicationWorkflowState_Running.String()) - mt.SwitchReadsAndWrites() - validateKeyspaceRoutingRules(t, vc, primaries, rulesMap, true) - // Note: here we have already switched and we can insert into the target keyspace and it should get reverse - // replicated to the source keyspace. The source and alias are also routed to the target keyspace at this point. - for _, ks := range []string{sourceKeyspace, sourceAliasKeyspace, targetKeyspace} { - lastIndex = insertRows(lastIndex, ks) - } - mt.Complete() - require.Zero(t, len(getKeyspaceRoutingRules(t, vc).Rules)) - actualRowsInserted := getRowCount(t, vtgateConn, fmt.Sprintf("%s.%s", targetKeyspace, "t1")) - log.Infof("Migration completed, total rows in target: %d", actualRowsInserted) - require.Equal(t, lastIndex, int64(actualRowsInserted)) -} - -// If switched queries with source/alias qualifiers should execute on target, else on source. Confirm that -// the routing rules are as expected and that the query executes on the expected tablet. -func validateKeyspaceRoutingRules(t *testing.T, vc *VitessCluster, primaries map[string]*cluster.VttabletProcess, rulesMap map[string]*vschemapb.KeyspaceRoutingRules, switched bool) { - currentRules := getKeyspaceRoutingRules(t, vc) - vtgateConn, closeConn := getVTGateConn() - defer closeConn() - queryTemplate := "select count(*) from %s.t1" - matchQuery := "select count(*) from t1" - - validateQueryRoute := func(qualifier, dest string) { - query := fmt.Sprintf(queryTemplate, qualifier) - assertQueryExecutesOnTablet(t, vtgateConn, primaries[dest], "", query, matchQuery) - log.Infof("query %s executed on %s", query, dest) - } - - if switched { - require.ElementsMatch(t, rulesMap["post"].Rules, currentRules.Rules) - validateQueryRoute("mt", "target") - validateQueryRoute("s1", "target") - validateQueryRoute("a1", "target") - } else { - require.ElementsMatch(t, rulesMap["pre"].Rules, currentRules.Rules) - // Note that with multi-tenant migration, we cannot redirect the target keyspace since - // there are multiple source keyspaces and the target has the aggregate of all the tenants. - validateQueryRoute("mt", "target") - validateQueryRoute("s1", "source") - validateQueryRoute("a1", "source") - } -} - -func getSourceKeyspace(tenantId int64) string { - return fmt.Sprintf(sourceKeyspaceTemplate, tenantId) -} - -func getSourceAliasKeyspace(tenantId int64) string { - return fmt.Sprintf(sourceAliasKeyspaceTemplate, tenantId) -} - -func (mtm *multiTenantMigration) insertSomeData(t *testing.T, tenantId int64, keyspace string, numRows int64) { - vtgateConn, closeConn := getVTGateConn() - defer closeConn() - idx := mtm.getLastID(tenantId) - for i := idx + 1; i <= idx+numRows; i++ { - execQueryWithRetry(t, vtgateConn, - fmt.Sprintf("insert into %s.t1(id, tenant_id) values(%d, %d)", keyspace, i, tenantId), queryTimeout) - } - mtm.setLastID(tenantId, idx+numRows) -} - -func getKeyspaceRoutingRules(t *testing.T, vc *VitessCluster) *vschemapb.KeyspaceRoutingRules { - output, err := vc.VtctldClient.ExecuteCommandWithOutput("GetKeyspaceRoutingRules") - require.NoError(t, err) - rules := &vschemapb.KeyspaceRoutingRules{} - err = json.Unmarshal([]byte(output), rules) - require.NoError(t, err) - return rules -} - -// TestMultiTenant tests a multi-tenant migration scenario where each tenant is in a separate database. -// It uses MoveTables to migrate all tenants to the same target keyspace. The test creates a separate source keyspace -// for each tenant. It then steps through the migration process for each tenant, and verifies that the data is migrated -// correctly. The migration steps are done concurrently and randomly to simulate an actual multi-tenant migration. -func TestMultiTenantComplex(t *testing.T) { - setSidecarDBName("_vt") - // Don't create RDONLY tablets to reduce number of tablets created to reduce resource requirements for the test. - origDefaultRdonly := defaultRdonly - defer func() { - defaultRdonly = origDefaultRdonly - }() - defaultRdonly = 0 - vc = setupMinimalCluster(t) - defer vc.TearDown() - - mtm := newMultiTenantMigration(t) - numTenantsMigrated := 0 - mtm.run() // Start the migration process for all tenants. - timer := time.NewTimer(waitTimeout) - for numTenantsMigrated < numTenants { - select { - case tenantId := <-chCompleted: - mtm.setTenantMigrationStatus(tenantId, tenantMigrationStatusMigrated) - numTenantsMigrated++ - timer.Reset(waitTimeout) - case <-timer.C: - require.FailNow(t, "Timed out waiting for all tenants to complete") - } - } - vtgateConn, closeConn := getVTGateConn() - defer closeConn() - t.Run("Verify all rows have been migrated", func(t *testing.T) { - numAdditionalInsertSets := 2 // during the SwitchTraffic stop - totalRowsInsertedPerTenant := numInitialRowsPerTenant + numAdditionalRowsPerTenant*numAdditionalInsertSets - totalRowsInserted := totalRowsInsertedPerTenant * numTenants - totalActualRowsInserted := getRowCount(t, vtgateConn, fmt.Sprintf("%s.%s", mtm.targetKeyspace, "t1")) - require.Equal(t, totalRowsInserted, totalActualRowsInserted) - log.Infof("Migration completed, total rows inserted in target: %d", totalActualRowsInserted) - }) -} - -func newMultiTenantMigration(t *testing.T) *multiTenantMigration { - _, err := vc.AddKeyspace(t, []*Cell{vc.Cells["zone1"]}, targetKeyspaceName, "0", mtVSchema, mtSchema, 1, 0, 200, nil) - require.NoError(t, err) - mtm := &multiTenantMigration{ - t: t, - tenantMigrationStatus: make(map[int64]tenantMigrationStatus), - activeMoveTables: make(map[int64]*VtctldMoveTables), - targetKeyspace: targetKeyspaceName, - tables: "t1", - tenantIdColumnName: "tenant_id", - lastIDs: make(map[int64]int64), - } - for i := 1; i <= numTenants; i++ { - mtm.setTenantMigrationStatus(int64(i), tenantMigrationStatusNotMigrated) - } - channelSize := numTenants + 1 // +1 to make sure the channels never block - for _, ch := range []*chan int64{&chNotSetup, &chNotCreated, &chInProgress, &chSwitched, &chCompleted} { - *ch = make(chan int64, channelSize) - } - return mtm -} - -func (mtm *multiTenantMigration) getTenantMigrationStatus(tenantId int64) tenantMigrationStatus { - mtm.mu.Lock() - defer mtm.mu.Unlock() - return mtm.tenantMigrationStatus[tenantId] -} - -func (mtm *multiTenantMigration) setTenantMigrationStatus(tenantId int64, status tenantMigrationStatus) { - mtm.mu.Lock() - defer mtm.mu.Unlock() - mtm.tenantMigrationStatus[tenantId] = status -} - -func (mtm *multiTenantMigration) getActiveMoveTables(tenantId int64) *VtctldMoveTables { - mtm.mu.Lock() - defer mtm.mu.Unlock() - return mtm.activeMoveTables[tenantId] -} - -func (mtm *multiTenantMigration) setActiveMoveTables(tenantId int64, mt *VtctldMoveTables) { - mtm.mu.Lock() - defer mtm.mu.Unlock() - mtm.activeMoveTables[tenantId] = mt -} - -func (mtm *multiTenantMigration) setLastID(tenantId, lastID int64) { - mtm.mu.Lock() - defer mtm.mu.Unlock() - mtm.lastIDs[tenantId] = lastID -} - -func (mtm *multiTenantMigration) getLastID(tenantId int64) int64 { - mtm.mu.Lock() - defer mtm.mu.Unlock() - return mtm.lastIDs[tenantId] -} - -func (mtm *multiTenantMigration) initTenantData(t *testing.T, tenantId int64, sourceAliasKeyspace string) { - mtm.insertSomeData(t, tenantId, getSourceKeyspace(tenantId), numInitialRowsPerTenant) -} - -func getInitialTabletIdForTenant(tenantId int64) int { - return int(baseInitialTabletId + tenantId*tabletIdStep) -} - -func (mtm *multiTenantMigration) setup(tenantId int64) { - log.Infof("Creating MoveTables for tenant %d", tenantId) - mtm.setLastID(tenantId, 0) - sourceKeyspace := getSourceKeyspace(tenantId) - sourceAliasKeyspace := getSourceAliasKeyspace(tenantId) - _, err := vc.AddKeyspace(mtm.t, []*Cell{vc.Cells["zone1"]}, sourceKeyspace, "0", stVSchema, stSchema, - 1, 0, getInitialTabletIdForTenant(tenantId), nil) - require.NoError(mtm.t, err) - mtm.initTenantData(mtm.t, tenantId, sourceAliasKeyspace) -} - -func (mtm *multiTenantMigration) start(tenantId int64) { - sourceKeyspace := getSourceKeyspace(tenantId) - sourceAliasKeyspace := getSourceAliasKeyspace(tenantId) - mtm.setTenantMigrationStatus(tenantId, tenantMigrationStatusMigrating) - mt := newVtctldMoveTables(&moveTablesWorkflow{ - workflowInfo: &workflowInfo{ - vc: vc, - workflowName: fmt.Sprintf("wf%d", tenantId), - targetKeyspace: mtm.targetKeyspace, - }, - sourceKeyspace: sourceKeyspace, - tables: mtm.tables, - createFlags: []string{ - "--tenant-id", strconv.FormatInt(tenantId, 10), - "--source-keyspace-alias", sourceAliasKeyspace, - }, - }) - mtm.setActiveMoveTables(tenantId, mt) - mt.Create() -} - -func (mtm *multiTenantMigration) switchTraffic(tenantId int64) { - t := mtm.t - sourceAliasKeyspace := getSourceAliasKeyspace(tenantId) - sourceKeyspaceName := getSourceKeyspace(tenantId) - mt := mtm.getActiveMoveTables(tenantId) - ksWorkflow := fmt.Sprintf("%s.%s", mtm.targetKeyspace, mt.workflowName) - waitForWorkflowState(t, vc, ksWorkflow, binlogdatapb.VReplicationWorkflowState_Running.String()) - // we intentionally insert first into the source alias keyspace and then the source keyspace to test routing rules for both. - mtm.insertSomeData(t, tenantId, sourceAliasKeyspace, numAdditionalRowsPerTenant) - mt.SwitchReadsAndWrites() - mtm.insertSomeData(t, tenantId, sourceKeyspaceName, numAdditionalRowsPerTenant) -} - -func (mtm *multiTenantMigration) complete(tenantId int64) { - mt := mtm.getActiveMoveTables(tenantId) - mt.Complete() - vtgateConn := vc.GetVTGateConn(mtm.t) - defer vtgateConn.Close() - waitForQueryResult(mtm.t, vtgateConn, "", - fmt.Sprintf("select count(*) from %s.t1 where tenant_id=%d", mt.targetKeyspace, tenantId), - fmt.Sprintf("[[INT64(%d)]]", mtm.getLastID(tenantId))) -} - -func randomWait() { - time.Sleep(time.Duration(rand.IntN(maxRandomDelaySeconds)) * time.Second) -} - -func (mtm *multiTenantMigration) doThis(name string, chIn, chOut chan int64, counter *atomic.Int64, f func(int64)) { - timer := time.NewTimer(waitTimeout) - for counter.Load() < numTenants { - select { - case tenantId := <-chIn: - f(tenantId) - counter.Add(1) - chOut <- tenantId - timer.Reset(waitTimeout) - case <-timer.C: - require.FailNowf(mtm.t, "Timed out: %s", name) - } - randomWait() - } -} - -// run starts the migration process for all tenants. It starts concurrent -func (mtm *multiTenantMigration) run() { - go mtm.doThis("Setup tenant keyspace/schemas", chNotSetup, chNotCreated, &numSetup, mtm.setup) - for i := int64(1); i <= numTenants; i++ { - chNotSetup <- i - } - // Wait for all tenants to be created before starting the workflows: 10 seconds per tenant to account for CI overhead. - perTenantLoadTimeout := 1 * time.Minute - require.NoError(mtm.t, waitForCondition("All tenants created", - func() bool { - return numSetup.Load() == numTenants - }, perTenantLoadTimeout*numTenants)) - - go mtm.doThis("Start Migrations", chNotCreated, chInProgress, &numInProgress, mtm.start) - go mtm.doThis("Switch Traffic", chInProgress, chSwitched, &numSwitched, mtm.switchTraffic) - go mtm.doThis("Mark Migrations Complete", chSwitched, chCompleted, &numCompleted, mtm.complete) -} diff --git a/go/vt/proto/tabletmanagerdata/tabletmanagerdata.pb.go b/go/vt/proto/tabletmanagerdata/tabletmanagerdata.pb.go index 4b5625c5e4c..50bc4bd0e72 100644 --- a/go/vt/proto/tabletmanagerdata/tabletmanagerdata.pb.go +++ b/go/vt/proto/tabletmanagerdata/tabletmanagerdata.pb.go @@ -5827,149 +5827,6 @@ func (x *UpdateVReplicationWorkflowResponse) GetResult() *query.QueryResult { return nil } -<<<<<<< HEAD -======= -// UpdateVReplicationWorkflowsRequest is used to update multiple existing VReplication -// workflows. Note that the following fields MUST have an explicit value provided -// if you do NOT wish to update the existing values to the given type's ZeroValue: -// state, message, and stop_position. -// TODO: leverage the optional modifier for these fields rather than using SimulatedNull -// values: https://github.com/vitessio/vitess/issues/15627 -type UpdateVReplicationWorkflowsRequest struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - - AllWorkflows bool `protobuf:"varint,1,opt,name=all_workflows,json=allWorkflows,proto3" json:"all_workflows,omitempty"` - IncludeWorkflows []string `protobuf:"bytes,2,rep,name=include_workflows,json=includeWorkflows,proto3" json:"include_workflows,omitempty"` - ExcludeWorkflows []string `protobuf:"bytes,3,rep,name=exclude_workflows,json=excludeWorkflows,proto3" json:"exclude_workflows,omitempty"` - State binlogdata.VReplicationWorkflowState `protobuf:"varint,4,opt,name=state,proto3,enum=binlogdata.VReplicationWorkflowState" json:"state,omitempty"` - Message string `protobuf:"bytes,5,opt,name=message,proto3" json:"message,omitempty"` - StopPosition string `protobuf:"bytes,6,opt,name=stop_position,json=stopPosition,proto3" json:"stop_position,omitempty"` -} - -func (x *UpdateVReplicationWorkflowsRequest) Reset() { - *x = UpdateVReplicationWorkflowsRequest{} - if protoimpl.UnsafeEnabled { - mi := &file_tabletmanagerdata_proto_msgTypes[116] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } -} - -func (x *UpdateVReplicationWorkflowsRequest) String() string { - return protoimpl.X.MessageStringOf(x) -} - -func (*UpdateVReplicationWorkflowsRequest) ProtoMessage() {} - -func (x *UpdateVReplicationWorkflowsRequest) ProtoReflect() protoreflect.Message { - mi := &file_tabletmanagerdata_proto_msgTypes[116] - if protoimpl.UnsafeEnabled && x != nil { - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - if ms.LoadMessageInfo() == nil { - ms.StoreMessageInfo(mi) - } - return ms - } - return mi.MessageOf(x) -} - -// Deprecated: Use UpdateVReplicationWorkflowsRequest.ProtoReflect.Descriptor instead. -func (*UpdateVReplicationWorkflowsRequest) Descriptor() ([]byte, []int) { - return file_tabletmanagerdata_proto_rawDescGZIP(), []int{116} -} - -func (x *UpdateVReplicationWorkflowsRequest) GetAllWorkflows() bool { - if x != nil { - return x.AllWorkflows - } - return false -} - -func (x *UpdateVReplicationWorkflowsRequest) GetIncludeWorkflows() []string { - if x != nil { - return x.IncludeWorkflows - } - return nil -} - -func (x *UpdateVReplicationWorkflowsRequest) GetExcludeWorkflows() []string { - if x != nil { - return x.ExcludeWorkflows - } - return nil -} - -func (x *UpdateVReplicationWorkflowsRequest) GetState() binlogdata.VReplicationWorkflowState { - if x != nil { - return x.State - } - return binlogdata.VReplicationWorkflowState(0) -} - -func (x *UpdateVReplicationWorkflowsRequest) GetMessage() string { - if x != nil { - return x.Message - } - return "" -} - -func (x *UpdateVReplicationWorkflowsRequest) GetStopPosition() string { - if x != nil { - return x.StopPosition - } - return "" -} - -type UpdateVReplicationWorkflowsResponse struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - - Result *query.QueryResult `protobuf:"bytes,1,opt,name=result,proto3" json:"result,omitempty"` -} - -func (x *UpdateVReplicationWorkflowsResponse) Reset() { - *x = UpdateVReplicationWorkflowsResponse{} - if protoimpl.UnsafeEnabled { - mi := &file_tabletmanagerdata_proto_msgTypes[117] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } -} - -func (x *UpdateVReplicationWorkflowsResponse) String() string { - return protoimpl.X.MessageStringOf(x) -} - -func (*UpdateVReplicationWorkflowsResponse) ProtoMessage() {} - -func (x *UpdateVReplicationWorkflowsResponse) ProtoReflect() protoreflect.Message { - mi := &file_tabletmanagerdata_proto_msgTypes[117] - if protoimpl.UnsafeEnabled && x != nil { - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - if ms.LoadMessageInfo() == nil { - ms.StoreMessageInfo(mi) - } - return ms - } - return mi.MessageOf(x) -} - -// Deprecated: Use UpdateVReplicationWorkflowsResponse.ProtoReflect.Descriptor instead. -func (*UpdateVReplicationWorkflowsResponse) Descriptor() ([]byte, []int) { - return file_tabletmanagerdata_proto_rawDescGZIP(), []int{117} -} - -func (x *UpdateVReplicationWorkflowsResponse) GetResult() *query.QueryResult { - if x != nil { - return x.Result - } - return nil -} - ->>>>>>> 0e2f1751d4 (VReplication: Fix workflow update changed handling (#15621)) type ResetSequencesRequest struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -7030,7 +6887,7 @@ var file_tabletmanagerdata_proto_rawDesc = []byte{ 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, 0x81, 0x03, 0x0a, + 0x65, 0x70, 0x6f, 0x72, 0x74, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0xef, 0x02, 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, @@ -7053,214 +6910,41 @@ var file_tabletmanagerdata_proto_rawDesc = []byte{ 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, -<<<<<<< HEAD - 0x65, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x73, 0x68, 0x61, 0x72, - 0x64, 0x73, 0x18, 0x07, 0x20, 0x03, 0x28, 0x09, 0x52, 0x06, 0x73, 0x68, 0x61, 0x72, 0x64, 0x73, - 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, 0x32, 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, 0x22, 0xc8, 0x01, 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, 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, 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, -======= - 0x65, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, - 0x61, 0x67, 0x65, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, - 0x67, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x72, 0x6f, 0x77, 0x73, 0x5f, 0x63, 0x6f, 0x70, 0x69, 0x65, - 0x64, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, 0x72, 0x6f, 0x77, 0x73, 0x43, 0x6f, 0x70, - 0x69, 0x65, 0x64, 0x12, 0x33, 0x0a, 0x0e, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x68, 0x65, 0x61, 0x72, - 0x74, 0x62, 0x65, 0x61, 0x74, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x76, 0x74, - 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x52, 0x0d, 0x74, 0x69, 0x6d, 0x65, 0x48, - 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x12, 0x33, 0x0a, 0x0e, 0x74, 0x69, 0x6d, 0x65, - 0x5f, 0x74, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x64, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x0c, 0x2e, 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x52, 0x0d, - 0x74, 0x69, 0x6d, 0x65, 0x54, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x64, 0x12, 0x2f, 0x0a, - 0x13, 0x63, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x5f, 0x74, 0x68, 0x72, 0x6f, 0x74, - 0x74, 0x6c, 0x65, 0x64, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x09, 0x52, 0x12, 0x63, 0x6f, 0x6d, 0x70, - 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x54, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x64, 0x22, 0xd7, - 0x01, 0x0a, 0x0c, 0x56, 0x44, 0x69, 0x66, 0x66, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, - 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x77, - 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x77, - 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x12, 0x16, 0x0a, 0x06, 0x61, 0x63, 0x74, 0x69, 0x6f, - 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, - 0x1d, 0x0a, 0x0a, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x61, 0x72, 0x67, 0x18, 0x04, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x09, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x72, 0x67, 0x12, 0x1d, - 0x0a, 0x0a, 0x76, 0x64, 0x69, 0x66, 0x66, 0x5f, 0x75, 0x75, 0x69, 0x64, 0x18, 0x05, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x09, 0x76, 0x64, 0x69, 0x66, 0x66, 0x55, 0x75, 0x69, 0x64, 0x12, 0x39, 0x0a, - 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, - 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, - 0x74, 0x61, 0x2e, 0x56, 0x44, 0x69, 0x66, 0x66, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, - 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0x6a, 0x0a, 0x0d, 0x56, 0x44, 0x69, 0x66, - 0x66, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x02, 0x69, 0x64, 0x12, 0x2a, 0x0a, 0x06, 0x6f, 0x75, 0x74, - 0x70, 0x75, 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, 0x6f, - 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x1d, 0x0a, 0x0a, 0x76, 0x64, 0x69, 0x66, 0x66, 0x5f, 0x75, - 0x75, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x76, 0x64, 0x69, 0x66, 0x66, - 0x55, 0x75, 0x69, 0x64, 0x22, 0x79, 0x0a, 0x12, 0x56, 0x44, 0x69, 0x66, 0x66, 0x50, 0x69, 0x63, - 0x6b, 0x65, 0x72, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x21, 0x0a, 0x0c, 0x74, 0x61, - 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x54, 0x79, 0x70, 0x65, 0x73, 0x12, 0x1f, 0x0a, - 0x0b, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x63, 0x65, 0x6c, 0x6c, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x0a, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x43, 0x65, 0x6c, 0x6c, 0x12, 0x1f, - 0x0a, 0x0b, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x5f, 0x63, 0x65, 0x6c, 0x6c, 0x18, 0x03, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x0a, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x43, 0x65, 0x6c, 0x6c, 0x22, - 0x90, 0x01, 0x0a, 0x12, 0x56, 0x44, 0x69, 0x66, 0x66, 0x52, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x4f, - 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x19, 0x0a, 0x08, 0x6f, 0x6e, 0x6c, 0x79, 0x5f, 0x70, - 0x6b, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x6f, 0x6e, 0x6c, 0x79, 0x50, 0x6b, - 0x73, 0x12, 0x1f, 0x0a, 0x0b, 0x64, 0x65, 0x62, 0x75, 0x67, 0x5f, 0x71, 0x75, 0x65, 0x72, 0x79, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0a, 0x64, 0x65, 0x62, 0x75, 0x67, 0x51, 0x75, 0x65, - 0x72, 0x79, 0x12, 0x16, 0x0a, 0x06, 0x66, 0x6f, 0x72, 0x6d, 0x61, 0x74, 0x18, 0x03, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x06, 0x66, 0x6f, 0x72, 0x6d, 0x61, 0x74, 0x12, 0x26, 0x0a, 0x0f, 0x6d, 0x61, - 0x78, 0x5f, 0x73, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x04, 0x20, - 0x01, 0x28, 0x03, 0x52, 0x0d, 0x6d, 0x61, 0x78, 0x53, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x52, 0x6f, - 0x77, 0x73, 0x22, 0xda, 0x02, 0x0a, 0x10, 0x56, 0x44, 0x69, 0x66, 0x66, 0x43, 0x6f, 0x72, 0x65, - 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x16, 0x0a, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, - 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x12, - 0x1d, 0x0a, 0x0a, 0x61, 0x75, 0x74, 0x6f, 0x5f, 0x72, 0x65, 0x74, 0x72, 0x79, 0x18, 0x02, 0x20, - 0x01, 0x28, 0x08, 0x52, 0x09, 0x61, 0x75, 0x74, 0x6f, 0x52, 0x65, 0x74, 0x72, 0x79, 0x12, 0x19, - 0x0a, 0x08, 0x6d, 0x61, 0x78, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, - 0x52, 0x07, 0x6d, 0x61, 0x78, 0x52, 0x6f, 0x77, 0x73, 0x12, 0x1a, 0x0a, 0x08, 0x63, 0x68, 0x65, - 0x63, 0x6b, 0x73, 0x75, 0x6d, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, 0x63, 0x68, 0x65, - 0x63, 0x6b, 0x73, 0x75, 0x6d, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x5f, - 0x70, 0x63, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x73, 0x61, 0x6d, 0x70, 0x6c, - 0x65, 0x50, 0x63, 0x74, 0x12, 0x27, 0x0a, 0x0f, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x5f, - 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x06, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0e, 0x74, - 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x53, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x12, 0x38, 0x0a, - 0x19, 0x6d, 0x61, 0x78, 0x5f, 0x65, 0x78, 0x74, 0x72, 0x61, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x5f, - 0x74, 0x6f, 0x5f, 0x63, 0x6f, 0x6d, 0x70, 0x61, 0x72, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x03, - 0x52, 0x15, 0x6d, 0x61, 0x78, 0x45, 0x78, 0x74, 0x72, 0x61, 0x52, 0x6f, 0x77, 0x73, 0x54, 0x6f, - 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x72, 0x65, 0x12, 0x2c, 0x0a, 0x12, 0x75, 0x70, 0x64, 0x61, 0x74, - 0x65, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x73, 0x18, 0x08, 0x20, - 0x01, 0x28, 0x08, 0x52, 0x10, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, - 0x53, 0x74, 0x61, 0x74, 0x73, 0x12, 0x28, 0x0a, 0x10, 0x6d, 0x61, 0x78, 0x5f, 0x64, 0x69, 0x66, - 0x66, 0x5f, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x09, 0x20, 0x01, 0x28, 0x03, 0x52, - 0x0e, 0x6d, 0x61, 0x78, 0x44, 0x69, 0x66, 0x66, 0x53, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x22, - 0xf2, 0x01, 0x0a, 0x0c, 0x56, 0x44, 0x69, 0x66, 0x66, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, - 0x12, 0x4c, 0x0a, 0x0e, 0x70, 0x69, 0x63, 0x6b, 0x65, 0x72, 0x5f, 0x6f, 0x70, 0x74, 0x69, 0x6f, - 0x6e, 0x73, 0x18, 0x01, 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, 0x50, 0x69, 0x63, 0x6b, 0x65, 0x72, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, - 0x0d, 0x70, 0x69, 0x63, 0x6b, 0x65, 0x72, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x46, - 0x0a, 0x0c, 0x63, 0x6f, 0x72, 0x65, 0x5f, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x02, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, - 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x44, 0x69, 0x66, 0x66, 0x43, 0x6f, - 0x72, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x0b, 0x63, 0x6f, 0x72, 0x65, 0x4f, - 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x4c, 0x0a, 0x0e, 0x72, 0x65, 0x70, 0x6f, 0x72, 0x74, - 0x5f, 0x6f, 0x70, 0x74, 0x69, 0x6f, 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, 0xef, 0x02, 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, 0x6c, 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, 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, 0x12, 0x2e, 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, 0x52, 0x05, 0x6f, 0x6e, - 0x44, 0x64, 0x6c, 0x12, 0x3b, 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, 0x52, 0x05, 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, 0x9f, 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, 0x3b, - 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, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 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, 0x23, 0x0a, 0x0d, 0x73, 0x74, 0x6f, 0x70, 0x5f, 0x70, 0x6f, - 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x73, 0x74, - 0x6f, 0x70, 0x50, 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, 0x32, 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, 0x22, 0xc8, 0x01, 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, 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, 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, ->>>>>>> 0e2f1751d4 (VReplication: Fix workflow update changed handling (#15621)) + 0x65, 0x52, 0x05, 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, 0x32, 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, 0x22, + 0xc8, 0x01, 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, 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, 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 ( diff --git a/go/vt/vttablet/tabletmanager/rpc_vreplication.go b/go/vt/vttablet/tabletmanager/rpc_vreplication.go index aad18ef6a45..a9ce8b841a5 100644 --- a/go/vt/vttablet/tabletmanager/rpc_vreplication.go +++ b/go/vt/vttablet/tabletmanager/rpc_vreplication.go @@ -328,32 +328,6 @@ func (tm *TabletManager) UpdateVReplicationWorkflow(ctx context.Context, req *ta }, nil } -<<<<<<< HEAD -======= -// UpdateVReplicationWorkflows operates in much the same way that -// UpdateVReplicationWorkflow does, but it allows you to update the -// metadata/flow control fields -- state, message, and stop_pos -- for -// multiple workflows. -// Note: today this is only used during Reshard as all of the vreplication -// streams need to be migrated from the old shards to the new ones. -func (tm *TabletManager) UpdateVReplicationWorkflows(ctx context.Context, req *tabletmanagerdatapb.UpdateVReplicationWorkflowsRequest) (*tabletmanagerdatapb.UpdateVReplicationWorkflowsResponse, error) { - query, err := tm.buildUpdateVReplicationWorkflowsQuery(req) - if err != nil { - return nil, err - } - res, err := tm.VREngine.Exec(query) - if err != nil { - return nil, err - } - - return &tabletmanagerdatapb.UpdateVReplicationWorkflowsResponse{ - Result: &querypb.QueryResult{ - RowsAffected: res.RowsAffected, - }, - }, nil -} - ->>>>>>> 0e2f1751d4 (VReplication: Fix workflow update changed handling (#15621)) // VReplicationExec executes a vreplication command. func (tm *TabletManager) VReplicationExec(ctx context.Context, query string) (*querypb.QueryResult, error) { // Replace any provided sidecar database qualifiers with the correct one. diff --git a/proto/tabletmanagerdata.proto b/proto/tabletmanagerdata.proto index 6a925842087..f33cb4914ed 100644 --- a/proto/tabletmanagerdata.proto +++ b/proto/tabletmanagerdata.proto @@ -651,28 +651,6 @@ message UpdateVReplicationWorkflowResponse { query.QueryResult result = 1; } -<<<<<<< HEAD -======= -// UpdateVReplicationWorkflowsRequest is used to update multiple existing VReplication -// workflows. Note that the following fields MUST have an explicit value provided -// if you do NOT wish to update the existing values to the given type's ZeroValue: -// state, message, and stop_position. -// TODO: leverage the optional modifier for these fields rather than using SimulatedNull -// values: https://github.com/vitessio/vitess/issues/15627 -message UpdateVReplicationWorkflowsRequest { - bool all_workflows = 1; - repeated string include_workflows = 2; - repeated string exclude_workflows = 3; - binlogdata.VReplicationWorkflowState state = 4; - string message = 5; - string stop_position = 6; -} - -message UpdateVReplicationWorkflowsResponse { - query.QueryResult result = 1; -} - ->>>>>>> 0e2f1751d4 (VReplication: Fix workflow update changed handling (#15621)) message ResetSequencesRequest { repeated string tables = 1; }