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 15, 2024
1 parent 2eb0e49 commit af477d4
Show file tree
Hide file tree
Showing 7 changed files with 132 additions and 9 deletions.
17 changes: 17 additions & 0 deletions pkg/ccl/backupccl/datadriven_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -712,6 +712,23 @@ func runTestDataDriven(t *testing.T, testFilePathFromWorkspace string) {
d.ScanArgs(t, "user", &user)
}
checkForClusterSetting(t, d.Input, ds.clusters[cluster].NumServers())
if d.HasArg("retry") {
var eventualOutput string
testutils.SucceedsSoon(t, func() error {
rows, err := ds.getSQLDB(t, cluster, user).Query(d.Input)
if err != nil {
return err
}
output, err := sqlutils.RowsToDataDrivenOutput(rows)
require.NoError(t, err)
if output != d.Expected {
return errors.Newf("latest output: %s\n expected: %s", output, d.Expected)
}
eventualOutput = output
return nil
})
return eventualOutput
}
rows, err := ds.getSQLDB(t, cluster, user).Query(d.Input)
if err != nil {
return err.Error()
Expand Down
11 changes: 11 additions & 0 deletions pkg/ccl/backupccl/restore_job.go
Original file line number Diff line number Diff line change
Expand Up @@ -2248,6 +2248,11 @@ func (r *restoreResumer) publishDescriptors(
return err
}

var tableAutoStatsSettings map[uint32]*catpb.AutoStatsSettings
if details.ExperimentalOnline {
tableAutoStatsSettings = make(map[uint32]*catpb.AutoStatsSettings, len(details.TableDescs))
}

// Write the new TableDescriptors and flip state over to public so they can be
// accessed.
for i := range details.TableDescs {
Expand All @@ -2258,6 +2263,9 @@ func (r *restoreResumer) publishDescriptors(
// download job finishes.
boolean := false
mutTable.AutoStatsSettings = &catpb.AutoStatsSettings{Enabled: &boolean}

// Preserve the backed up table stats so the download job re-enables them
tableAutoStatsSettings[uint32(details.TableDescs[i].ID)] = details.TableDescs[i].AutoStatsSettings
}

// Note that we don't need to worry about the re-validated indexes for descriptors
Expand Down Expand Up @@ -2362,6 +2370,9 @@ func (r *restoreResumer) publishDescriptors(
details.SchemaDescs = newSchemas
details.DatabaseDescs = newDBs
details.FunctionDescs = newFunctions
if details.ExperimentalOnline {
details.PostDownloadTableAutoStatsSettings = tableAutoStatsSettings
}
if err := r.job.WithTxn(txn).SetDetails(ctx, details); err != nil {
return errors.Wrap(err,
"updating job details after publishing tables")
Expand Down
34 changes: 26 additions & 8 deletions pkg/ccl/backupccl/restore_online.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/isql"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/sem/catid"
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
"github.com/cockroachdb/cockroach/pkg/util/ctxgroup"
"github.com/cockroachdb/cockroach/pkg/util/envutil"
Expand Down Expand Up @@ -491,7 +492,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, PostDownloadTableAutoStatsSettings: details.PostDownloadTableAutoStatsSettings},
Progress: jobspb.RestoreProgress{},
}

Expand Down Expand Up @@ -621,13 +622,30 @@ func (r *restoreResumer) cleanupAfterDownload(
ctx, sp := tracing.ChildSpan(ctx, "backupccl.cleanupAfterDownload")
defer sp.Finish()

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

// Re-enable automatic stats collection on restored tables.
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)
// Try to restore automatic stats collection preference on each restored
// table.
for id, settings := range details.PostDownloadTableAutoStatsSettings {
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, catid.DescID(id))
if err != nil {
return err
}
newTableDesc.AutoStatsSettings = settings
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", id)
}
}
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 @@ -47,6 +47,9 @@ func TestOnlineRestoreBasic(t *testing.T) {
defer cleanupFn()
externalStorage := "nodelocal://1/backup"

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 Down Expand Up @@ -74,6 +77,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)
}

