Skip to content

Commit

Permalink
backup: only invalidate user cache for system restores
Browse files Browse the repository at this point in the history
As of #135852, restore increments the generation of the user role table
in order to invalidate the user cache. This step isn't needed for table
or database level restores. Skipping the cache invalidation is desirable
because it prevents an unrelated long running transaction from blocking
the completion of the restore.

Release Note: none
Part of: #138010
  • Loading branch information
jeffswenson committed Jan 17, 2025
1 parent 7a33951 commit 725fc14
Show file tree
Hide file tree
Showing 3 changed files with 79 additions and 16 deletions.
1 change: 1 addition & 0 deletions pkg/ccl/backupccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -196,6 +196,7 @@ go_test(
"restore_planning_test.go",
"restore_progress_test.go",
"restore_span_covering_test.go",
"restore_test.go",
"revision_reader_test.go",
"schedule_pts_chaining_test.go",
"show_test.go",
Expand Down
34 changes: 18 additions & 16 deletions pkg/ccl/backupccl/restore_job.go
Original file line number Diff line number Diff line change
Expand Up @@ -2011,24 +2011,26 @@ func (r *restoreResumer) doResume(ctx context.Context, execCtx interface{}) erro
}
}

// Bump the version of the role membership table so that the cache is
// invalidated.
if err := r.execCfg.InternalDB.DescsTxn(ctx, func(ctx context.Context, txn descs.Txn) error {
txn.KV().SetDebugName("system-restore-bump-role-membership-table")
log.Eventf(ctx, "bumping table version of %s", systemschema.RoleMembersTable.GetName())
if details.DescriptorCoverage != tree.RequestedDescriptors {
// Bump the version of the role membership table so that the cache is
// invalidated.
if err := r.execCfg.InternalDB.DescsTxn(ctx, func(ctx context.Context, txn descs.Txn) error {
txn.KV().SetDebugName("system-restore-bump-role-membership-table")
log.Eventf(ctx, "bumping table version of %s", systemschema.RoleMembersTable.GetName())

td, err := txn.Descriptors().MutableByID(txn.KV()).Table(ctx, keys.RoleMembersTableID)
if err != nil {
return errors.Wrapf(err, "fetching table %s", systemschema.RoleMembersTable.GetName())
}
td.MaybeIncrementVersion()
if err := txn.Descriptors().WriteDesc(ctx, false, td, txn.KV()); err != nil {
return errors.Wrapf(err, "bumping table version for %s", systemschema.RoleMembersTable.GetName())
}

td, err := txn.Descriptors().MutableByID(txn.KV()).Table(ctx, keys.RoleMembersTableID)
if err != nil {
return errors.Wrapf(err, "fetching table %s", systemschema.RoleMembersTable.GetName())
}
td.MaybeIncrementVersion()
if err := txn.Descriptors().WriteDesc(ctx, false, td, txn.KV()); err != nil {
return errors.Wrapf(err, "bumping table version for %s", systemschema.RoleMembersTable.GetName())
return nil
}); err != nil {
return err
}

return nil
}); err != nil {
return err
}

if err := r.execCfg.ProtectedTimestampManager.Unprotect(ctx, r.job); err != nil {
Expand Down
60 changes: 60 additions & 0 deletions pkg/ccl/backupccl/restore_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,60 @@
// Copyright 2025 The Cockroach Authors.
//
// Use of this software is governed by the CockroachDB Software License
// included in the /LICENSE file.

package backup

import (
"context"
"testing"
"time"

"github.com/cockroachdb/cockroach/pkg/backup/backuptestutils"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/stretchr/testify/require"
)

func TestRestoreWithOpenTransaction(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

clusterSize := 1
tc, sqlDB, _, cleanupFn := backuptestutils.StartBackupRestoreTestCluster(t, clusterSize)
defer cleanupFn()

sqlDB.Exec(t, `CREATE ROLE testuser WITH LOGIN PASSWORD 'password'`)
sqlDB.Exec(t, `CREATE DATABASE restoretarget;`)

userConn := sqlutils.MakeSQLRunner(tc.Servers[0].SQLConn(t, serverutils.UserPassword("testuser", "password")))
userConn.Exec(t, "CREATE TABLE ids (id UUID PRIMARY KEY NOT NULL DEFAULT gen_random_uuid());")
sqlDB.Exec(t, `BACKUP TABLE ids INTO 'nodelocal://1/ids'`)

userConn.Exec(t, "BEGIN")
// Query the id table to take out a lease and perform role access checks.
_ = userConn.QueryStr(t, "SELECT * FROM ids")

result := make(chan error)
go func() {
_, err := sqlDB.DB.ExecContext(context.Background(), `RESTORE TABLE ids FROM LATEST IN 'nodelocal://1/ids' WITH into_db = 'restoretarget'`)
result <- err
}()

select {
case <-time.After(2 * time.Minute):
// This is a regression test for misbehavior in restore. Restore was
// incrementing the role table's descriptor version in order to flush the
// role cache. This is necessary for full cluster restores, since they
// modify the role table, but caused a regression for table and database
// level restores. Table and database restores would hang if there were any
// open long running transactions.
t.Fatal("restore is blocked by an open transaction")
case err := <-result:
require.NoError(t, err)
}

userConn.Exec(t, "COMMIT")
}

0 comments on commit 725fc14

Please sign in to comment.