Skip to content

Commit

Permalink
backupccl: actually enable auto stats at end of download job
Browse files Browse the repository at this point in the history
PR #116977 didn't re-enable auto stats collection for online restored tables
after the download job. This patch fixes this bug.

Fixes #119935

Release note: none
  • Loading branch information
msbutler committed Mar 6, 2024
1 parent 49a7b23 commit adf4906
Show file tree
Hide file tree
Showing 3 changed files with 35 additions and 8 deletions.
3 changes: 1 addition & 2 deletions pkg/ccl/backupccl/restore_job.go
Original file line number Diff line number Diff line change
Expand Up @@ -1867,7 +1867,6 @@ func (r *restoreResumer) doResume(ctx context.Context, execCtx interface{}) erro
if err := insertStats(ctx, r.job, p.ExecCfg(), remappedStats); err != nil {
return errors.Wrap(err, "inserting table statistics")
}

publishDescriptors := func(ctx context.Context, txn descs.Txn) (err error) {
return r.publishDescriptors(
ctx, p.ExecCfg().JobRegistry, p.ExecCfg().JobsKnobs(), txn, p.User(),
Expand Down Expand Up @@ -1997,7 +1996,7 @@ func (r *restoreResumer) maybeWriteDownloadJob(
downloadJobRecord := jobs.Record{
Description: fmt.Sprintf("Background Data Download for %s", r.job.Payload().Description),
Username: r.job.Payload().UsernameProto.Decode(),
Details: jobspb.RestoreDetails{DownloadSpans: downloadSpans},
Details: jobspb.RestoreDetails{DownloadSpans: downloadSpans, TableDescs: details.TableDescs},
Progress: jobspb.RestoreProgress{},
}

Expand Down
35 changes: 29 additions & 6 deletions pkg/ccl/backupccl/restore_online.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,8 @@ import (
"github.com/cockroachdb/cockroach/pkg/server/serverpb"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descs"
"github.com/cockroachdb/cockroach/pkg/sql/execinfrapb"
"github.com/cockroachdb/cockroach/pkg/sql/isql"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
Expand Down Expand Up @@ -501,14 +503,35 @@ func (r *restoreResumer) cleanupAfterDownload(
) error {
ctx, sp := tracing.ChildSpan(ctx, "backupccl.cleanupAfterDownload")
defer sp.Finish()
if len(details.TableDescs) == 0 {
panic("no table descs")
}

executor := r.execCfg.InternalDB.Executor()

// Re-enable automatic stats collection on restored tables.
// Try to restore automatic stats collection preference on each restored
// table.
for _, table := range details.TableDescs {
_, err := executor.Exec(ctx, "enable-stats", nil, `ALTER TABLE $1 SET (sql_stats_automatic_collection_enabled = true);`, table.Name)
if err != nil {
log.Warningf(ctx, "could not enable automatic stats on table %s", table)
if err := sql.DescsTxn(ctx, r.execCfg, func(
ctx context.Context, txn isql.Txn, descsCol *descs.Collection,
) error {
b := txn.KV().NewBatch()
newTableDesc, err := descsCol.MutableByID(txn.KV()).Table(ctx, table.ID)
if err != nil {
return err
}
newTableDesc.AutoStatsSettings = &catpb.AutoStatsSettings{}
if err := descsCol.WriteDescToBatch(
ctx, false /* kvTrace */, newTableDesc, b); err != nil {
return err
}
if err := txn.KV().Run(ctx, b); err != nil {
return err
}
return nil
}); err != nil {
// Re-enabling stats is best effort. The user may have dropped the table
// since it came online.
//log.Warningf(ctx, "failed to re-enable auto stats on table %d", table.ID)
return err
}
}
return nil
Expand Down
5 changes: 5 additions & 0 deletions pkg/ccl/backupccl/restore_online_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -48,6 +48,9 @@ func TestOnlineRestoreBasic(t *testing.T) {
// TODO(dt): remove this when OR supports synthesis.
sqlDB.Exec(t, `SET CLUSTER SETTING bulkio.backup.elide_common_prefix.enabled = false`)

createStmt := `SELECT create_statement FROM [SHOW CREATE TABLE data.bank]`
createStmtRes := sqlDB.QueryStr(t, createStmt)

sqlDB.Exec(t, fmt.Sprintf("BACKUP INTO '%s'", externalStorage))

params := base.TestClusterArgs{
Expand All @@ -63,6 +66,8 @@ func TestOnlineRestoreBasic(t *testing.T) {
var downloadJobID jobspb.JobID
rSQLDB.QueryRow(t, `SELECT job_id FROM [SHOW JOBS] WHERE description LIKE '%Background Data Download%'`).Scan(&downloadJobID)
jobutils.WaitForJobToSucceed(t, rSQLDB, downloadJobID)

rSQLDB.CheckQueryResults(t, createStmt, createStmtRes)
}

// TestOnlineRestoreTenant runs an online restore of a tenant and ensures the
Expand Down

0 comments on commit adf4906

Please sign in to comment.