From 6f695f38c4fbb7aa07ce4c4c3d9a90fa89546574 Mon Sep 17 00:00:00 2001 From: Casper Date: Wed, 13 Oct 2021 16:03:09 -0400 Subject: [PATCH 1/5] backupccl: Support RESTORE SYSTEM USERS from a backup Support a new variant of RESTORE that recreates system users that don't exist in current cluster from a backup that contains system.users and also grant roles for these users. Example invocation: RESTORE SYSTEM USERS FROM 'nodelocal://foo/1'; Similar with full cluster restore, we firstly restore a temp system database which contains system.users and system.role_members into the restoring cluster and insert users and roles into the current system table from the temp system table. Fixes: #45358 Release note (sql change): A special flavor of RESTORE, RESTORE SYSTEM USERS FROM ..., is added to support restoring system users from a backup. When executed, the statement recreates those users which are in a backup of system.users but do not currently exist (ignoring those who do) and re-grant roles for users if the backup contains system.role_members. --- docs/generated/sql/bnf/restore.bnf | 12 ++++ docs/generated/sql/bnf/stmt_block.bnf | 2 + pkg/ccl/backupccl/backup_test.go | 88 +++++++++++++++++++++++++++ pkg/ccl/backupccl/restore_job.go | 64 ++++++++++++++++++- pkg/ccl/backupccl/restore_planning.go | 81 ++++++++++++++---------- pkg/ccl/backupccl/targets.go | 29 ++++++++- pkg/jobs/jobspb/jobs.proto | 3 +- pkg/sql/parser/sql.y | 23 +++++++ pkg/sql/sem/tree/backup.go | 4 +- pkg/sql/sem/tree/grant.go | 2 + 10 files changed, 271 insertions(+), 37 deletions(-) diff --git a/docs/generated/sql/bnf/restore.bnf b/docs/generated/sql/bnf/restore.bnf index fc480ff4ecb8..d907009fe1a9 100644 --- a/docs/generated/sql/bnf/restore.bnf +++ b/docs/generated/sql/bnf/restore.bnf @@ -23,3 +23,15 @@ restore_stmt ::= | 'RESTORE' ( 'TABLE' table_pattern ( ( ',' table_pattern ) )* | 'DATABASE' database_name ( ( ',' database_name ) )* ) 'FROM' subdirectory 'IN' ( destination | '(' partitioned_backup_location ( ',' partitioned_backup_location )* ')' ) 'WITH' restore_options_list | 'RESTORE' ( 'TABLE' table_pattern ( ( ',' table_pattern ) )* | 'DATABASE' database_name ( ( ',' database_name ) )* ) 'FROM' subdirectory 'IN' ( destination | '(' partitioned_backup_location ( ',' partitioned_backup_location )* ')' ) 'WITH' 'OPTIONS' '(' restore_options_list ')' | 'RESTORE' ( 'TABLE' table_pattern ( ( ',' table_pattern ) )* | 'DATABASE' database_name ( ( ',' database_name ) )* ) 'FROM' subdirectory 'IN' ( destination | '(' partitioned_backup_location ( ',' partitioned_backup_location )* ')' ) + | 'RESTORE' 'SYSTEM' 'USERS' 'FROM' ( destination | '(' partitioned_backup_location ( ',' partitioned_backup_location )* ')' ) 'AS' 'OF' 'SYSTEM' 'TIME' timestamp 'WITH' restore_options_list + | 'RESTORE' 'SYSTEM' 'USERS' 'FROM' ( destination | '(' partitioned_backup_location ( ',' partitioned_backup_location )* ')' ) 'AS' 'OF' 'SYSTEM' 'TIME' timestamp 'WITH' 'OPTIONS' '(' restore_options_list ')' + | 'RESTORE' 'SYSTEM' 'USERS' 'FROM' ( destination | '(' partitioned_backup_location ( ',' partitioned_backup_location )* ')' ) 'AS' 'OF' 'SYSTEM' 'TIME' timestamp + | 'RESTORE' 'SYSTEM' 'USERS' 'FROM' ( destination | '(' partitioned_backup_location ( ',' partitioned_backup_location )* ')' ) 'WITH' restore_options_list + | 'RESTORE' 'SYSTEM' 'USERS' 'FROM' ( destination | '(' partitioned_backup_location ( ',' partitioned_backup_location )* ')' ) 'WITH' 'OPTIONS' '(' restore_options_list ')' + | 'RESTORE' 'SYSTEM' 'USERS' 'FROM' ( destination | '(' partitioned_backup_location ( ',' partitioned_backup_location )* ')' ) + | 'RESTORE' 'SYSTEM' 'USERS' 'FROM' subdirectory 'IN' ( destination | '(' partitioned_backup_location ( ',' partitioned_backup_location )* ')' ) 'AS' 'OF' 'SYSTEM' 'TIME' timestamp 'WITH' restore_options_list + | 'RESTORE' 'SYSTEM' 'USERS' 'FROM' subdirectory 'IN' ( destination | '(' partitioned_backup_location ( ',' partitioned_backup_location )* ')' ) 'AS' 'OF' 'SYSTEM' 'TIME' timestamp 'WITH' 'OPTIONS' '(' restore_options_list ')' + | 'RESTORE' 'SYSTEM' 'USERS' 'FROM' subdirectory 'IN' ( destination | '(' partitioned_backup_location ( ',' partitioned_backup_location )* ')' ) 'AS' 'OF' 'SYSTEM' 'TIME' timestamp + | 'RESTORE' 'SYSTEM' 'USERS' 'FROM' subdirectory 'IN' ( destination | '(' partitioned_backup_location ( ',' partitioned_backup_location )* ')' ) 'WITH' restore_options_list + | 'RESTORE' 'SYSTEM' 'USERS' 'FROM' subdirectory 'IN' ( destination | '(' partitioned_backup_location ( ',' partitioned_backup_location )* ')' ) 'WITH' 'OPTIONS' '(' restore_options_list ')' + | 'RESTORE' 'SYSTEM' 'USERS' 'FROM' subdirectory 'IN' ( destination | '(' partitioned_backup_location ( ',' partitioned_backup_location )* ')' ) diff --git a/docs/generated/sql/bnf/stmt_block.bnf b/docs/generated/sql/bnf/stmt_block.bnf index cb000bfc2ebc..16669083c307 100644 --- a/docs/generated/sql/bnf/stmt_block.bnf +++ b/docs/generated/sql/bnf/stmt_block.bnf @@ -188,6 +188,8 @@ restore_stmt ::= | 'RESTORE' 'FROM' string_or_placeholder 'IN' list_of_string_or_placeholder_opt_list opt_as_of_clause opt_with_restore_options | 'RESTORE' targets 'FROM' list_of_string_or_placeholder_opt_list opt_as_of_clause opt_with_restore_options | 'RESTORE' targets 'FROM' string_or_placeholder 'IN' list_of_string_or_placeholder_opt_list opt_as_of_clause opt_with_restore_options + | 'RESTORE' 'SYSTEM' 'USERS' 'FROM' list_of_string_or_placeholder_opt_list opt_as_of_clause opt_with_restore_options + | 'RESTORE' 'SYSTEM' 'USERS' 'FROM' string_or_placeholder 'IN' list_of_string_or_placeholder_opt_list opt_as_of_clause opt_with_restore_options | 'RESTORE' targets 'FROM' 'REPLICATION' 'STREAM' 'FROM' string_or_placeholder_opt_list opt_as_of_clause resume_stmt ::= diff --git a/pkg/ccl/backupccl/backup_test.go b/pkg/ccl/backupccl/backup_test.go index f2aa7b36b05c..3790c2e53313 100644 --- a/pkg/ccl/backupccl/backup_test.go +++ b/pkg/ccl/backupccl/backup_test.go @@ -9456,3 +9456,91 @@ func TestExportRequestBelowGCThresholdOnDataExcludedFromBackup(t *testing.T) { _, err = conn.Exec(fmt.Sprintf("BACKUP TABLE foo TO $1 AS OF SYSTEM TIME '%s'", tsBefore), localFoo) require.NoError(t, err) } + +// TestBackupRestoreSystemUsers tests RESTORE SYSTEM USERS feature which allows user to +// restore users from a backup into current cluster and regrant roles. +func TestBackupRestoreSystemUsers(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + sqlDB, tempDir, cleanupFn := createEmptyCluster(t, singleNode) + _, sqlDBRestore, cleanupEmptyCluster := backupRestoreTestSetupEmpty(t, singleNode, tempDir, InitManualReplication, base.TestClusterArgs{}) + defer cleanupFn() + defer cleanupEmptyCluster() + + sqlDB.Exec(t, `CREATE USER app; CREATE USER test`) + sqlDB.Exec(t, `CREATE ROLE app_role; CREATE ROLE test_role`) + sqlDB.Exec(t, `GRANT app_role TO test_role;`) // 'test_role' is a member of 'app_role' + sqlDB.Exec(t, `GRANT admin, app_role TO app; GRANT test_role TO test`) + sqlDB.Exec(t, `CREATE DATABASE db; CREATE TABLE db.foo (ind INT)`) + sqlDB.Exec(t, `BACKUP TO $1`, localFoo+"/1") + sqlDB.Exec(t, `BACKUP DATABASE db TO $1`, localFoo+"/2") + sqlDB.Exec(t, `BACKUP TABLE system.users TO $1`, localFoo+"/3") + + // User 'test' exists in both clusters but 'app' only exists in the backup + sqlDBRestore.Exec(t, `CREATE USER test`) + sqlDBRestore.Exec(t, `CREATE DATABASE db`) + // Create multiple databases to make max descriptor ID be larger than max descriptor ID + // in the backup to test if we correctly generate new descriptor IDs + sqlDBRestore.Exec(t, `CREATE DATABASE db1; CREATE DATABASE db2; CREATE DATABASE db3`) + + t.Run("system users", func(t *testing.T) { + sqlDBRestore.Exec(t, "RESTORE SYSTEM USERS FROM $1", localFoo+"/1") + + // Role 'app_role' and user 'app' will be added, and 'app' is granted with 'app_role' + // User test will remain untouched with no role granted + sqlDBRestore.CheckQueryResults(t, "SELECT * FROM system.users", [][]string{ + {"admin", "", "true"}, + {"app", "NULL", "false"}, + {"app_role", "NULL", "true"}, + {"root", "", "false"}, + {"test", "NULL", "false"}, + {"test_role", "NULL", "true"}, + }) + sqlDBRestore.CheckQueryResults(t, "SELECT * FROM system.role_members", [][]string{ + {"admin", "app", "false"}, + {"admin", "root", "true"}, + {"app_role", "app", "false"}, + {"app_role", "test_role", "false"}, + }) + sqlDBRestore.CheckQueryResults(t, "SHOW USERS", [][]string{ + {"admin", "", "{}"}, + {"app", "", "{admin,app_role}"}, + {"app_role", "", "{}"}, + {"root", "", "{admin}"}, + {"test", "", "{}"}, + {"test_role", "", "{app_role}"}, + }) + }) + + t.Run("restore-from-backup-with-no-system-users", func(t *testing.T) { + sqlDBRestore.ExpectErr(t, "cannot restore system users as no system.users table in the backup", + "RESTORE SYSTEM USERS FROM $1", localFoo+"/2") + }) + + _, sqlDBRestore1, cleanupEmptyCluster1 := backupRestoreTestSetupEmpty(t, singleNode, tempDir, InitManualReplication, base.TestClusterArgs{}) + 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 * 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 * FROM system.role_members", [][]string{ + {"admin", "root", "true"}, + }) + sqlDBRestore1.CheckQueryResults(t, "SHOW USERS", [][]string{ + {"admin", "", "{}"}, + {"app", "", "{}"}, + {"app_role", "", "{}"}, + {"root", "", "{admin}"}, + {"test", "", "{}"}, + {"test_role", "", "{}"}, + }) + }) +} diff --git a/pkg/ccl/backupccl/restore_job.go b/pkg/ccl/backupccl/restore_job.go index 28bf6cff574f..318079f05bff 100644 --- a/pkg/ccl/backupccl/restore_job.go +++ b/pkg/ccl/backupccl/restore_job.go @@ -1627,6 +1627,15 @@ func (r *restoreResumer) doResume(ctx context.Context, execCtx interface{}) erro // Reload the details as we may have updated the job. details = r.job.Details().(jobspb.RestoreDetails) + if err := r.cleanupTempSystemTables(ctx, nil /* txn */); err != nil { + return err + } + } else if details.RestoreSystemUsers { + if err := r.restoreSystemUsers(ctx, p.ExecCfg().DB, mainData.systemTables); err != nil { + return err + } + details = r.job.Details().(jobspb.RestoreDetails) + if err := r.cleanupTempSystemTables(ctx, nil /* txn */); err != nil { return err } @@ -1784,7 +1793,7 @@ func (r *restoreResumer) notifyStatsRefresherOfNewTables() { // This is the last of the IDs pre-allocated by the restore planner. // TODO(postamar): Store it directly in the details instead? This is brittle. func tempSystemDatabaseID(details jobspb.RestoreDetails) descpb.ID { - if details.DescriptorCoverage != tree.AllDescriptors { + if details.DescriptorCoverage != tree.AllDescriptors && !details.RestoreSystemUsers { return descpb.InvalidID } var maxPreAllocatedID descpb.ID @@ -2520,6 +2529,59 @@ type systemTableNameWithConfig struct { config systemBackupConfiguration } +// Restore system.users from the backup into the restoring cluster. Only recreate users +// which are in a backup of system.users but do not currently exist (ignoring those who do) +// and re-grant roles for users if the backup has system.role_members. +func (r *restoreResumer) restoreSystemUsers( + ctx context.Context, db *kv.DB, systemTables []catalog.TableDescriptor, +) error { + executor := r.execCfg.InternalExecutor + return db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + selectNonExistentUsers := "SELECT * FROM crdb_temp_system.users temp " + + "WHERE NOT EXISTS (SELECT * FROM system.users u WHERE temp.username = u.username)" + users, err := executor.QueryBuffered(ctx, "get-users", + txn, selectNonExistentUsers) + if err != nil { + return err + } + + insertUser := `INSERT INTO system.users ("username", "hashedPassword", "isRole") VALUES ($1, $2, $3)` + newUsernames := make(map[string]bool) + for _, user := range users { + newUsernames[user[0].String()] = true + if _, err = executor.Exec(ctx, "insert-non-existent-users", txn, insertUser, + user[0], user[1], user[2]); err != nil { + return err + } + } + + // We skip granting roles if the backup does not contain system.role_members. + if len(systemTables) == 1 { + return nil + } + + selectNonExistentRoleMembers := "SELECT * FROM crdb_temp_system.role_members temp_rm WHERE " + + "NOT EXISTS (SELECT * FROM system.role_members rm WHERE temp_rm.role = rm.role AND temp_rm.member = rm.member)" + roleMembers, err := executor.QueryBuffered(ctx, "get-role-members", + txn, selectNonExistentRoleMembers) + if err != nil { + return err + } + + insertRoleMember := `INSERT INTO system.role_members ("role", "member", "isAdmin") VALUES ($1, $2, $3)` + for _, roleMember := range roleMembers { + // Only grant roles to users that don't currently exist, i.e., new users we just added + if _, ok := newUsernames[roleMember[1].String()]; ok { + if _, err = executor.Exec(ctx, "insert-non-existent-role-members", txn, insertRoleMember, + roleMember[0], roleMember[1], roleMember[2]); err != nil { + return err + } + } + } + return nil + }) +} + // restoreSystemTables atomically replaces the contents of the system tables // with the data from the restored system tables. func (r *restoreResumer) restoreSystemTables( diff --git a/pkg/ccl/backupccl/restore_planning.go b/pkg/ccl/backupccl/restore_planning.go index 63637bc6150f..0af7b208c3be 100644 --- a/pkg/ccl/backupccl/restore_planning.go +++ b/pkg/ccl/backupccl/restore_planning.go @@ -308,6 +308,7 @@ func allocateDescriptorRewrites( opts tree.RestoreOptions, intoDB string, newDBName string, + restoreSystemUsers bool, ) (DescRewriteMap, error) { descriptorRewrites := make(DescRewriteMap) @@ -419,38 +420,47 @@ func allocateDescriptorRewrites( } needsNewParentIDs := make(map[string][]descpb.ID) - // Increment the DescIDSequenceKey so that it is higher than the max desc ID - // in the backup. This generator keeps produced the next descriptor ID. + + // Increment the DescIDSequenceKey so that it is higher than both the max desc ID + // in the backup and current max desc ID in the restoring cluster. This generator + // keeps produced the next descriptor ID. var tempSysDBID descpb.ID - if descriptorCoverage == tree.AllDescriptors { + if descriptorCoverage == tree.AllDescriptors || restoreSystemUsers { var err error - // Restore the key which generates descriptor IDs. - if err = p.ExecCfg().DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - v, err := txn.Get(ctx, p.ExecCfg().Codec.DescIDSequenceKey()) + if descriptorCoverage == tree.AllDescriptors { + // Restore the key which generates descriptor IDs. + if err = p.ExecCfg().DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + v, err := txn.Get(ctx, p.ExecCfg().Codec.DescIDSequenceKey()) + if err != nil { + return err + } + newValue := maxDescIDInBackup + 1 + if newValue <= v.ValueInt() { + // This case may happen when restoring backups from older versions. + newValue = v.ValueInt() + 1 + } + b := txn.NewBatch() + // N.B. This key is usually mutated using the Inc command. That + // command warns that if the key was every Put directly, Inc will + // return an error. This is only to ensure that the type of the key + // doesn't change. Here we just need to be very careful that we only + // write int64 values. + // The generator's value should be set to the value of the next ID + // to generate. + b.Put(p.ExecCfg().Codec.DescIDSequenceKey(), newValue) + return txn.Run(ctx, b) + }); err != nil { + return nil, err + } + tempSysDBID, err = descidgen.GenerateUniqueDescID(ctx, p.ExecCfg().DB, p.ExecCfg().Codec) if err != nil { - return err + return nil, err + } + } else if restoreSystemUsers { + tempSysDBID, err = descidgen.GenerateUniqueDescID(ctx, p.ExecCfg().DB, p.ExecCfg().Codec) + if err != nil { + return nil, err } - newValue := maxDescIDInBackup + 1 - if newValue <= v.ValueInt() { - // This case may happen when restoring backups from older versions. - newValue = v.ValueInt() + 1 - } - b := txn.NewBatch() - // N.B. This key is usually mutated using the Inc command. That - // command warns that if the key was every Put directly, Inc will - // return an error. This is only to ensure that the type of the key - // doesn't change. Here we just need to be very careful that we only - // write int64 values. - // The generator's value should be set to the value of the next ID - // to generate. - b.Put(p.ExecCfg().Codec.DescIDSequenceKey(), newValue) - return txn.Run(ctx, b) - }); err != nil { - return nil, err - } - tempSysDBID, err = descidgen.GenerateUniqueDescID(ctx, p.ExecCfg().DB, p.ExecCfg().Codec) - if err != nil { - return nil, err } // Remap all of the descriptor belonging to system tables to the temp system // DB. @@ -829,7 +839,7 @@ func allocateDescriptorRewrites( // backup should have the same ID as they do in the backup. descriptorsToRemap := make([]catalog.Descriptor, 0, len(tablesByID)) for _, table := range tablesByID { - if descriptorCoverage == tree.AllDescriptors { + if descriptorCoverage == tree.AllDescriptors || restoreSystemUsers { if table.ParentID == systemschema.SystemDB.GetID() { // This is a system table that should be marked for descriptor creation. descriptorsToRemap = append(descriptorsToRemap, table) @@ -1429,6 +1439,7 @@ func restoreJobDescription( kmsURIs []string, ) (string, error) { r := &tree.Restore{ + SystemUsers: restore.SystemUsers, DescriptorCoverage: restore.DescriptorCoverage, AsOf: restore.AsOf, Targets: restore.Targets, @@ -1505,6 +1516,9 @@ func restorePlanHook( var intoDBFn func() (string, error) if restoreStmt.Options.IntoDB != nil { + if restoreStmt.SystemUsers { + return nil, nil, nil, false, errors.New("cannot set into_db option when only restoring system users") + } intoDBFn, err = p.TypeAsString(ctx, restoreStmt.Options.IntoDB, "RESTORE") if err != nil { return nil, nil, nil, false, err @@ -1540,6 +1554,9 @@ func restorePlanHook( " database") return nil, nil, nil, false, err } + if restoreStmt.SystemUsers { + return nil, nil, nil, false, errors.New("cannot set new_db_name option when only restoring system users") + } newDBNameFn, err = p.TypeAsString(ctx, restoreStmt.Options.NewDBName, "RESTORE") if err != nil { return nil, nil, nil, false, err @@ -1926,7 +1943,7 @@ func doRestorePlan( } sqlDescs, restoreDBs, tenants, err := selectTargets( - ctx, p, mainBackupManifests, restoreStmt.Targets, restoreStmt.DescriptorCoverage, endTime, + ctx, p, mainBackupManifests, restoreStmt.Targets, restoreStmt.DescriptorCoverage, endTime, restoreStmt.SystemUsers, ) if err != nil { return errors.Wrap(err, @@ -2054,7 +2071,8 @@ func doRestorePlan( restoreStmt.DescriptorCoverage, restoreStmt.Options, intoDB, - newDBName) + newDBName, + restoreStmt.SystemUsers) if err != nil { return err } @@ -2137,6 +2155,7 @@ func doRestorePlan( RevalidateIndexes: revalidateIndexes, DatabaseModifiers: databaseModifiers, DebugPauseOn: debugPauseOn, + RestoreSystemUsers: restoreStmt.SystemUsers, }, Progress: jobspb.RestoreProgress{}, } diff --git a/pkg/ccl/backupccl/targets.go b/pkg/ccl/backupccl/targets.go index bfd113828a22..30e8548207f1 100644 --- a/pkg/ccl/backupccl/targets.go +++ b/pkg/ccl/backupccl/targets.go @@ -318,9 +318,10 @@ func fullClusterTargetsBackup( return fullClusterDescs, fullClusterDBIDs, nil } -// selectTargets loads all descriptors from the selected backup manifest(s), and -// filters the descriptors based on the targets specified in the restore. Post -// filtering, the method returns: +// selectTargets loads all descriptors from the selected backup manifest(s), +// filters the descriptors based on the targets specified in the restore, and +// calculates the max descriptor ID in the backup. +// Post filtering, the method returns: // - A list of all descriptors (table, type, database, schema) along with their // parent databases. // - A list of database descriptors IFF the user is restoring on the cluster or @@ -333,6 +334,7 @@ func selectTargets( targets tree.TargetList, descriptorCoverage tree.DescriptorCoverage, asOf hlc.Timestamp, + restoreSystemUsers bool, ) ([]catalog.Descriptor, []catalog.DatabaseDescriptor, []descpb.TenantInfoWithUsage, error) { allDescs, lastBackupManifest := loadSQLDescsFromBackupsAtTime(backupManifests, asOf) @@ -340,6 +342,27 @@ func selectTargets( return fullClusterTargetsRestore(allDescs, lastBackupManifest) } + if restoreSystemUsers { + systemTables := make([]catalog.Descriptor, 0) + var users catalog.Descriptor + for _, desc := range allDescs { + if desc.GetParentID() == systemschema.SystemDB.GetID() { + switch desc.GetName() { + case systemschema.UsersTable.GetName(): + users = desc + systemTables = append(systemTables, desc) + case systemschema.RoleMembersTable.GetName(): + systemTables = append(systemTables, desc) + // TODO(casper): should we handle role_options table? + } + } + } + if users == nil { + return nil, nil, nil, errors.Errorf("cannot restore system users as no system.users table in the backup") + } + return systemTables, nil, nil, nil + } + if targets.Tenant != (roachpb.TenantID{}) { for _, tenant := range lastBackupManifest.GetTenants() { // TODO(dt): for now it is zero-or-one but when that changes, we should diff --git a/pkg/jobs/jobspb/jobs.proto b/pkg/jobs/jobspb/jobs.proto index b39599593293..cad1b66d0fa5 100644 --- a/pkg/jobs/jobspb/jobs.proto +++ b/pkg/jobs/jobspb/jobs.proto @@ -317,8 +317,9 @@ message RestoreDetails { // DebugPauseOn describes the events that the job should pause itself on for debugging purposes. string debug_pause_on = 20; + bool restore_system_users = 22; - // NEXT ID: 22. + // NEXT ID: 23. } message RestoreProgress { diff --git a/pkg/sql/parser/sql.y b/pkg/sql/parser/sql.y index 834e6e8ca482..d41ab3ae7b47 100644 --- a/pkg/sql/parser/sql.y +++ b/pkg/sql/parser/sql.y @@ -2995,6 +2995,10 @@ opt_with_schedule_options: // RESTORE FROM // [ AS OF SYSTEM TIME ] // [ WITH