// TestOnlineRestoreWaitForDownload checks that the download job succeeeds even
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,65 @@
# This test ensures that online restore restores the backed up auto stats
# settings

reset test-nodelocal
----

new-cluster name=s1 disable-tenant
----

exec-sql
USE data;
CREATE TABLE reg (i INT PRIMARY KEY, s STRING);
INSERT INTO reg VALUES (1, 'x'),(2,'y'),(3,'z');
CREATE TABLE stats (i INT PRIMARY KEY, s STRING);
CREATE TABLE nostats (i INT PRIMARY KEY, s STRING);
ALTER TABLE stats SET (sql_stats_automatic_collection_enabled = true);
ALTER TABLE nostats SET (sql_stats_automatic_collection_enabled = false);
----

exec-sql
BACKUP INTO 'nodelocal://1/cluster/';
----


new-cluster name=s2 share-io-dir=s1 allow-implicit-access disable-tenant
----


exec-sql
RESTORE DATABASE data FROM LATEST IN 'nodelocal://1/cluster/' with EXPERIMENTAL DEFERRED COPY;
----

# Wait for download job to complete

query-sql retry
SELECT count(*) FROM [SHOW JOBS] WHERE job_type='RESTORE' and status='succeeded';
----
2

query-sql
SHOW CREATE TABLE data.reg
----
data.public.reg CREATE TABLE public.reg (
i INT8 NOT NULL,
s STRING NULL,
CONSTRAINT reg_pkey PRIMARY KEY (i ASC)
)

query-sql
SHOW CREATE TABLE data.stats
----
data.public.stats CREATE TABLE public.stats (
i INT8 NOT NULL,
s STRING NULL,
CONSTRAINT stats_pkey PRIMARY KEY (i ASC)
) WITH (sql_stats_automatic_collection_enabled = true)

query-sql
SHOW CREATE TABLE data.nostats
----
data.public.nostats CREATE TABLE public.nostats (
i INT8 NOT NULL,
s STRING NULL,
CONSTRAINT nostats_pkey PRIMARY KEY (i ASC)
) WITH (sql_stats_automatic_collection_enabled = false)
1 change: 1 addition & 0 deletions pkg/jobs/jobspb/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,7 @@ proto_library(
"//pkg/kv/kvpb:kvpb_proto",
"//pkg/multitenant/mtinfopb:mtinfopb_proto",
"//pkg/roachpb:roachpb_proto",
"//pkg/sql/catalog/catpb:catpb_proto",
"//pkg/sql/catalog/descpb:descpb_proto",
"//pkg/sql/sessiondatapb:sessiondatapb_proto",
"//pkg/util/hlc:hlc_proto",
Expand Down
8 changes: 7 additions & 1 deletion pkg/jobs/jobspb/jobs.proto
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ import "roachpb/data.proto";
import "roachpb/metadata.proto";
import "roachpb/io-formats.proto";
import "sql/catalog/descpb/structured.proto";
import "sql/catalog/catpb/catalog.proto";
import "multitenant/mtinfopb/info.proto";
import "sql/sessiondatapb/session_data.proto";
import "util/hlc/timestamp.proto";
Expand Down Expand Up @@ -504,8 +505,13 @@ message RestoreDetails {
// UnsafeRestoreIncompatibleVersion allows restoring a backup older than the min compatible
// version.
bool unsafe_restore_incompatible_version = 34;

// PostDownloadTableAutoStatsSettings contains the backed up auto stats
// settings for online restored tables. These setting will be restored at the
// end of the download job.
map<uint32,cockroach.sql.catalog.catpb.AutoStatsSettings> post_download_table_auto_stats_settings = 35;

// NEXT ID: 34.
// NEXT ID: 36.
}


Expand Down

0 comments on commit af477d4

Please sign in to comment.