Skip to content

Commit d740cff

Browse files
committed
restore: handle lookup of dropped system tables in classic restore
Previously, if a lookup was attempted on a dropped system table after a failed restore, an error would be surfaced. This caused the restore jobs in the reverting state to be stuck in a retry loop. However, if we are attempting to do cleanup, we can simply skip any dropped descriptors, which this commit teaches restore to do. Epic: CRDB-51394 Fixes: #148088 Release note: Classic restore no longer gets stuck in the reverting state after failed cleanup of dropped temporary system tables. wip
1 parent abf1a09 commit d740cff

File tree

2 files changed

+100
-17
lines changed

2 files changed

+100
-17
lines changed

pkg/backup/restore_job.go

Lines changed: 73 additions & 17 deletions
Original file line numberDiff line numberDiff line change
@@ -55,6 +55,8 @@ import (
5555
"github.com/cockroachdb/cockroach/pkg/sql/catalog/typedesc"
5656
"github.com/cockroachdb/cockroach/pkg/sql/execinfrapb"
5757
"github.com/cockroachdb/cockroach/pkg/sql/isql"
58+
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
59+
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
5860
"github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scbackup"
5961
"github.com/cockroachdb/cockroach/pkg/sql/sem/catconstants"
6062
"github.com/cockroachdb/cockroach/pkg/sql/sem/catid"
@@ -2081,6 +2083,12 @@ func (r *restoreResumer) doResume(ctx context.Context, execCtx interface{}) erro
20812083
}
20822084
}
20832085

