Skip to content

Commit

Permalink
restoreccl: allow restoring system users without user ids
Browse files Browse the repository at this point in the history
During restore, when users in the backup did not have ids, they'll be given one.

Release note: None
  • Loading branch information
RichardJCai committed Aug 4, 2022
1 parent 2548cb7 commit 1bc6c6d
Show file tree
Hide file tree
Showing 35 changed files with 369 additions and 93 deletions.
1 change: 1 addition & 0 deletions pkg/ccl/backupccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -78,6 +78,7 @@ go_library(
"//pkg/sql/catalog/colinfo",
"//pkg/sql/catalog/dbdesc",
"//pkg/sql/catalog/descbuilder",
"//pkg/sql/catalog/descidgen",
"//pkg/sql/catalog/descpb",
"//pkg/sql/catalog/descs",
"//pkg/sql/catalog/ingesting",
Expand Down
129 changes: 38 additions & 91 deletions pkg/ccl/backupccl/backup_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -66,13 +66,11 @@ import (
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/security/username"
"github.com/cockroachdb/cockroach/pkg/server"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/spanconfig"
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/bootstrap"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/desctestutils"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema"
Expand Down Expand Up @@ -1071,17 +1069,14 @@ func TestBackupRestoreSystemTables(t *testing.T) {

// At the time this test was written, these were the only system tables that
// were reasonable for a user to backup and restore into another cluster.
tables := []string{"locations", "role_members", "users", "zones", "role_id_seq"}
tables := []string{"locations", "role_members", "users", "zones"}
tableSpec := "system." + strings.Join(tables, ", system.")

// Take a consistent fingerprint of the original tables.
var backupAsOf string
expectedFingerprints := map[string][][]string{}
err := crdb.ExecuteTx(ctx, conn, nil /* txopts */, func(tx *gosql.Tx) error {
for _, table := range tables {
if table == "role_id_seq" {
continue
}
rows, err := conn.Query("SHOW EXPERIMENTAL_FINGERPRINTS FROM TABLE system." + table)
if err != nil {
return err
Expand All @@ -1107,9 +1102,6 @@ func TestBackupRestoreSystemTables(t *testing.T) {

// Verify the fingerprints match.
for _, table := range tables {
if table == "role_id_seq" {
continue
}
a := sqlDB.QueryStr(t, "SHOW EXPERIMENTAL_FINGERPRINTS FROM TABLE system_new."+table)
if e := expectedFingerprints[table]; !reflect.DeepEqual(e, a) {
t.Fatalf("fingerprints between system.%[1]s and system_new.%[1]s did not match:%s\n",
Expand Down Expand Up @@ -9555,7 +9547,6 @@ func TestExcludeDataFromBackupDoesNotHoldupGC(t *testing.T) {
func TestBackupRestoreSystemUsers(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
skip.WithIssue(t, 78963)

sqlDB, tempDir, cleanupFn := createEmptyCluster(t, singleNode)
_, sqlDBRestore, cleanupEmptyCluster := backupRestoreTestSetupEmpty(t, singleNode, tempDir, InitManualReplication, base.TestClusterArgs{})
Expand Down Expand Up @@ -9616,18 +9607,17 @@ func TestBackupRestoreSystemUsers(t *testing.T) {
defer cleanupEmptyCluster1()
t.Run("restore-from-backup-with-no-system-role-members", func(t *testing.T) {
sqlDBRestore1.Exec(t, "RESTORE SYSTEM USERS FROM $1", localFoo+"/3")

sqlDBRestore1.CheckQueryResults(t, "SELECT username, \"hashedPassword\", \"isRole\" FROM system.users", [][]string{
{"admin", "", "true"},
{"app", "NULL", "false"},
{"app_role", "NULL", "true"},
{"root", "", "false"},
{"test", "NULL", "false"},
{"test_role", "NULL", "true"},
})
sqlDBRestore1.CheckQueryResults(t, "SELECT \"role\", \"member\", \"isAdmin\" FROM system.role_members", [][]string{
{"admin", "root", "true"},
})
sqlDBRestore1.CheckQueryResults(t, "SELECT username, \"hashedPassword\", \"isRole\", \"user_id\" FROM system.users", [][]string{
{"admin", "", "true", "2"},
{"app", "NULL", "false", "100"},
{"app_role", "NULL", "true", "101"},
{"root", "", "false", "1"},
{"test", "NULL", "false", "102"},
{"test_role", "NULL", "true", "103"},
})
sqlDBRestore1.CheckQueryResults(t, "SHOW USERS", [][]string{
{"admin", "", "{}"},
{"app", "", "{}"},
Expand All @@ -9637,6 +9627,35 @@ func TestBackupRestoreSystemUsers(t *testing.T) {
{"test_role", "", "{}"},
})
})
_, sqlDBRestore2, cleanupEmptyCluster2 := backupRestoreTestSetupEmpty(t, singleNode, tempDir, InitManualReplication, base.TestClusterArgs{})
defer cleanupEmptyCluster2()
t.Run("restore-from-backup-with-existing-user", func(t *testing.T) {
// Create testuser and verify that the system user ids are
// allocated properly in the restore.
sqlDBRestore2.Exec(t, "CREATE USER testuser")
sqlDBRestore2.Exec(t, "RESTORE SYSTEM USERS FROM $1", localFoo+"/3")
sqlDBRestore2.CheckQueryResults(t, "SELECT \"role\", \"member\", \"isAdmin\" FROM system.role_members", [][]string{
{"admin", "root", "true"},
})
sqlDBRestore2.CheckQueryResults(t, "SELECT username, \"hashedPassword\", \"isRole\", \"user_id\" FROM system.users", [][]string{
{"admin", "", "true", "2"},
{"app", "NULL", "false", "101"},
{"app_role", "NULL", "true", "102"},
{"root", "", "false", "1"},
{"test", "NULL", "false", "103"},
{"test_role", "NULL", "true", "104"},
{"testuser", "NULL", "false", "100"},
})
sqlDBRestore2.CheckQueryResults(t, "SHOW USERS", [][]string{
{"admin", "", "{}"},
{"app", "", "{}"},
{"app_role", "", "{}"},
{"root", "", "{admin}"},
{"test", "", "{}"},
{"test_role", "", "{}"},
{"testuser", "", "{}"},
})
})
}

// TestUserfileNormalizationIncrementalShowBackup tests to see that file
Expand Down Expand Up @@ -10157,78 +10176,6 @@ func TestBackupNoOverwriteLatest(t *testing.T) {
require.NotEqual(t, firstLatest, thirdLatest)
}

// TestBackupLatestInBaseDirectory tests to see that a LATEST
// file in the base directory can be properly read when one is not found
// in metadata/latest. This can occur when an older version node creates
// the backup.
func TestBackupLatestInBaseDirectory(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

disableUpgradeCh := make(chan struct{})
const numAccounts = 1
const userfile = "'userfile:///a'"
args := base.TestClusterArgs{
ServerArgs: base.TestServerArgs{
Knobs: base.TestingKnobs{
Server: &server.TestingKnobs{
BinaryVersionOverride: clusterversion.ByKey(clusterversion.BackupDoesNotOverwriteLatestAndCheckpoint - 1),
DisableAutomaticVersionUpgrade: disableUpgradeCh,
},
},
},
}

tc, sqlDB, _, cleanupFn := backupRestoreTestSetupWithParams(t, singleNode, numAccounts, InitManualReplication, args)
defer cleanupFn()
execCfg := tc.Server(0).ExecutorConfig().(sql.ExecutorConfig)
ctx := context.Background()
store, err := execCfg.DistSQLSrv.ExternalStorageFromURI(ctx, "userfile:///a", username.RootUserName())
require.NoError(t, err)

query := fmt.Sprintf("BACKUP INTO %s", userfile)
sqlDB.Exec(t, query)

// Confirm that the LATEST file was written to the base directory.
r, err := store.ReadFile(ctx, backupbase.LatestFileName)
require.NoError(t, err)
r.Close(ctx)

// Drop the system.role_seq_id so that we can perform the migration.
sqlDB.Exec(t, `INSERT INTO system.users VALUES ('node', '', false, 0)`)
sqlDB.Exec(t, `GRANT node TO root`)
sqlDB.Exec(t, `DROP SEQUENCE system.role_id_seq`)
sqlDB.Exec(t, `REVOKE node FROM root`)

err = tc.Servers[0].DB().Del(ctx, catalogkeys.MakeDescMetadataKey(keys.SystemSQLCodec, keys.RoleIDSequenceID))
require.NoError(t, err)
err = tc.Servers[0].DB().Del(ctx, keys.SystemSQLCodec.SequenceKey(uint32(keys.RoleIDSequenceID)))
require.NoError(t, err)

// Close the channel so that the cluster version is upgraded.
close(disableUpgradeCh)
// Check the cluster version is bumped to newVersion.
testutils.SucceedsSoon(t, func() error {
var version string
sqlDB.QueryRow(t, "SELECT value FROM system.settings WHERE name = 'version'").Scan(&version)
var v clusterversion.ClusterVersion
if err := protoutil.Unmarshal([]byte(version), &v); err != nil {
return err
}
version = v.String()
if version != clusterversion.TestingBinaryVersion.String() {
return errors.Errorf("cluster version is still %s, should be %s", version, clusterversion.TestingBinaryVersion.String())
}
return nil
})

// Take an incremental backup on the new version using the latest file
// written by the old version in the base directory.
query = fmt.Sprintf("BACKUP INTO LATEST IN %s", userfile)
sqlDB.Exec(t, query)

}

// TestBackupRestoreTelemetryEvents tests that BACKUP and RESTORE correctly
// publishes telemetry events.
func TestBackupRestoreTelemetryEvents(t *testing.T) {
Expand Down
48 changes: 48 additions & 0 deletions pkg/ccl/backupccl/full_cluster_backup_restore_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1112,3 +1112,51 @@ DROP DATABASE defaultdb;
{fmt.Sprint(parentID), fmt.Sprint(parentSchemaID), name, fmt.Sprint(ID)},
})
}

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

params := base.TestClusterArgs{
ServerArgs: base.TestServerArgs{
Knobs: base.TestingKnobs{
JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(),
},
},
}
const numAccounts = 10
_, sqlDB, tempDir, cleanupFn := backupRestoreTestSetupWithParams(t, singleNode, numAccounts, InitManualReplication, params)
_, sqlDBRestore, cleanupEmptyCluster := backupRestoreTestSetupEmpty(t, singleNode, tempDir, InitManualReplication, params)
defer cleanupFn()
defer cleanupEmptyCluster()

sqlDB.Exec(t, `CREATE USER test1`)
sqlDB.Exec(t, `CREATE USER test2`)
sqlDB.Exec(t, `BACKUP TO $1`, localFoo)

sqlDB.CheckQueryResults(t, `SELECT * FROM system.users ORDER BY user_id`, [][]string{
{"root", "", "false", "1"},
{"admin", "", "true", "2"},
{"test1", "NULL", "false", "100"},
{"test2", "NULL", "false", "101"},
})
// Ensure that the new backup succeeds.
sqlDBRestore.Exec(t, `RESTORE FROM $1`, localFoo)

sqlDBRestore.CheckQueryResults(t, `SELECT * FROM system.users ORDER BY user_id`, [][]string{
{"root", "", "false", "1"},
{"admin", "", "true", "2"},
{"test1", "NULL", "false", "100"},
{"test2", "NULL", "false", "101"},
})

sqlDBRestore.Exec(t, `CREATE USER test3`)

sqlDBRestore.CheckQueryResults(t, `SELECT * FROM system.users ORDER BY user_id`, [][]string{
{"root", "", "false", "1"},
{"admin", "", "true", "2"},
{"test1", "NULL", "false", "100"},
{"test2", "NULL", "false", "101"},
{"test3", "NULL", "false", "102"},
})
}
17 changes: 16 additions & 1 deletion pkg/ccl/backupccl/restore_job.go
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/dbdesc"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descidgen"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descs"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/ingesting"
Expand Down Expand Up @@ -2400,11 +2401,25 @@ func (r *restoreResumer) restoreSystemUsers(
}

insertUser := `INSERT INTO system.users ("username", "hashedPassword", "isRole") VALUES ($1, $2, $3)`
if r.execCfg.Settings.Version.IsActive(ctx, clusterversion.AddSystemUserIDColumn) {
insertUser = `INSERT INTO system.users ("username", "hashedPassword", "isRole", "user_id") VALUES ($1, $2, $3, $4)`
}
newUsernames := make(map[string]bool)
args := make([]interface{}, 4)
for _, user := range users {
newUsernames[user[0].String()] = true
args[0] = user[0]
args[1] = user[1]
args[2] = user[2]
if r.execCfg.Settings.Version.IsActive(ctx, clusterversion.AddSystemUserIDColumn) {
id, err := descidgen.GenerateUniqueRoleID(ctx, r.execCfg.DB, r.execCfg.Codec)
if err != nil {
return err
}
args[3] = id
}
if _, err = executor.Exec(ctx, "insert-non-existent-users", txn, insertUser,
user[0], user[1], user[2]); err != nil {
args...); err != nil {
return err
}
}
Expand Down
Loading

0 comments on commit 1bc6c6d

Please sign in to comment.