From 99d39f9b1347d7a19a3db14d8b9650fb95de2fe6 Mon Sep 17 00:00:00 2001 From: Manan Gupta <35839558+GuptaManan100@users.noreply.github.com> Date: Wed, 5 Jul 2023 18:50:09 +0530 Subject: [PATCH] [release-16.0] Upgrade-Downgrade Fix: Schema-initialization stuck on semi-sync ACKs while upgrading (#13411) (#13441) --- .../upgrade_downgrade_test_backups_manual.yml | 12 +- ...grade_test_backups_manual_next_release.yml | 12 +- examples/backups/restart_tablets.sh | 6 +- examples/backups/start_cluster.sh | 10 +- examples/backups/upgrade_cluster.sh | 97 ++++ examples/common/scripts/vttablet-up.sh | 1 - .../vtctl/reparentutil/planned_reparenter.go | 86 ++- .../planned_reparenter_flaky_test.go | 511 ++++++------------ .../testlib/planned_reparent_shard_test.go | 45 +- 9 files changed, 347 insertions(+), 433 deletions(-) create mode 100755 examples/backups/upgrade_cluster.sh diff --git a/.github/workflows/upgrade_downgrade_test_backups_manual.yml b/.github/workflows/upgrade_downgrade_test_backups_manual.yml index cefa6833d2a..079943f1925 100644 --- a/.github/workflows/upgrade_downgrade_test_backups_manual.yml +++ b/.github/workflows/upgrade_downgrade_test_backups_manual.yml @@ -269,14 +269,6 @@ jobs: source build.env ; cd examples/backups ./take_backups.sh - # Stopping the tablets so we can perform the upgrade. - - name: Stop tablets - if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - timeout-minutes: 10 - run: | - source build.env ; cd examples/backups - ./stop_tablets.sh - # We upgrade: we swap binaries and use the version N of the tablet. - name: Upgrade - Swap binaries, use VTTablet N if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' @@ -293,9 +285,7 @@ jobs: timeout-minutes: 10 run: | source build.env ; cd examples/backups - ./restart_tablets.sh - # give enough time to the tablets to restore the backup - sleep 90 + ./upgrade_cluster.sh # We count the number of rows in every table to check that the restore step was successful. - name: Assert the number of rows in every table diff --git a/.github/workflows/upgrade_downgrade_test_backups_manual_next_release.yml b/.github/workflows/upgrade_downgrade_test_backups_manual_next_release.yml index 9cd64760afc..7148834efe7 100644 --- a/.github/workflows/upgrade_downgrade_test_backups_manual_next_release.yml +++ b/.github/workflows/upgrade_downgrade_test_backups_manual_next_release.yml @@ -272,14 +272,6 @@ jobs: source build.env ; cd examples/backups ./take_backups.sh - # Stopping the tablets so we can perform the upgrade. - - name: Stop tablets - if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - timeout-minutes: 10 - run: | - source build.env ; cd examples/backups - ./stop_tablets.sh - # We upgrade: we swap binaries and use the version N of the tablet. - name: Upgrade - Swap binaries, use VTTablet N if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' @@ -296,9 +288,7 @@ jobs: timeout-minutes: 10 run: | source build.env ; cd examples/backups - ./restart_tablets.sh - # give enough time to the tablets to restore the backup - sleep 90 + ./upgrade_cluster.sh # We count the number of rows in every table to check that the restore step was successful. - name: Assert the number of rows in every table diff --git a/examples/backups/restart_tablets.sh b/examples/backups/restart_tablets.sh index 13c4ed0cfb6..bfafcf26d4f 100755 --- a/examples/backups/restart_tablets.sh +++ b/examples/backups/restart_tablets.sh @@ -58,6 +58,6 @@ for i in 101 201 301; do exit 1 done -vtctldclient InitShardPrimary --force commerce/0 zone1-100 -vtctldclient InitShardPrimary --force customer/-80 zone1-200 -vtctldclient InitShardPrimary --force customer/80- zone1-300 +vtctldclient PlannedReparentShard commerce/0 --new-primary "zone1-100" +vtctldclient PlannedReparentShard customer/-80 --new-primary "zone1-200" +vtctldclient PlannedReparentShard customer/80- --new-primary "zone1-300" diff --git a/examples/backups/start_cluster.sh b/examples/backups/start_cluster.sh index a9e2de606cb..9855171ea4d 100755 --- a/examples/backups/start_cluster.sh +++ b/examples/backups/start_cluster.sh @@ -31,6 +31,8 @@ fi # start vtctld CELL=zone1 ../common/scripts/vtctld-up.sh +# Create keyspace and set the semi_sync durability policy. +vtctldclient CreateKeyspace --durability-policy=semi_sync commerce || fail "Failed to create and configure the commerce keyspace" # start vttablets for keyspace commerce for i in 100 101 102; do @@ -39,12 +41,14 @@ for i in 100 101 102; do done # set one of the replicas to primary -vtctldclient InitShardPrimary --force commerce/0 zone1-100 +vtctldclient PlannedReparentShard commerce/0 --new-primary "zone1-100" # create the schema for commerce vtctlclient ApplySchema -- --sql-file ./create_commerce_schema.sql commerce || fail "Could not apply schema for the commerce keyspace" vtctlclient ApplyVSchema -- --vschema_file ../local/vschema_commerce_seq.json commerce || fail "Could not apply vschema for the commerce keyspace" +# Create keyspace and set the semi_sync durability policy. +vtctldclient CreateKeyspace --durability-policy=semi_sync customer || fail "Failed to create and configure the customer keyspace" # start vttablets for keyspace customer for i in 200 201 202; do @@ -57,8 +61,8 @@ for i in 300 301 302; do done # set one of the replicas to primary -vtctldclient InitShardPrimary --force customer/-80 zone1-200 -vtctldclient InitShardPrimary --force customer/80- zone1-300 +vtctldclient PlannedReparentShard customer/-80 --new-primary "zone1-200" +vtctldclient PlannedReparentShard customer/80- --new-primary "zone1-300" for shard in "-80" "80-"; do wait_for_healthy_shard customer "${shard}" || exit 1 diff --git a/examples/backups/upgrade_cluster.sh b/examples/backups/upgrade_cluster.sh new file mode 100755 index 00000000000..0144dc94579 --- /dev/null +++ b/examples/backups/upgrade_cluster.sh @@ -0,0 +1,97 @@ +#!/bin/bash + +# Copyright 2023 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. + +# this script brings up new tablets for the two new shards that we will +# be creating in the customer keyspace and copies the schema + +source ../common/env.sh + +# Restart the replica tablets so that they come up with new vttablet versions +for i in 101 102; do + echo "Shutting down tablet zone1-$i" + CELL=zone1 TABLET_UID=$i ../common/scripts/vttablet-down.sh + echo "Shutting down mysql zone1-$i" + CELL=zone1 TABLET_UID=$i ../common/scripts/mysqlctl-down.sh + echo "Removing tablet directory zone1-$i" + vtctlclient DeleteTablet -- --allow_primary=true zone1-$i + rm -Rf $VTDATAROOT/vt_0000000$i + echo "Starting tablet zone1-$i again" + CELL=zone1 TABLET_UID=$i ../common/scripts/mysqlctl-up.sh + CELL=zone1 KEYSPACE=commerce TABLET_UID=$i ../common/scripts/vttablet-up.sh +done + +for i in 201 202; do + echo "Shutting down tablet zone1-$i" + CELL=zone1 TABLET_UID=$i ../common/scripts/vttablet-down.sh + echo "Shutting down mysql zone1-$i" + CELL=zone1 TABLET_UID=$i ../common/scripts/mysqlctl-down.sh + echo "Removing tablet directory zone1-$i" + vtctlclient DeleteTablet -- --allow_primary=true zone1-$i + rm -Rf $VTDATAROOT/vt_0000000$i + echo "Starting tablet zone1-$i again" + CELL=zone1 TABLET_UID=$i ../common/scripts/mysqlctl-up.sh + SHARD=-80 CELL=zone1 KEYSPACE=customer TABLET_UID=$i ../common/scripts/vttablet-up.sh +done + +for i in 301 302; do + echo "Shutting down tablet zone1-$i" + CELL=zone1 TABLET_UID=$i ../common/scripts/vttablet-down.sh + echo "Shutting down mysql zone1-$i" + CELL=zone1 TABLET_UID=$i ../common/scripts/mysqlctl-down.sh + echo "Removing tablet directory zone1-$i" + vtctlclient DeleteTablet -- --allow_primary=true zone1-$i + rm -Rf $VTDATAROOT/vt_0000000$i + echo "Starting tablet zone1-$i again" + CELL=zone1 TABLET_UID=$i ../common/scripts/mysqlctl-up.sh + SHARD=80- CELL=zone1 KEYSPACE=customer TABLET_UID=$i ../common/scripts/vttablet-up.sh +done + +# Wait for all the replica tablets to be in the serving state before reparenting to them. +totalTime=600 +for i in 101 201 301; do + while [ $totalTime -gt 0 ]; do + status=$(curl "http://$hostname:15$i/debug/status_details") + echo "$status" | grep "REPLICA: Serving" && break + totalTime=$((totalTime-1)) + sleep 0.1 + done +done + +# Check that all the replica tablets have reached REPLICA: Serving state +for i in 101 201 301; do + status=$(curl "http://$hostname:15$i/debug/status_details") + echo "$status" | grep "REPLICA: Serving" && continue + echo "tablet-$i did not reach REPLICA: Serving state. Exiting due to failure." + exit 1 +done + +# Promote the replica tablets to primary +vtctldclient PlannedReparentShard commerce/0 --new-primary "zone1-101" +vtctldclient PlannedReparentShard customer/-80 --new-primary "zone1-201" +vtctldclient PlannedReparentShard customer/80- --new-primary "zone1-301" + +# Restart the old primary tablets so that they are on the latest version of vttablet too. +echo "Restarting tablet zone1-100" +CELL=zone1 TABLET_UID=100 ../common/scripts/vttablet-down.sh +CELL=zone1 KEYSPACE=commerce TABLET_UID=100 ../common/scripts/vttablet-up.sh + +echo "Restarting tablet zone1-200" +CELL=zone1 TABLET_UID=200 ../common/scripts/vttablet-down.sh +SHARD=-80 CELL=zone1 KEYSPACE=customer TABLET_UID=200 ../common/scripts/vttablet-up.sh + +echo "Restarting tablet zone1-300" +CELL=zone1 TABLET_UID=300 ../common/scripts/vttablet-down.sh +SHARD=80- CELL=zone1 KEYSPACE=customer TABLET_UID=300 ../common/scripts/vttablet-up.sh \ No newline at end of file diff --git a/examples/common/scripts/vttablet-up.sh b/examples/common/scripts/vttablet-up.sh index 0e70837d235..21c15b8d547 100755 --- a/examples/common/scripts/vttablet-up.sh +++ b/examples/common/scripts/vttablet-up.sh @@ -55,7 +55,6 @@ vttablet \ --service_map 'grpc-queryservice,grpc-tabletmanager,grpc-updatestream' \ --pid_file $VTDATAROOT/$tablet_dir/vttablet.pid \ --vtctld_addr http://$hostname:$vtctld_web_port/ \ - --disable_active_reparents \ > $VTDATAROOT/$tablet_dir/vttablet.out 2>&1 & # Block waiting for the tablet to be listening diff --git a/go/vt/vtctl/reparentutil/planned_reparenter.go b/go/vt/vtctl/reparentutil/planned_reparenter.go index fc0e1c80a06..8633afa13d0 100644 --- a/go/vt/vtctl/reparentutil/planned_reparenter.go +++ b/go/vt/vtctl/reparentutil/planned_reparenter.go @@ -213,7 +213,7 @@ func (pr *PlannedReparenter) performGracefulPromotion( primaryElect *topodatapb.Tablet, tabletMap map[string]*topo.TabletInfo, opts PlannedReparentOptions, -) (string, error) { +) error { primaryElectAliasStr := topoproto.TabletAliasString(primaryElect.Alias) ev.OldPrimary = proto.Clone(currentPrimary.Tablet).(*topodatapb.Tablet) @@ -231,7 +231,7 @@ func (pr *PlannedReparenter) performGracefulPromotion( snapshotPos, err := pr.tmc.PrimaryPosition(snapshotCtx, currentPrimary.Tablet) if err != nil { - return "", vterrors.Wrapf(err, "cannot get replication position on current primary %v; current primary must be healthy to perform PlannedReparent", currentPrimary.AliasString()) + return vterrors.Wrapf(err, "cannot get replication position on current primary %v; current primary must be healthy to perform PlannedReparent", currentPrimary.AliasString()) } // Next, we wait for the primary-elect to catch up to that snapshot point. @@ -246,12 +246,12 @@ func (pr *PlannedReparenter) performGracefulPromotion( defer setSourceCancel() if err := pr.tmc.SetReplicationSource(setSourceCtx, primaryElect, currentPrimary.Alias, 0, snapshotPos, true, IsReplicaSemiSync(opts.durability, currentPrimary.Tablet, primaryElect)); err != nil { - return "", vterrors.Wrapf(err, "replication on primary-elect %v did not catch up in time; replication must be healthy to perform PlannedReparent", primaryElectAliasStr) + return vterrors.Wrapf(err, "replication on primary-elect %v did not catch up in time; replication must be healthy to perform PlannedReparent", primaryElectAliasStr) } // Verify we still have the topology lock before doing the demotion. if err := topo.CheckShardLocked(ctx, keyspace, shard); err != nil { - return "", vterrors.Wrap(err, "lost topology lock; aborting") + return vterrors.Wrap(err, "lost topology lock; aborting") } // Next up, demote the current primary and get its replication position. @@ -265,7 +265,7 @@ func (pr *PlannedReparenter) performGracefulPromotion( primaryStatus, err := pr.tmc.DemotePrimary(demoteCtx, currentPrimary.Tablet) if err != nil { - return "", vterrors.Wrapf(err, "failed to DemotePrimary on current primary %v: %v", currentPrimary.AliasString(), err) + return vterrors.Wrapf(err, "failed to DemotePrimary on current primary %v: %v", currentPrimary.AliasString(), err) } // Wait for the primary-elect to catch up to the position we demoted the @@ -298,26 +298,10 @@ func (pr *PlannedReparenter) performGracefulPromotion( finalWaitErr = vterrors.Wrapf(finalWaitErr, "encountered error while performing UndoDemotePrimary(%v): %v", currentPrimary.AliasString(), undoErr) } - return "", finalWaitErr - } - - // Primary-elect is caught up to the current primary. We can do the - // promotion now. - promoteCtx, promoteCancel := context.WithTimeout(ctx, opts.WaitReplicasTimeout) - defer promoteCancel() - - rp, err := pr.tmc.PromoteReplica(promoteCtx, primaryElect, SemiSyncAckers(opts.durability, primaryElect) > 0) - if err != nil { - return "", vterrors.Wrapf(err, "primary-elect tablet %v failed to be promoted to primary; please try again", primaryElectAliasStr) - } - - if ctx.Err() == context.DeadlineExceeded { - // PromoteReplica succeeded, but we ran out of time. PRS needs to be - // re-run to complete fully. - return "", vterrors.Errorf(vtrpc.Code_DEADLINE_EXCEEDED, "PLannedReparent timed out after successfully promoting primary-elect %v; please re-run to fix up the replicas", primaryElectAliasStr) + return finalWaitErr } - return rp, nil + return nil } func (pr *PlannedReparenter) performInitialPromotion( @@ -383,7 +367,7 @@ func (pr *PlannedReparenter) performPotentialPromotion( primaryElect *topodatapb.Tablet, tabletMap map[string]*topo.TabletInfo, opts PlannedReparentOptions, -) (string, error) { +) error { primaryElectAliasStr := topoproto.TabletAliasString(primaryElect.Alias) pr.logger.Infof("no clear winner found for current primary term; checking if it's safe to recover by electing %v", primaryElectAliasStr) @@ -457,7 +441,7 @@ func (pr *PlannedReparenter) performPotentialPromotion( close(positions) if rec.HasErrors() { - return "", vterrors.Wrap(rec.Error(), "failed to demote all tablets") + return vterrors.Wrap(rec.Error(), "failed to demote all tablets") } // Construct a mapping of alias to tablet position. @@ -478,7 +462,7 @@ func (pr *PlannedReparenter) performPotentialPromotion( // if the candidate primary is behind that tablet. tp, ok := tabletPosMap[primaryElectAliasStr] if !ok { - return "", vterrors.Errorf(vtrpc.Code_FAILED_PRECONDITION, "primary-elect tablet %v not found in tablet map", primaryElectAliasStr) + return vterrors.Errorf(vtrpc.Code_FAILED_PRECONDITION, "primary-elect tablet %v not found in tablet map", primaryElectAliasStr) } primaryElectPos := tp.pos @@ -487,7 +471,7 @@ func (pr *PlannedReparenter) performPotentialPromotion( // The primary-elect pos has to be at least as advanced as every tablet // in the shard. if !primaryElectPos.AtLeast(tp.pos) { - return "", vterrors.Errorf( + return vterrors.Errorf( vtrpc.Code_FAILED_PRECONDITION, "tablet %v (position: %v) contains transactions not found in primary-elect %v (position: %v)", tp.alias, tp.pos, primaryElectAliasStr, primaryElectPos, @@ -497,19 +481,9 @@ func (pr *PlannedReparenter) performPotentialPromotion( // Check that we still have the topology lock. if err := topo.CheckShardLocked(ctx, keyspace, shard); err != nil { - return "", vterrors.Wrap(err, "lost topology lock; aborting") - } - - // Promote the candidate primary to type:PRIMARY. - promoteCtx, promoteCancel := context.WithTimeout(ctx, topo.RemoteOperationTimeout) - defer promoteCancel() - - rp, err := pr.tmc.PromoteReplica(promoteCtx, primaryElect, SemiSyncAckers(opts.durability, primaryElect) > 0) - if err != nil { - return "", vterrors.Wrapf(err, "failed to promote %v to primary", primaryElectAliasStr) + return vterrors.Wrap(err, "lost topology lock; aborting") } - - return rp, nil + return nil } func (pr *PlannedReparenter) reparentShardLocked( @@ -553,6 +527,11 @@ func (pr *PlannedReparenter) reparentShardLocked( currentPrimary := FindCurrentPrimary(tabletMap, pr.logger) reparentJournalPos := "" + // promoteReplicaRequired is a boolean that is used to store whether we need to call + // `PromoteReplica` when we reparent the tablets. This is required to be done when we are doing + // a potential or a graceful promotion. + // InitialPromotion calls `InitPrimary` and for partial promotion, the tablet is already a primary. + promoteReplicaRequired := false // needsRefresh is used to keep track of whether we need to refresh the state // of the new primary tablet. The only case that we need to reload the state // is when we are initializing the new primary. The reason is that the first @@ -601,7 +580,9 @@ func (pr *PlannedReparenter) reparentShardLocked( case currentPrimary == nil && ev.ShardInfo.PrimaryAlias != nil: // Case (2): no clear current primary. Try to find a safe promotion // candidate, and promote to it. - reparentJournalPos, err = pr.performPotentialPromotion(ctx, keyspace, shard, ev.NewPrimary, tabletMap, opts) + err = pr.performPotentialPromotion(ctx, keyspace, shard, ev.NewPrimary, tabletMap, opts) + // We need to call `PromoteReplica` when we reparent the tablets. + promoteReplicaRequired = true case topoproto.TabletAliasEqual(currentPrimary.Alias, opts.NewPrimaryAlias): // Case (3): desired new primary is the current primary. Attempt to fix // up replicas to recover from a previous partial promotion. @@ -609,7 +590,9 @@ func (pr *PlannedReparenter) reparentShardLocked( default: // Case (4): desired primary and current primary differ. Do a graceful // demotion-then-promotion. - reparentJournalPos, err = pr.performGracefulPromotion(ctx, ev, keyspace, shard, currentPrimary, ev.NewPrimary, tabletMap, opts) + err = pr.performGracefulPromotion(ctx, ev, keyspace, shard, currentPrimary, ev.NewPrimary, tabletMap, opts) + // We need to call `PromoteReplica` when we reparent the tablets. + promoteReplicaRequired = true } if err != nil { @@ -620,7 +603,7 @@ func (pr *PlannedReparenter) reparentShardLocked( return vterrors.Wrap(err, "lost topology lock, aborting") } - if err := pr.reparentTablets(ctx, ev, reparentJournalPos, tabletMap, opts); err != nil { + if err := pr.reparentTablets(ctx, ev, reparentJournalPos, promoteReplicaRequired, tabletMap, opts); err != nil { return err } @@ -637,6 +620,7 @@ func (pr *PlannedReparenter) reparentTablets( ctx context.Context, ev *events.Reparent, reparentJournalPosition string, + promoteReplicaRequired bool, tabletMap map[string]*topo.TabletInfo, opts PlannedReparentOptions, ) error { @@ -645,7 +629,7 @@ func (pr *PlannedReparenter) reparentTablets( replCtx, replCancel := context.WithTimeout(ctx, opts.WaitReplicasTimeout) defer replCancel() - // Go thorugh all the tablets. + // Go through all the tablets. // - New primary: populate the reparent journal. // - Everybody else: reparent to the new primary; wait for the reparent // journal row. @@ -660,7 +644,7 @@ func (pr *PlannedReparenter) reparentTablets( // Point all replicas at the new primary and check that they receive the // reparent journal entry, proving that they are replicating from the new - // primary. We do this concurrently with adding the journal entry (after + // primary. We do this concurrently with adding the journal entry (after // this loop), because if semi-sync is enabled, the update to the journal // table will block until at least one replica is successfully attached to // the new primary. @@ -688,6 +672,20 @@ func (pr *PlannedReparenter) reparentTablets( }(alias, tabletInfo.Tablet) } + // If `PromoteReplica` call is required, we should call it and use the position that it returns. + if promoteReplicaRequired { + // Promote the candidate primary to type:PRIMARY. + primaryPosition, err := pr.tmc.PromoteReplica(replCtx, ev.NewPrimary, SemiSyncAckers(opts.durability, ev.NewPrimary) > 0) + if err != nil { + pr.logger.Warningf("primary %v failed to PromoteReplica; cancelling replica reparent attempts", primaryElectAliasStr) + replCancel() + replicasWg.Wait() + + return vterrors.Wrapf(err, "failed PromoteReplica(primary=%v, ts=%v): %v", primaryElectAliasStr, reparentJournalTimestamp, err) + } + reparentJournalPosition = primaryPosition + } + // Add a reparent journal entry on the new primary. If semi-sync is enabled, // this blocks until at least one replica is reparented (above) and // successfully replicating from the new primary. diff --git a/go/vt/vtctl/reparentutil/planned_reparenter_flaky_test.go b/go/vt/vtctl/reparentutil/planned_reparenter_flaky_test.go index 5c79caeadb7..20815db3dfc 100644 --- a/go/vt/vtctl/reparentutil/planned_reparenter_flaky_test.go +++ b/go/vt/vtctl/reparentutil/planned_reparenter_flaky_test.go @@ -18,6 +18,7 @@ package reparentutil import ( "context" + "errors" "fmt" "strings" "testing" @@ -963,13 +964,12 @@ func TestPlannedReparenter_performGracefulPromotion(t *testing.T) { tabletMap map[string]*topo.TabletInfo opts PlannedReparentOptions - expectedPos string expectedEvent *events.Reparent shouldErr bool // Optional function to run some additional post-test assertions. Will // be run in the main test body before the common assertions are run, // regardless of the value of tt.shouldErr for that test case. - extraAssertions func(t *testing.T, pos string, err error) + extraAssertions func(t *testing.T, err error) }{ { name: "successful promotion", @@ -998,15 +998,6 @@ func TestPlannedReparenter_performGracefulPromotion(t *testing.T) { Position: "MySQL56/3E11FA47-71CA-11E1-9E33-C80AA9429562:1-10", }, }, - PromoteReplicaResults: map[string]struct { - Result string - Error error - }{ - "zone1-0000000200": { - Result: "successful reparent journal position", - Error: nil, - }, - }, SetReplicationSourceResults: map[string]error{ "zone1-0000000200": nil, }, @@ -1033,10 +1024,9 @@ func TestPlannedReparenter_performGracefulPromotion(t *testing.T) { Uid: 200, }, }, - tabletMap: map[string]*topo.TabletInfo{}, - opts: PlannedReparentOptions{}, - expectedPos: "successful reparent journal position", - shouldErr: false, + tabletMap: map[string]*topo.TabletInfo{}, + opts: PlannedReparentOptions{}, + shouldErr: false, }, { name: "cannot get snapshot of current primary", @@ -1376,20 +1366,6 @@ func TestPlannedReparenter_performGracefulPromotion(t *testing.T) { Position: "MySQL56/3E11FA47-71CA-11E1-9E33-C80AA9429562:1-10", }, }, - PromoteReplicaResults: map[string]struct { - Result string - Error error - }{ - // This being present means that if we don't encounter a - // a case where either WaitForPosition errors, or the parent - // context times out, then we will fail the test, since it - // will cause the overall function under test to return no - // error. - "zone1-0000000200": { - Result: "success!", - Error: nil, - }, - }, SetReplicationSourceResults: map[string]error{ "zone1-0000000200": nil, }, @@ -1483,7 +1459,7 @@ func TestPlannedReparenter_performGracefulPromotion(t *testing.T) { tabletMap: map[string]*topo.TabletInfo{}, opts: PlannedReparentOptions{}, shouldErr: true, - extraAssertions: func(t *testing.T, pos string, err error) { + extraAssertions: func(t *testing.T, err error) { assert.Contains(t, err.Error(), "UndoDemotePrimary", "expected error to include information about failed demotion rollback") }, }, @@ -1546,144 +1522,10 @@ func TestPlannedReparenter_performGracefulPromotion(t *testing.T) { tabletMap: map[string]*topo.TabletInfo{}, opts: PlannedReparentOptions{}, shouldErr: true, - extraAssertions: func(t *testing.T, pos string, err error) { + extraAssertions: func(t *testing.T, err error) { assert.NotContains(t, err.Error(), "UndoDemotePrimary", "expected error to not include information about failed demotion rollback") }, }, - { - name: "primary-elect fails to promote", - ts: memorytopo.NewServer("zone1"), - tmc: &testutil.TabletManagerClient{ - DemotePrimaryResults: map[string]struct { - Status *replicationdatapb.PrimaryStatus - Error error - }{ - "zone1-0000000100": { - Status: &replicationdatapb.PrimaryStatus{ - // value of Position doesn't strictly matter for - // this test case, as long as it matches the inner - // key of the WaitForPositionResults map for the - // primary-elect. - Position: "position1", - }, - Error: nil, - }, - }, - PrimaryPositionResults: map[string]struct { - Position string - Error error - }{ - "zone1-0000000100": { - Position: "MySQL56/3E11FA47-71CA-11E1-9E33-C80AA9429562:1-10", - }, - }, - PromoteReplicaResults: map[string]struct { - Result string - Error error - }{ - "zone1-0000000200": { - Error: assert.AnError, - }, - }, - SetReplicationSourceResults: map[string]error{ - "zone1-0000000200": nil, - }, - WaitForPositionResults: map[string]map[string]error{ - "zone1-0000000200": { - "position1": nil, - }, - }, - }, - ev: &events.Reparent{}, - keyspace: "testkeyspace", - shard: "-", - currentPrimary: &topo.TabletInfo{ - Tablet: &topodatapb.Tablet{ - Alias: &topodatapb.TabletAlias{ - Cell: "zone1", - Uid: 100, - }, - }, - }, - primaryElect: &topodatapb.Tablet{ - Alias: &topodatapb.TabletAlias{ - Cell: "zone1", - Uid: 200, - }, - }, - tabletMap: map[string]*topo.TabletInfo{}, - opts: PlannedReparentOptions{}, - shouldErr: true, - }, - { - name: "promotion succeeds but parent context times out", - ts: memorytopo.NewServer("zone1"), - tmc: &testutil.TabletManagerClient{ - DemotePrimaryResults: map[string]struct { - Status *replicationdatapb.PrimaryStatus - Error error - }{ - "zone1-0000000100": { - Status: &replicationdatapb.PrimaryStatus{ - // value of Position doesn't strictly matter for - // this test case, as long as it matches the inner - // key of the WaitForPositionResults map for the - // primary-elect. - Position: "position1", - }, - Error: nil, - }, - }, - PrimaryPositionResults: map[string]struct { - Position string - Error error - }{ - "zone1-0000000100": { - Position: "MySQL56/3E11FA47-71CA-11E1-9E33-C80AA9429562:1-10", - }, - }, - PromoteReplicaPostDelays: map[string]time.Duration{ - "zone1-0000000200": time.Millisecond * 100, // 10x the parent context timeout - }, - PromoteReplicaResults: map[string]struct { - Result string - Error error - }{ - "zone1-0000000200": { - Error: nil, - }, - }, - SetReplicationSourceResults: map[string]error{ - "zone1-0000000200": nil, - }, - WaitForPositionResults: map[string]map[string]error{ - "zone1-0000000200": { - "position1": nil, - }, - }, - }, - ctxTimeout: time.Millisecond * 10, - ev: &events.Reparent{}, - keyspace: "testkeyspace", - shard: "-", - currentPrimary: &topo.TabletInfo{ - Tablet: &topodatapb.Tablet{ - Alias: &topodatapb.TabletAlias{ - Cell: "zone1", - Uid: 100, - }, - }, - }, - primaryElect: &topodatapb.Tablet{ - Alias: &topodatapb.TabletAlias{ - Cell: "zone1", - Uid: 200, - }, - }, - tabletMap: map[string]*topo.TabletInfo{}, - opts: PlannedReparentOptions{}, - shouldErr: true, - }, } ctx := context.Background() @@ -1727,7 +1569,7 @@ func TestPlannedReparenter_performGracefulPromotion(t *testing.T) { require.NoError(t, err) tt.opts.durability = durability - pos, err := pr.performGracefulPromotion( + err = pr.performGracefulPromotion( ctx, tt.ev, tt.keyspace, @@ -1739,7 +1581,7 @@ func TestPlannedReparenter_performGracefulPromotion(t *testing.T) { ) if tt.extraAssertions != nil { - tt.extraAssertions(t, pos, err) + tt.extraAssertions(t, err) } if tt.shouldErr { @@ -1749,7 +1591,6 @@ func TestPlannedReparenter_performGracefulPromotion(t *testing.T) { } assert.NoError(t, err) - assert.Equal(t, tt.expectedPos, pos) }) } } @@ -2066,8 +1907,7 @@ func TestPlannedReparenter_performPotentialPromotion(t *testing.T) { primaryElect *topodatapb.Tablet tabletMap map[string]*topo.TabletInfo - expectedPos string - shouldErr bool + shouldErr bool }{ { name: "success", @@ -2096,15 +1936,6 @@ func TestPlannedReparenter_performPotentialPromotion(t *testing.T) { Error: nil, }, }, - PromoteReplicaResults: map[string]struct { - Result string - Error error - }{ - "zone1-0000000100": { - Result: "reparent journal position", - Error: nil, - }, - }, }, unlockTopo: false, keyspace: "testkeyspace", @@ -2141,8 +1972,7 @@ func TestPlannedReparenter_performPotentialPromotion(t *testing.T) { }, }, }, - expectedPos: "reparent journal position", - shouldErr: false, + shouldErr: false, }, { name: "failed to DemotePrimary on a tablet", @@ -2403,158 +2233,6 @@ func TestPlannedReparenter_performPotentialPromotion(t *testing.T) { }, shouldErr: true, }, - { - name: "failed to promote primary-elect", - ts: memorytopo.NewServer("zone1"), - tmc: &testutil.TabletManagerClient{ - DemotePrimaryResults: map[string]struct { - Status *replicationdatapb.PrimaryStatus - Error error - }{ - "zone1-0000000100": { - Status: &replicationdatapb.PrimaryStatus{ - Position: "MySQL56/3E11FA47-71CA-11E1-9E33-C80AA9429562:1-10", - }, - Error: nil, - }, - "zone1-0000000101": { - Status: &replicationdatapb.PrimaryStatus{ - Position: "MySQL56/3E11FA47-71CA-11E1-9E33-C80AA9429562:1-10", - }, - Error: nil, - }, - "zone1-0000000102": { - Status: &replicationdatapb.PrimaryStatus{ - Position: "MySQL56/3E11FA47-71CA-11E1-9E33-C80AA9429562:1-5", - }, - Error: nil, - }, - }, - PromoteReplicaResults: map[string]struct { - Result string - Error error - }{ - "zone1-0000000100": { - Result: "", - Error: assert.AnError, - }, - }, - }, - unlockTopo: false, - keyspace: "testkeyspace", - shard: "-", - primaryElect: &topodatapb.Tablet{ - Alias: &topodatapb.TabletAlias{ - Cell: "zone1", - Uid: 100, - }, - }, - tabletMap: map[string]*topo.TabletInfo{ - "zone1-0000000100": { - Tablet: &topodatapb.Tablet{ - Alias: &topodatapb.TabletAlias{ - Cell: "zone1", - Uid: 100, - }, - }, - }, - "zone1-0000000101": { - Tablet: &topodatapb.Tablet{ - Alias: &topodatapb.TabletAlias{ - Cell: "zone1", - Uid: 101, - }, - }, - }, - "zone1-0000000102": { - Tablet: &topodatapb.Tablet{ - Alias: &topodatapb.TabletAlias{ - Cell: "zone1", - Uid: 102, - }, - }, - }, - }, - shouldErr: true, - }, - { - name: "timed out while promoting primary-elect", - ts: memorytopo.NewServer("zone1"), - tmc: &testutil.TabletManagerClient{ - DemotePrimaryResults: map[string]struct { - Status *replicationdatapb.PrimaryStatus - Error error - }{ - "zone1-0000000100": { - Status: &replicationdatapb.PrimaryStatus{ - Position: "MySQL56/3E11FA47-71CA-11E1-9E33-C80AA9429562:1-10", - }, - Error: nil, - }, - "zone1-0000000101": { - Status: &replicationdatapb.PrimaryStatus{ - Position: "MySQL56/3E11FA47-71CA-11E1-9E33-C80AA9429562:1-10", - }, - Error: nil, - }, - "zone1-0000000102": { - Status: &replicationdatapb.PrimaryStatus{ - Position: "MySQL56/3E11FA47-71CA-11E1-9E33-C80AA9429562:1-5", - }, - Error: nil, - }, - }, - PromoteReplicaDelays: map[string]time.Duration{ - "zone1-0000000100": time.Millisecond * 100, - }, - PromoteReplicaResults: map[string]struct { - Result string - Error error - }{ - "zone1-0000000100": { - Result: "reparent journal position", - Error: nil, - }, - }, - }, - timeout: time.Millisecond * 50, - unlockTopo: false, - keyspace: "testkeyspace", - shard: "-", - primaryElect: &topodatapb.Tablet{ - Alias: &topodatapb.TabletAlias{ - Cell: "zone1", - Uid: 100, - }, - }, - tabletMap: map[string]*topo.TabletInfo{ - "zone1-0000000100": { - Tablet: &topodatapb.Tablet{ - Alias: &topodatapb.TabletAlias{ - Cell: "zone1", - Uid: 100, - }, - }, - }, - "zone1-0000000101": { - Tablet: &topodatapb.Tablet{ - Alias: &topodatapb.TabletAlias{ - Cell: "zone1", - Uid: 101, - }, - }, - }, - "zone1-0000000102": { - Tablet: &topodatapb.Tablet{ - Alias: &topodatapb.TabletAlias{ - Cell: "zone1", - Uid: 102, - }, - }, - }, - }, - shouldErr: true, - }, } ctx := context.Background() @@ -2595,7 +2273,7 @@ func TestPlannedReparenter_performPotentialPromotion(t *testing.T) { durability, err := GetDurabilityPolicy("none") require.NoError(t, err) - rp, err := pr.performPotentialPromotion(ctx, tt.keyspace, tt.shard, tt.primaryElect, tt.tabletMap, PlannedReparentOptions{durability: durability}) + err = pr.performPotentialPromotion(ctx, tt.keyspace, tt.shard, tt.primaryElect, tt.tabletMap, PlannedReparentOptions{durability: durability}) if tt.shouldErr { assert.Error(t, err) @@ -2603,7 +2281,6 @@ func TestPlannedReparenter_performPotentialPromotion(t *testing.T) { } assert.NoError(t, err) - assert.Equal(t, tt.expectedPos, rp) }) } } @@ -3335,10 +3012,12 @@ func TestPlannedReparenter_reparentTablets(t *testing.T) { durability string ev *events.Reparent reparentJournalPosition string + promoteReplicaRequired bool tabletMap map[string]*topo.TabletInfo opts PlannedReparentOptions shouldErr bool + wantErr string }{ { name: "success - durability = none", @@ -3473,6 +3152,158 @@ func TestPlannedReparenter_reparentTablets(t *testing.T) { }, }, shouldErr: false, + }, { + name: "success - promote replica required", + durability: "semi_sync", + promoteReplicaRequired: true, + tmc: &testutil.TabletManagerClient{ + PromoteReplicaResults: map[string]struct { + Result string + Error error + }{ + "zone1-0000000100": { + Result: "successful reparent journal position", + Error: nil, + }, + }, + PopulateReparentJournalResults: map[string]error{ + "zone1-0000000100": nil, + }, + SetReplicationSourceResults: map[string]error{ + "zone1-0000000200": nil, + "zone1-0000000201": nil, + "zone1-0000000202": nil, + }, + SetReplicationSourceSemiSync: map[string]bool{ + "zone1-0000000200": true, + "zone1-0000000201": true, + "zone1-0000000202": false, + }, + }, + ev: &events.Reparent{ + NewPrimary: &topodatapb.Tablet{ + Alias: &topodatapb.TabletAlias{ + Cell: "zone1", + Uid: 100, + }, + Type: topodatapb.TabletType_PRIMARY, + }, + }, + tabletMap: map[string]*topo.TabletInfo{ + "zone1-0000000100": { + Tablet: &topodatapb.Tablet{ + Alias: &topodatapb.TabletAlias{ + Cell: "zone1", + Uid: 100, + }, + Type: topodatapb.TabletType_PRIMARY, + }, + }, + "zone1-0000000200": { + Tablet: &topodatapb.Tablet{ + Alias: &topodatapb.TabletAlias{ + Cell: "zone1", + Uid: 200, + }, + Type: topodatapb.TabletType_REPLICA, + }, + }, + "zone1-0000000201": { + Tablet: &topodatapb.Tablet{ + Alias: &topodatapb.TabletAlias{ + Cell: "zone1", + Uid: 201, + }, + Type: topodatapb.TabletType_REPLICA, + }, + }, + "zone1-0000000202": { + Tablet: &topodatapb.Tablet{ + Alias: &topodatapb.TabletAlias{ + Cell: "zone1", + Uid: 202, + }, + Type: topodatapb.TabletType_RDONLY, + }, + }, + }, + shouldErr: false, + }, { + name: "Promote replica failed", + durability: "semi_sync", + promoteReplicaRequired: true, + tmc: &testutil.TabletManagerClient{ + PromoteReplicaResults: map[string]struct { + Result string + Error error + }{ + "zone1-0000000100": { + Error: errors.New("failed promote replica"), + }, + }, + PopulateReparentJournalResults: map[string]error{ + "zone1-0000000100": nil, + }, + SetReplicationSourceResults: map[string]error{ + "zone1-0000000200": nil, + "zone1-0000000201": nil, + "zone1-0000000202": nil, + }, + SetReplicationSourceSemiSync: map[string]bool{ + "zone1-0000000200": true, + "zone1-0000000201": true, + "zone1-0000000202": false, + }, + }, + ev: &events.Reparent{ + NewPrimary: &topodatapb.Tablet{ + Alias: &topodatapb.TabletAlias{ + Cell: "zone1", + Uid: 100, + }, + Type: topodatapb.TabletType_PRIMARY, + }, + }, + tabletMap: map[string]*topo.TabletInfo{ + "zone1-0000000100": { + Tablet: &topodatapb.Tablet{ + Alias: &topodatapb.TabletAlias{ + Cell: "zone1", + Uid: 100, + }, + Type: topodatapb.TabletType_PRIMARY, + }, + }, + "zone1-0000000200": { + Tablet: &topodatapb.Tablet{ + Alias: &topodatapb.TabletAlias{ + Cell: "zone1", + Uid: 200, + }, + Type: topodatapb.TabletType_REPLICA, + }, + }, + "zone1-0000000201": { + Tablet: &topodatapb.Tablet{ + Alias: &topodatapb.TabletAlias{ + Cell: "zone1", + Uid: 201, + }, + Type: topodatapb.TabletType_REPLICA, + }, + }, + "zone1-0000000202": { + Tablet: &topodatapb.Tablet{ + Alias: &topodatapb.TabletAlias{ + Cell: "zone1", + Uid: 202, + }, + Type: topodatapb.TabletType_RDONLY, + }, + }, + }, + shouldErr: true, + wantErr: "failed PromoteReplica(primary=zone1-0000000100,", }, { name: "SetReplicationSource failed on replica", @@ -3534,6 +3365,7 @@ func TestPlannedReparenter_reparentTablets(t *testing.T) { }, }, shouldErr: true, + wantErr: "retry failed replicas: tablet zone1-0000000201 failed to SetReplicationSource(zone1-0000000100): assert.AnError general error for testing", }, { name: "SetReplicationSource timed out on replica", @@ -3601,6 +3433,7 @@ func TestPlannedReparenter_reparentTablets(t *testing.T) { WaitReplicasTimeout: time.Millisecond * 10, }, shouldErr: true, + wantErr: "retry failed replicas: tablet zone1-0000000201 failed to SetReplicationSource(zone1-0000000100): context deadline exceeded", }, { name: "PopulateReparentJournal failed out on new primary", @@ -3662,6 +3495,7 @@ func TestPlannedReparenter_reparentTablets(t *testing.T) { }, }, shouldErr: true, + wantErr: "failed PopulateReparentJournal(primary=zone1-0000000100", }, { name: "PopulateReparentJournal timed out on new primary", @@ -3729,6 +3563,7 @@ func TestPlannedReparenter_reparentTablets(t *testing.T) { WaitReplicasTimeout: time.Millisecond * 10, }, shouldErr: true, + wantErr: "failed PopulateReparentJournal(primary=zone1-0000000100", }, } @@ -3749,10 +3584,12 @@ func TestPlannedReparenter_reparentTablets(t *testing.T) { durability, err := GetDurabilityPolicy(durabilityPolicy) require.NoError(t, err) tt.opts.durability = durability - err = pr.reparentTablets(ctx, tt.ev, tt.reparentJournalPosition, tt.tabletMap, tt.opts) + err = pr.reparentTablets(ctx, tt.ev, tt.reparentJournalPosition, tt.promoteReplicaRequired, tt.tabletMap, tt.opts) if tt.shouldErr { assert.Error(t, err) - + if tt.wantErr != "" { + require.ErrorContains(t, err, tt.wantErr) + } return } diff --git a/go/vt/wrangler/testlib/planned_reparent_shard_test.go b/go/vt/wrangler/testlib/planned_reparent_shard_test.go index f3eaa987ac4..3c025a8c7f1 100644 --- a/go/vt/wrangler/testlib/planned_reparent_shard_test.go +++ b/go/vt/wrangler/testlib/planned_reparent_shard_test.go @@ -97,7 +97,9 @@ func TestPlannedReparentShardNoPrimaryProvided(t *testing.T) { oldPrimary.FakeMysqlDaemon.ExpectedExecuteSuperQueryList = []string{ "FAKE SET MASTER", "START SLAVE", - // we end up calling SetReplicationSource twice on the old primary + // We might end up calling SetReplicationSource twice on the old primary + // one coming from `PlannedReparentShard` and one coming from `endPrimaryTerm`. + // This is a race though between SetReplicationSource on this tablet and `PromoteReplica` on the new primary. "FAKE SET MASTER", "START SLAVE", } @@ -132,8 +134,9 @@ func TestPlannedReparentShardNoPrimaryProvided(t *testing.T) { err = newPrimary.FakeMysqlDaemon.CheckSuperQueryList() require.NoError(t, err) - err = oldPrimary.FakeMysqlDaemon.CheckSuperQueryList() - require.NoError(t, err) + // Because of the race we can either end up using all the queries, or 2 might be left. + queriesLeft := len(oldPrimary.FakeMysqlDaemon.ExpectedExecuteSuperQueryList) - oldPrimary.FakeMysqlDaemon.ExpectedExecuteSuperQueryCurrent + require.True(t, queriesLeft == 0 || queriesLeft == 2) err = goodReplica1.FakeMysqlDaemon.CheckSuperQueryList() require.NoError(t, err) @@ -209,7 +212,9 @@ func TestPlannedReparentShardNoError(t *testing.T) { oldPrimary.FakeMysqlDaemon.ExpectedExecuteSuperQueryList = []string{ "FAKE SET MASTER", "START SLAVE", - // we end up calling SetReplicationSource twice on the old primary + // We might end up calling SetReplicationSource twice on the old primary + // one coming from `PlannedReparentShard` and one coming from `endPrimaryTerm`. + // This is a race though between SetReplicationSource on this tablet and `PromoteReplica` on the new primary. "FAKE SET MASTER", "START SLAVE", } @@ -258,8 +263,9 @@ func TestPlannedReparentShardNoError(t *testing.T) { // check what was run err = newPrimary.FakeMysqlDaemon.CheckSuperQueryList() require.NoError(t, err) - err = oldPrimary.FakeMysqlDaemon.CheckSuperQueryList() - require.NoError(t, err) + // Because of the race we can either end up using all the queries, or 2 might be left. + queriesLeft := len(oldPrimary.FakeMysqlDaemon.ExpectedExecuteSuperQueryList) - oldPrimary.FakeMysqlDaemon.ExpectedExecuteSuperQueryCurrent + require.True(t, queriesLeft == 0 || queriesLeft == 2) err = goodReplica1.FakeMysqlDaemon.CheckSuperQueryList() require.NoError(t, err) err = goodReplica2.FakeMysqlDaemon.CheckSuperQueryList() @@ -795,6 +801,9 @@ func TestPlannedReparentShardPromoteReplicaFail(t *testing.T) { oldPrimary.FakeMysqlDaemon.ExpectedExecuteSuperQueryList = []string{ "FAKE SET MASTER", "START SLAVE", + // extra SetReplicationSource call due to retry + "FAKE SET MASTER", + "START SLAVE", } oldPrimary.StartActionLoop(t, wr) defer oldPrimary.StopActionLoop(t) @@ -814,6 +823,10 @@ func TestPlannedReparentShardPromoteReplicaFail(t *testing.T) { "STOP SLAVE", "FAKE SET MASTER", "START SLAVE", + // extra SetReplicationSource call due to retry + "STOP SLAVE", + "FAKE SET MASTER", + "START SLAVE", } goodReplica1.StartActionLoop(t, wr) defer goodReplica1.StopActionLoop(t) @@ -827,6 +840,8 @@ func TestPlannedReparentShardPromoteReplicaFail(t *testing.T) { "FAKE SET MASTER", "START SLAVE", "FAKE SET MASTER", + // extra SetReplicationSource call due to retry + "FAKE SET MASTER", } goodReplica2.StartActionLoop(t, wr) goodReplica2.FakeMysqlDaemon.Replicating = false @@ -844,23 +859,7 @@ func TestPlannedReparentShardPromoteReplicaFail(t *testing.T) { // retrying should work newPrimary.FakeMysqlDaemon.PromoteError = nil - newPrimary.FakeMysqlDaemon.ExpectedExecuteSuperQueryList = []string{ - "STOP SLAVE", - "FAKE SET MASTER", - "START SLAVE", - // extra commands because of retry - "STOP SLAVE", - "FAKE SET MASTER", - "START SLAVE", - "SUBINSERT INTO _vt.reparent_journal (time_created_ns, action_name, primary_alias, replication_position) VALUES", - } - oldPrimary.FakeMysqlDaemon.ExpectedExecuteSuperQueryList = []string{ - "FAKE SET MASTER", - "START SLAVE", - // extra commands because of retry - "FAKE SET MASTER", - "START SLAVE", - } + newPrimary.FakeMysqlDaemon.CurrentPrimaryPosition = newPrimary.FakeMysqlDaemon.WaitPrimaryPositions[0] // run PlannedReparentShard err = vp.Run([]string{"PlannedReparentShard", "--wait_replicas_timeout", "10s", "--keyspace_shard", newPrimary.Tablet.Keyspace + "/" + newPrimary.Tablet.Shard, "--new_primary", topoproto.TabletAliasString(newPrimary.Tablet.Alias)})