2086+
if err := p.ExecCfg().JobRegistry.CheckPausepoint(
2087+
"restore.after_cleanup_temp_system_tables",
2088+
); err != nil {
2089+
return err
2090+
}
2091+
20842092
if details.DescriptorCoverage != tree.RequestedDescriptors {
20852093
// Bump the version of the role membership table so that the cache is
20862094
// invalidated.
@@ -2767,24 +2775,20 @@ func (r *restoreResumer) dropDescriptors(
27672775
b := txn.KV().NewBatch()
27682776
const kvTrace = false
27692777
// Collect the tables into mutable versions.
2770-
mutableTables := make([]*tabledesc.Mutable, len(details.TableDescs))
2771-
for i := range details.TableDescs {
2772-
var err error
2773-
mutableTables[i], err = descsCol.MutableByID(txn.KV()).Table(ctx, details.TableDescs[i].ID)
2774-
if err != nil {
2775-
return err
2776-
}
2777-
// Ensure that the version matches what we expect. In the case that it
2778-
// doesn't, it's not really clear what to do. Just log and carry on. If the
2779-
// descriptors have already been published, then there's nothing to fuss
2780-
// about so we only do this check if they have not been published.
2781-
if !details.DescriptorsPublished {
2782-
if got, exp := mutableTables[i].Version, details.TableDescs[i].Version; got != exp {
2783-
log.Errorf(ctx, "version changed for restored descriptor %d before "+
2784-
"drop: got %d, expected %d", mutableTables[i].GetID(), got, exp)
2785-
}
2778+
mutableTables, err := getUndroppedTablesFromRestore(
2779+
ctx, txn.KV(), details, descsCol,
2780+
)
2781+
if err != nil {
2782+
return errors.Wrap(err, "getting mutable tables from restore")
2783+
}
2784+
// Ensure that the table versions matches what we expect. In the case that it
2785+
// doesn't, it's not really clear what to do. Just log and carry on. If the
2786+
// descriptors have already been published, then there's nothing to fuss
2787+
// about so we only do this check if they have not been published.
2788+
if !details.DescriptorsPublished {
2789+
if err := checkRestoredTableDescriptorVersions(details, mutableTables); err != nil {
2790+
log.Errorf(ctx, "table version mismatch during drop: %v", err)
27862791
}
2787-
27882792
}
27892793

27902794
// Remove any back references installed from existing types to tables being restored.
@@ -3364,6 +3368,58 @@ func (r *restoreResumer) cleanupTempSystemTables(ctx context.Context) error {
33643368
return nil
33653369
}
33663370

3371+
// getUndroppedTablesFromRestore retrieves all table descriptors, offline or
3372+
// online, as listed in the restore details, excluding any tables that have been
3373+
// dropped or marked as dropped. This helps avoid situations where the temporary
3374+
// system database tables have already been copied over to the real system
3375+
// database and dropped but attempting to load those temporary tables again
3376+
// would result in an error.
3377+
func getUndroppedTablesFromRestore(
3378+
ctx context.Context, txn *kv.Txn, details jobspb.RestoreDetails, descCol *descs.Collection,
3379+
) ([]*tabledesc.Mutable, error) {
3380+
var tables []*tabledesc.Mutable
3381+
for _, desc := range details.TableDescs {
3382+
mutableTable, err := descCol.MutableByID(txn).Table(ctx, desc.ID)
3383+
if err != nil {
3384+
if pgerror.GetPGCode(err) == pgcode.UndefinedTable {
3385+
continue
3386+
}
3387+
return nil, err
3388+
}
3389+
if mutableTable.Dropped() {
3390+
continue
3391+
}
3392+
tables = append(tables, mutableTable)
3393+
}
3394+
return tables, nil
3395+
}
3396+
3397+
// checkeRestoredTableDescriptorVersions compares the versions of descriptors at
3398+
// the time of restore with the versions of the restored tables. It returns an
3399+
// error if any of the restored tables have a version that does not match the
3400+
// version that was recorded at the time of restore in the job details.
3401+
func checkRestoredTableDescriptorVersions(
3402+
details jobspb.RestoreDetails, restoredTables []*tabledesc.Mutable,
3403+
) error {
3404+
versionsAtRestoreTime := make(map[descpb.ID]descpb.DescriptorVersion)
3405+
for _, desc := range details.TableDescs {
3406+
versionsAtRestoreTime[desc.ID] = desc.Version
3407+
}
3408+
for _, table := range restoredTables {
3409+
if expVersion, ok := versionsAtRestoreTime[table.GetID()]; ok {
3410+
if table.Version != expVersion {
3411+
return errors.Errorf(
3412+
"version mismatch for restored descriptor %d, expected version %d, got %d",
3413+
table.GetID(), expVersion, table.Version,
3414+
)
3415+
}
3416+
} else {
3417+
return errors.Errorf("restored table %d not found in restore details", table.GetID())
3418+
}
3419+
}
3420+
return nil
3421+
}
3422+
33673423
var _ jobs.Resumer = &restoreResumer{}
33683424

33693425
func init() {

pkg/backup/restore_test.go

Lines changed: 27 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -11,6 +11,8 @@ import (
1111
"time"
1212

1313
"github.com/cockroachdb/cockroach/pkg/backup/backuptestutils"
14+
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
15+
"github.com/cockroachdb/cockroach/pkg/testutils/jobutils"
1416
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
1517
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
1618
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
@@ -58,3 +60,28 @@ func TestRestoreWithOpenTransaction(t *testing.T) {
5860

5961
userConn.Exec(t, "COMMIT")
6062
}
63+
64+
// This test verifies that restore cleanup does not fail due to dropped
65+
// temporary system tables as described in #148088.
66+
func TestFailAfterCleanupSystemTables(t *testing.T) {
67+
defer leaktest.AfterTest(t)()
68+
defer log.Scope(t).Close(t)
69+
70+
clusterSize := 1
71+
_, sqlDB, _, cleanupFn := backuptestutils.StartBackupRestoreTestCluster(t, clusterSize)
72+
defer cleanupFn()
73+
74+
// Must set cluster setting before backup to ensure the setting is preserved.
75+
sqlDB.Exec(
76+
t, "SET CLUSTER SETTING jobs.debug.pausepoints = 'restore.after_cleanup_temp_system_tables'",
77+
)
78+
sqlDB.Exec(t, "BACKUP INTO 'nodelocal://1/backup'")
79+
80+
var jobID jobspb.JobID
81+
sqlDB.QueryRow(t, "RESTORE FROM LATEST IN 'nodelocal://1/backup' WITH detached").Scan(&jobID)
82+
sqlDB.Exec(t, "USE system")
83+
jobutils.WaitForJobToPause(t, sqlDB, jobID)
84+
85+
sqlDB.Exec(t, "CANCEL JOB $1", jobID)
86+
jobutils.WaitForJobToCancel(t, sqlDB, jobID)
87+
}

0 commit comments

Comments
 (0)