Skip to content

Commit

Permalink
backupccl: tenants should be restored in their correct state
Browse files Browse the repository at this point in the history
Previously, all backed up tenants were unconditionally moved through
an `ADD` and then `ACTIVE` state during a cluster/tenant restore. This
behaviour appears incorrect. If the tenant was backed up in and adding or
dropped state then it should be restored in the same state as well.

This change only moves `ACTIVE` backed up tenants through an `ADD` and then
`ACTIVE` state thereby fixing this bug.

Fixes: #87915

Release note (bug fix): Cluster and tenant restores of dropped or adding tenants
would incorrectly activate those tenants during restore.
  • Loading branch information
adityamaru committed Sep 13, 2022
1 parent f7b7889 commit dcc7fbb
Show file tree
Hide file tree
Showing 3 changed files with 95 additions and 6 deletions.
30 changes: 26 additions & 4 deletions pkg/ccl/backupccl/restore_job.go
Original file line number Diff line number Diff line change
Expand Up @@ -1153,8 +1153,18 @@ func createImportingDescriptors(
return err
}
for _, tenant := range details.Tenants {
// Mark the tenant info as adding.
tenant.State = descpb.TenantInfo_ADD
switch tenant.State {
case descpb.TenantInfo_ACTIVE:
// If the tenant was backed up in an `ACTIVE` state then we create
// the restored record in an `ADDING` state and mark it `ACTIVE` at
// the end of the restore.
tenant.State = descpb.TenantInfo_ADD
case descpb.TenantInfo_DROP, descpb.TenantInfo_ADD:
// If the tenant was backed up in a `DROP` or `ADD` state then we must
// create the restored tenant record in that state as well.
default:
return errors.AssertionFailedf("unknown tenant state %v", tenant)
}
if err := sql.CreateTenantRecord(ctx, p.ExecCfg(), txn, &tenant, initialTenantZoneConfig); err != nil {
return err
}
Expand Down Expand Up @@ -2042,8 +2052,20 @@ func (r *restoreResumer) publishDescriptors(
}

for _, tenant := range details.Tenants {
if err := sql.ActivateTenant(ctx, r.execCfg, txn, tenant.ID); err != nil {
return err
switch tenant.State {
case descpb.TenantInfo_ACTIVE:
// If the tenant was backed up in an `ACTIVE` state then we must activate
// the tenant as the final step of the restore. The tenant has already
// been created at an earlier stage in the restore in an `ADD` state.
tenant.State = descpb.TenantInfo_ADD
if err := sql.ActivateTenant(ctx, r.execCfg, txn, tenant.ID); err != nil {
return err
}
case descpb.TenantInfo_DROP, descpb.TenantInfo_ADD:
// If the tenant was backed up in a `DROP` or `ADD` state then we do not
// want to activate the tenant.
default:
return errors.AssertionFailedf("unknown tenant state %v", tenant)
}
}

Expand Down
11 changes: 9 additions & 2 deletions pkg/ccl/backupccl/targets.go
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/cockroach/pkg/util/tracing"
"github.com/cockroachdb/errors"
)

Expand Down Expand Up @@ -276,14 +277,18 @@ func fullClusterTargets(
}

func fullClusterTargetsRestore(
allDescs []catalog.Descriptor, lastBackupManifest backuppb.BackupManifest,
ctx context.Context, allDescs []catalog.Descriptor, lastBackupManifest backuppb.BackupManifest,
) (
[]catalog.Descriptor,
[]catalog.DatabaseDescriptor,
map[tree.TablePattern]catalog.Descriptor,
[]descpb.TenantInfoWithUsage,
error,
) {
ctx, span := tracing.ChildSpan(ctx, "backupccl.fullClusterTargetsRestore")
_ = ctx // ctx is currently unused, but this new ctx should be used below in the future.
defer span.Finish()

fullClusterDescs, fullClusterDBs, err := fullClusterTargets(allDescs)
var filteredDescs []catalog.Descriptor
var filteredDBs []catalog.DatabaseDescriptor
Expand Down Expand Up @@ -354,10 +359,12 @@ func selectTargets(
[]descpb.TenantInfoWithUsage,
error,
) {
ctx, span := tracing.ChildSpan(ctx, "backupccl.selectTargets")
defer span.Finish()
allDescs, lastBackupManifest := backupinfo.LoadSQLDescsFromBackupsAtTime(backupManifests, asOf)

if descriptorCoverage == tree.AllDescriptors {
return fullClusterTargetsRestore(allDescs, lastBackupManifest)
return fullClusterTargetsRestore(ctx, allDescs, lastBackupManifest)
}

if descriptorCoverage == tree.SystemUsers {
Expand Down
60 changes: 60 additions & 0 deletions pkg/ccl/backupccl/testdata/backup-restore/restore-tenants
Original file line number Diff line number Diff line change
@@ -0,0 +1,60 @@
new-server name=s1
----

# Create a few tenants.
exec-sql
SELECT crdb_internal.create_tenant(5);
----

exec-sql
SELECT crdb_internal.create_tenant(6);
----

# Drop one of them.
exec-sql
SELECT crdb_internal.destroy_tenant(5);
----

query-sql
SELECT id,active,crdb_internal.pb_to_json('cockroach.sql.sqlbase.TenantInfo', info, true) FROM system.tenants;
----
5 false {"id": "5", "state": "DROP"}
6 true {"id": "6", "state": "ACTIVE"}

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

exec-sql expect-error-regex=(tenant 5 is not active)
BACKUP TENANT 5 INTO 'nodelocal://1/tenant5'
----
regex matches error

exec-sql
BACKUP TENANT 6 INTO 'nodelocal://1/tenant6'
----

new-server name=s2 share-io-dir=s1
----

exec-sql
RESTORE FROM LATEST IN 'nodelocal://1/cluster'
----

# A dropped tenant should be restored as an inactive tenant.
query-sql
SELECT id,active,crdb_internal.pb_to_json('cockroach.sql.sqlbase.TenantInfo', info, true) FROM system.tenants;
----
5 false {"id": "5", "state": "DROP"}
6 true {"id": "6", "state": "ACTIVE"}

exec-sql
RESTORE TENANT 6 FROM LATEST IN 'nodelocal://1/tenant6' WITH tenant = '7';
----

query-sql
SELECT id,active,crdb_internal.pb_to_json('cockroach.sql.sqlbase.TenantInfo', info, true) FROM system.tenants;
----
5 false {"id": "5", "state": "DROP"}
6 true {"id": "6", "state": "ACTIVE"}
7 true {"id": "7", "state": "ACTIVE"}

0 comments on commit dcc7fbb

Please sign in to comment.