Skip to content

Commit

Permalink
Revert "removing DeleteBeforeRestore"
Browse files Browse the repository at this point in the history
This reverts commit e62ae95.

Signed-off-by: Renan Rangel <rrangel@slack-corp.com>
  • Loading branch information
rvrangel committed Sep 10, 2024
1 parent 4257ae4 commit e958558
Show file tree
Hide file tree
Showing 11 changed files with 91 additions and 65 deletions.
1 change: 1 addition & 0 deletions go/cmd/vtbackup/cli/vtbackup.go
Original file line number Diff line number Diff line change
Expand Up @@ -428,6 +428,7 @@ func takeBackup(ctx, backgroundCtx context.Context, topoServer *topo.Server, bac
Logger: logutil.NewConsoleLogger(),
Concurrency: concurrency,
HookExtraEnv: extraEnv,
DeleteBeforeRestore: true,
DbName: dbName,
Keyspace: initKeyspace,
Shard: initShard,
Expand Down
28 changes: 26 additions & 2 deletions go/vt/mysqlctl/backup.go
Original file line number Diff line number Diff line change
Expand Up @@ -237,6 +237,29 @@ func ParseBackupName(dir string, name string) (backupTime *time.Time, alias *top
return backupTime, alias, nil
}

// checkNoDB makes sure there is no user data already there.
// Used by Restore, as we do not want to destroy an existing DB.
// The user's database name must be given since we ignore all others.
// Returns (true, nil) if the specified DB doesn't exist.
// Returns (false, nil) if the check succeeds but the condition is not
// satisfied (there is a DB).
// Returns (false, non-nil error) if one occurs while trying to perform the check.
func checkNoDB(ctx context.Context, mysqld MysqlDaemon, dbName string) (bool, error) {
qr, err := mysqld.FetchSuperQuery(ctx, "SHOW DATABASES")
if err != nil {
return false, vterrors.Wrap(err, "checkNoDB failed")
}

for _, row := range qr.Rows {
if row[0].ToString() == dbName {
// found active db
log.Warningf("checkNoDB failed, found active db %v", dbName)
return false, nil
}
}
return true, nil
}

// removeExistingFiles will delete existing files in the data dir to prevent
// conflicts with the restored archive. In particular, binlogs can be created
// even during initial bootstrap, and these can interfere with configuring
Expand Down Expand Up @@ -289,9 +312,10 @@ func removeExistingFiles(cnf *Mycnf) error {
// ShouldRestore checks whether a database with tables already exists
// and returns whether a restore action should be performed
func ShouldRestore(ctx context.Context, params RestoreParams) (bool, error) {
if RestoreWasInterrupted(params.Cnf) {
if params.DeleteBeforeRestore || RestoreWasInterrupted(params.Cnf) {
return true, nil
}
params.Logger.Infof("Restore: No %v file found, checking no existing data is present", RestoreState)
// Wait for mysqld to be ready, in case it was launched in parallel with us.
// If this doesn't succeed, we should not attempt a restore
if err := params.Mysqld.Wait(ctx, params.Cnf); err != nil {
Expand All @@ -301,7 +325,7 @@ func ShouldRestore(ctx context.Context, params RestoreParams) (bool, error) {
params.Logger.Errorf("error waiting for the grants: %v", err)
return false, err
}
return true, nil
return checkNoDB(ctx, params.Mysqld, params.DbName)
}

// ensureRestoredGTIDPurgedMatchesManifest sees the following: when you restore a full backup, you want the MySQL server to have
Expand Down
1 change: 1 addition & 0 deletions go/vt/mysqlctl/backup_blackbox_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -512,6 +512,7 @@ func TestExecuteRestoreWithTimedOutContext(t *testing.T) {
Mysqld: mysqld,
Concurrency: 2,
HookExtraEnv: map[string]string{},
DeleteBeforeRestore: false,
DbName: "test",
Keyspace: "test",
Shard: "-",
Expand Down
32 changes: 18 additions & 14 deletions go/vt/mysqlctl/backup_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,7 @@ import (
"vitess.io/vitess/go/mysql"
"vitess.io/vitess/go/mysql/fakesqldb"
"vitess.io/vitess/go/mysql/replication"
"vitess.io/vitess/go/sqltypes"
"vitess.io/vitess/go/test/utils"
"vitess.io/vitess/go/vt/logutil"
"vitess.io/vitess/go/vt/mysqlctl/backupstats"
Expand Down Expand Up @@ -584,6 +585,7 @@ func createFakeBackupRestoreEnv(t *testing.T) *fakeBackupRestoreEnv {
Mysqld: mysqld,
Concurrency: 1,
HookExtraEnv: map[string]string{},
DeleteBeforeRestore: false,
DbName: "test",
Keyspace: "test",
Shard: "-",
Expand Down Expand Up @@ -688,26 +690,28 @@ func TestShouldRestore(t *testing.T) {
env := createFakeBackupRestoreEnv(t)

b, err := ShouldRestore(env.ctx, env.restoreParams)
assert.True(t, b)
assert.NoError(t, err)

// if MySQL is online on time, we should continue the restore
env.mysqld.WaitDuration = time.Second
ctx, cancel := context.WithTimeout(context.Background(), time.Second*3)
defer cancel()
assert.False(t, b)
assert.Error(t, err)

b, err = ShouldRestore(ctx, env.restoreParams)
env.restoreParams.DeleteBeforeRestore = true
b, err = ShouldRestore(env.ctx, env.restoreParams)
assert.True(t, b)
assert.NoError(t, err)
env.restoreParams.DeleteBeforeRestore = false

// but should return an error if waiting for MySQL longer than the context expected
env.mysqld.WaitDuration = time.Second * 2
ctx, cancel = context.WithTimeout(context.Background(), time.Second)
defer cancel()
env.mysqld.FetchSuperQueryMap = map[string]*sqltypes.Result{
"SHOW DATABASES": {Rows: [][]sqltypes.Value{{sqltypes.NewVarBinary("any_db")}}},
}
b, err = ShouldRestore(env.ctx, env.restoreParams)
assert.NoError(t, err)
assert.True(t, b)

b, err = ShouldRestore(ctx, env.restoreParams)
env.mysqld.FetchSuperQueryMap = map[string]*sqltypes.Result{
"SHOW DATABASES": {Rows: [][]sqltypes.Value{{sqltypes.NewVarBinary("test")}}},
}
b, err = ShouldRestore(env.ctx, env.restoreParams)
assert.False(t, b)
assert.ErrorIs(t, err, context.DeadlineExceeded)
assert.NoError(t, err)
}

func TestScanLinesToLogger(t *testing.T) {
Expand Down
5 changes: 5 additions & 0 deletions go/vt/mysqlctl/backupengine.go
Original file line number Diff line number Diff line change
Expand Up @@ -118,6 +118,10 @@ type RestoreParams struct {
Concurrency int
// Extra env variables for pre-restore and post-restore transform hooks
HookExtraEnv map[string]string
// DeleteBeforeRestore tells us whether existing data should be deleted before
// restoring. This is always set to false when starting a tablet with -restore_from_backup,
// but is set to true when executing a RestoreFromBackup command on an already running vttablet
DeleteBeforeRestore bool
// DbName is the name of the managed database / schema
DbName string
// Keyspace and Shard are used to infer the directory where backups are stored
Expand Down Expand Up @@ -148,6 +152,7 @@ func (p *RestoreParams) Copy() RestoreParams {
Logger: p.Logger,
Concurrency: p.Concurrency,
HookExtraEnv: p.HookExtraEnv,
DeleteBeforeRestore: p.DeleteBeforeRestore,
DbName: p.DbName,
Keyspace: p.Keyspace,
Shard: p.Shard,
Expand Down
18 changes: 2 additions & 16 deletions go/vt/mysqlctl/fakemysqldaemon.go
Original file line number Diff line number Diff line change
Expand Up @@ -139,10 +139,6 @@ type FakeMysqlDaemon struct {
// function returns an error.
WaitPrimaryPositions []replication.Position

// WaitDuration is used by the Wait() and WaitForDBAGrants() calls to
// simulate and arbitrary amount of time it would take waiting for mysql
WaitDuration time.Duration

// PromoteResult is returned by Promote.
PromoteResult replication.Position

Expand Down Expand Up @@ -275,21 +271,11 @@ func (fmd *FakeMysqlDaemon) RefreshConfig(ctx context.Context, cnf *Mycnf) error

// Wait is part of the MysqlDaemon interface.
func (fmd *FakeMysqlDaemon) Wait(ctx context.Context, cnf *Mycnf) error {
select {
case <-ctx.Done():
return ctx.Err()
case <-time.After(fmd.WaitDuration):
return nil
}
return nil
}

func (fmd *FakeMysqlDaemon) WaitForDBAGrants(ctx context.Context, waitTime time.Duration) (err error) {
select {
case <-ctx.Done():
return ctx.Err()
case <-time.After(fmd.WaitDuration):
return nil
}
return nil
}

// GetMysqlPort is part of the MysqlDaemon interface.
Expand Down
49 changes: 26 additions & 23 deletions go/vt/mysqlctl/mysqlshellbackupengine.go
Original file line number Diff line number Diff line change
Expand Up @@ -100,7 +100,7 @@ const (
)

func (be *MySQLShellBackupEngine) ExecuteBackup(ctx context.Context, params BackupParams, bh backupstorage.BackupHandle) (result BackupResult, finalErr error) {
params.Logger.Infof("running ExecuteBackup in %s", params.TabletAlias)
params.Logger.Infof("Starting ExecuteBackup in %s", params.TabletAlias)

location := path.Join(mysqlShellBackupLocation, bh.Directory(), bh.Name())

Expand Down Expand Up @@ -225,7 +225,7 @@ func (be *MySQLShellBackupEngine) ExecuteBackup(ctx context.Context, params Back
}

func (be *MySQLShellBackupEngine) ExecuteRestore(ctx context.Context, params RestoreParams, bh backupstorage.BackupHandle) (*BackupManifest, error) {
params.Logger.Infof("running ExecuteRestore for %s", params.DbName)
params.Logger.Infof("Calling ExecuteRestore for %s (DeleteBeforeRestore: %v)", params.DbName, params.DeleteBeforeRestore)

err := be.restorePreCheck(ctx, params)
if err != nil {
Expand All @@ -248,35 +248,38 @@ func (be *MySQLShellBackupEngine) ExecuteRestore(ctx context.Context, params Res
return nil, vterrors.Wrap(err, "disable semi-sync failed")
}

// if super read only is enabled, we need to disable it first so we can then remove existing databases
readonly, err := params.Mysqld.IsSuperReadOnly(ctx)
if err != nil {
return nil, vterrors.Wrap(err, fmt.Sprintf("checking if mysqld has super_read_only=enable: %v", err))
}
// if we received a RestoreFromBackup API call instead of it being a command line argument,
// we need to first clean the host before we start the restore.
if params.DeleteBeforeRestore {
params.Logger.Infof("restoring on an existing tablet, so dropping database %q", params.DbName)

if readonly {
resetFunc, err := params.Mysqld.SetSuperReadOnly(ctx, false)
readonly, err := params.Mysqld.IsSuperReadOnly(ctx)
if err != nil {
return nil, vterrors.Wrap(err, fmt.Sprintf("unable to disable super-read-only: %v", err))
return nil, vterrors.Wrap(err, fmt.Sprintf("checking if mysqld has super_read_only=enable: %v", err))
}

defer func() { // make sure we enable it back on after the restore is done
err := resetFunc()
if readonly {
resetFunc, err := params.Mysqld.SetSuperReadOnly(ctx, false)
if err != nil {
params.Logger.Errorf("Not able to set super_read_only to its original value after restore")
return nil, vterrors.Wrap(err, fmt.Sprintf("unable to disable super-read-only: %v", err))
}
}()
}

// we first need to drop existing databases so mysql shell can properly restore them from backup
params.Logger.Infof("restoring on an existing tablet, so dropping database %q", params.DbName)
defer func() {
err := resetFunc()
if err != nil {
params.Logger.Errorf("Not able to set super_read_only to its original value after restore")
}
}()
}

err = params.Mysqld.ExecuteSuperQueryList(ctx,
[]string{fmt.Sprintf("DROP DATABASE IF EXISTS `%s`", params.DbName),
fmt.Sprintf("DROP DATABASE IF EXISTS `%s`", sidecar.GetName())},
)
if err != nil {
return nil, vterrors.Wrap(err, fmt.Sprintf("dropping database %q failed", params.DbName))
}

err = params.Mysqld.ExecuteSuperQueryList(ctx,
[]string{fmt.Sprintf("DROP DATABASE IF EXISTS `%s`", params.DbName),
fmt.Sprintf("DROP DATABASE IF EXISTS `%s`", sidecar.GetName())},
)
if err != nil {
return nil, vterrors.Wrap(err, fmt.Sprintf("dropping database %q failed", params.DbName))
}

// we need to get rid of all the current replication information on the host.
Expand Down
6 changes: 4 additions & 2 deletions go/vt/vttablet/tabletmanager/restore.go
Original file line number Diff line number Diff line change
Expand Up @@ -128,6 +128,7 @@ func (tm *TabletManager) RestoreData(
ctx context.Context,
logger logutil.Logger,
waitForBackupInterval time.Duration,
deleteBeforeRestore bool,
backupTime time.Time,
restoreToTimetamp time.Time,
restoreToPos string,
Expand Down Expand Up @@ -180,14 +181,14 @@ func (tm *TabletManager) RestoreData(
RestoreToPos: restoreToPos,
RestoreToTimestamp: protoutil.TimeToProto(restoreToTimetamp),
}
err = tm.restoreDataLocked(ctx, logger, waitForBackupInterval, req, mysqlShutdownTimeout)
err = tm.restoreDataLocked(ctx, logger, waitForBackupInterval, deleteBeforeRestore, req, mysqlShutdownTimeout)
if err != nil {
return err
}
return nil
}

func (tm *TabletManager) restoreDataLocked(ctx context.Context, logger logutil.Logger, waitForBackupInterval time.Duration, request *tabletmanagerdatapb.RestoreFromBackupRequest, mysqlShutdownTimeout time.Duration) error {
func (tm *TabletManager) restoreDataLocked(ctx context.Context, logger logutil.Logger, waitForBackupInterval time.Duration, deleteBeforeRestore bool, request *tabletmanagerdatapb.RestoreFromBackupRequest, mysqlShutdownTimeout time.Duration) error {

tablet := tm.Tablet()
originalType := tablet.Type
Expand Down Expand Up @@ -222,6 +223,7 @@ func (tm *TabletManager) restoreDataLocked(ctx context.Context, logger logutil.L
Logger: logger,
Concurrency: restoreConcurrency,
HookExtraEnv: tm.hookExtraEnv(),
DeleteBeforeRestore: deleteBeforeRestore,
DbName: topoproto.TabletDbName(tablet),
Keyspace: keyspace,
Shard: tablet.Shard,
Expand Down
2 changes: 1 addition & 1 deletion go/vt/vttablet/tabletmanager/rpc_backup.go
Original file line number Diff line number Diff line change
Expand Up @@ -190,7 +190,7 @@ func (tm *TabletManager) RestoreFromBackup(ctx context.Context, logger logutil.L
l := logutil.NewTeeLogger(logutil.NewConsoleLogger(), logger)

// Now we can run restore.
err = tm.restoreDataLocked(ctx, l, 0 /* waitForBackupInterval */, request, mysqlShutdownTimeout)
err = tm.restoreDataLocked(ctx, l, 0 /* waitForBackupInterval */, true /* deleteBeforeRestore */, request, mysqlShutdownTimeout)

// Re-run health check to be sure to capture any replication delay.
tm.QueryServiceControl.BroadcastHealth()
Expand Down
2 changes: 1 addition & 1 deletion go/vt/vttablet/tabletmanager/tm_init.go
Original file line number Diff line number Diff line change
Expand Up @@ -843,7 +843,7 @@ func (tm *TabletManager) handleRestore(ctx context.Context, config *tabletenv.Ta
}
// restoreFromBackup will just be a regular action
// (same as if it was triggered remotely)
if err := tm.RestoreData(ctx, logutil.NewConsoleLogger(), waitForBackupInterval, backupTime, restoreToTimestamp, restoreToPos, mysqlShutdownTimeout); err != nil {
if err := tm.RestoreData(ctx, logutil.NewConsoleLogger(), waitForBackupInterval, false /* deleteBeforeRestore */, backupTime, restoreToTimestamp, restoreToPos, mysqlShutdownTimeout); err != nil {
log.Exitf("RestoreFromBackup failed: %v", err)
}

Expand Down
12 changes: 6 additions & 6 deletions go/vt/wrangler/testlib/backup_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -264,7 +264,7 @@ func testBackupRestore(t *testing.T, cDetails *compressionDetails) error {
RelayLogInfoPath: path.Join(root, "relay-log.info"),
}

err = destTablet.TM.RestoreData(ctx, logutil.NewConsoleLogger(), 0 /* waitForBackupInterval */, time.Time{} /* backupTime */, time.Time{} /* restoreToTimestamp */, "", mysqlShutdownTimeout)
err = destTablet.TM.RestoreData(ctx, logutil.NewConsoleLogger(), 0 /* waitForBackupInterval */, false /* deleteBeforeRestore */, time.Time{} /* backupTime */, time.Time{} /* restoreToTimestamp */, "", mysqlShutdownTimeout)
if err != nil {
return err
}
Expand Down Expand Up @@ -304,7 +304,7 @@ func testBackupRestore(t *testing.T, cDetails *compressionDetails) error {
primary.FakeMysqlDaemon.SetReplicationPositionPos = primary.FakeMysqlDaemon.CurrentPrimaryPosition

// restore primary from latest backup
require.NoError(t, primary.TM.RestoreData(ctx, logutil.NewConsoleLogger(), 0 /* waitForBackupInterval */, time.Time{} /* restoreFromBackupTs */, time.Time{} /* restoreToTimestamp */, "", mysqlShutdownTimeout),
require.NoError(t, primary.TM.RestoreData(ctx, logutil.NewConsoleLogger(), 0 /* waitForBackupInterval */, false /* deleteBeforeRestore */, time.Time{} /* restoreFromBackupTs */, time.Time{} /* restoreToTimestamp */, "", mysqlShutdownTimeout),
"RestoreData failed")
// tablet was created as PRIMARY, so it's baseTabletType is PRIMARY
assert.Equal(t, topodatapb.TabletType_PRIMARY, primary.Tablet.Type)
Expand All @@ -320,7 +320,7 @@ func testBackupRestore(t *testing.T, cDetails *compressionDetails) error {
}

// Test restore with the backup timestamp
require.NoError(t, primary.TM.RestoreData(ctx, logutil.NewConsoleLogger(), 0 /* waitForBackupInterval */, backupTime, time.Time{} /* restoreToTimestamp */, "", mysqlShutdownTimeout),
require.NoError(t, primary.TM.RestoreData(ctx, logutil.NewConsoleLogger(), 0 /* waitForBackupInterval */, false /* deleteBeforeRestore */, backupTime, time.Time{} /* restoreToTimestamp */, "", mysqlShutdownTimeout),
"RestoreData with backup timestamp failed")
assert.Equal(t, topodatapb.TabletType_PRIMARY, primary.Tablet.Type)
assert.False(t, primary.FakeMysqlDaemon.Replicating)
Expand Down Expand Up @@ -522,7 +522,7 @@ func TestBackupRestoreLagged(t *testing.T) {

errCh = make(chan error, 1)
go func(ctx context.Context, tablet *FakeTablet) {
errCh <- tablet.TM.RestoreData(ctx, logutil.NewConsoleLogger(), 0 /* waitForBackupInterval */, time.Time{} /* restoreFromBackupTs */, time.Time{} /* restoreToTimestamp */, "", mysqlShutdownTimeout)
errCh <- tablet.TM.RestoreData(ctx, logutil.NewConsoleLogger(), 0 /* waitForBackupInterval */, false /* deleteBeforeRestore */, time.Time{} /* restoreFromBackupTs */, time.Time{} /* restoreToTimestamp */, "", mysqlShutdownTimeout)
}(ctx, destTablet)

timer = time.NewTicker(1 * time.Second)
Expand Down Expand Up @@ -717,7 +717,7 @@ func TestRestoreUnreachablePrimary(t *testing.T) {
// set a short timeout so that we don't have to wait 30 seconds
topo.RemoteOperationTimeout = 2 * time.Second
// Restore should still succeed
require.NoError(t, destTablet.TM.RestoreData(ctx, logutil.NewConsoleLogger(), 0 /* waitForBackupInterval */, time.Time{} /* restoreFromBackupTs */, time.Time{} /* restoreToTimestamp */, "", mysqlShutdownTimeout))
require.NoError(t, destTablet.TM.RestoreData(ctx, logutil.NewConsoleLogger(), 0 /* waitForBackupInterval */, false /* deleteBeforeRestore */, time.Time{} /* restoreFromBackupTs */, time.Time{} /* restoreToTimestamp */, "", mysqlShutdownTimeout))
// verify the full status
require.NoError(t, destTablet.FakeMysqlDaemon.CheckSuperQueryList(), "destTablet.FakeMysqlDaemon.CheckSuperQueryList failed")
assert.True(t, destTablet.FakeMysqlDaemon.Replicating)
Expand Down Expand Up @@ -872,7 +872,7 @@ func TestDisableActiveReparents(t *testing.T) {
RelayLogInfoPath: path.Join(root, "relay-log.info"),
}

require.NoError(t, destTablet.TM.RestoreData(ctx, logutil.NewConsoleLogger(), 0 /* waitForBackupInterval */, time.Time{} /* restoreFromBackupTs */, time.Time{} /* restoreToTimestamp */, "", mysqlShutdownTimeout))
require.NoError(t, destTablet.TM.RestoreData(ctx, logutil.NewConsoleLogger(), 0 /* waitForBackupInterval */, false /* deleteBeforeRestore */, time.Time{} /* restoreFromBackupTs */, time.Time{} /* restoreToTimestamp */, "", mysqlShutdownTimeout))
// verify the full status
require.NoError(t, destTablet.FakeMysqlDaemon.CheckSuperQueryList(), "destTablet.FakeMysqlDaemon.CheckSuperQueryList failed")
assert.False(t, destTablet.FakeMysqlDaemon.Replicating)
Expand Down

0 comments on commit e958558

Please sign in to comment.