diff --git a/docs/generated/settings/settings-for-tenants.txt b/docs/generated/settings/settings-for-tenants.txt index 4443f195c9ee..14848a114bf1 100644 --- a/docs/generated/settings/settings-for-tenants.txt +++ b/docs/generated/settings/settings-for-tenants.txt @@ -283,4 +283,4 @@ trace.jaeger.agent string the address of a Jaeger agent to receive traces using trace.opentelemetry.collector string address of an OpenTelemetry trace collector to receive traces using the otel gRPC protocol, as :. If no port is specified, 4317 will be used. trace.span_registry.enabled boolean true if set, ongoing traces can be seen at https:///#/debug/tracez trace.zipkin.collector string the address of a Zipkin instance to receive traces, as :. If no port is specified, 9411 will be used. -version version 22.1-32 set the active cluster version in the format '.' +version version 22.1-40 set the active cluster version in the format '.' diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index 9376a4c93c4f..3de6c8914aa8 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -214,6 +214,6 @@ trace.opentelemetry.collectorstringaddress of an OpenTelemetry trace collector to receive traces using the otel gRPC protocol, as :. If no port is specified, 4317 will be used. trace.span_registry.enabledbooleantrueif set, ongoing traces can be seen at https:///#/debug/tracez trace.zipkin.collectorstringthe address of a Zipkin instance to receive traces, as :. If no port is specified, 9411 will be used. -versionversion22.1-32set the active cluster version in the format '.' +versionversion22.1-40set the active cluster version in the format '.' diff --git a/pkg/bench/rttanalysis/testdata/benchmark_expectations b/pkg/bench/rttanalysis/testdata/benchmark_expectations index 4a4dee0f5fb8..c9d14d2b734e 100644 --- a/pkg/bench/rttanalysis/testdata/benchmark_expectations +++ b/pkg/bench/rttanalysis/testdata/benchmark_expectations @@ -35,9 +35,9 @@ exp,benchmark 16,DropDatabase/drop_database_1_table 17,DropDatabase/drop_database_2_tables 18,DropDatabase/drop_database_3_tables -18,DropRole/drop_1_role -25,DropRole/drop_2_roles -32,DropRole/drop_3_roles +19,DropRole/drop_1_role +27,DropRole/drop_2_roles +35,DropRole/drop_3_roles 17,DropSequence/drop_1_sequence 19,DropSequence/drop_2_sequences 21,DropSequence/drop_3_sequences diff --git a/pkg/ccl/backupccl/BUILD.bazel b/pkg/ccl/backupccl/BUILD.bazel index feb2a7c57182..2424f7e75374 100644 --- a/pkg/ccl/backupccl/BUILD.bazel +++ b/pkg/ccl/backupccl/BUILD.bazel @@ -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", diff --git a/pkg/ccl/backupccl/backup_test.go b/pkg/ccl/backupccl/backup_test.go index 9e45e7c51a8d..ea3cdc9b95a2 100644 --- a/pkg/ccl/backupccl/backup_test.go +++ b/pkg/ccl/backupccl/backup_test.go @@ -9574,7 +9574,7 @@ func TestBackupRestoreSystemUsers(t *testing.T) { // 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{ + sqlDBRestore.CheckQueryResults(t, "SELECT username, \"hashedPassword\", \"isRole\" FROM system.users", [][]string{ {"admin", "", "true"}, {"app", "NULL", "false"}, {"app_role", "NULL", "true"}, @@ -9607,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 * 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{ + 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", "", "{}"}, @@ -9628,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 diff --git a/pkg/ccl/backupccl/full_cluster_backup_restore_test.go b/pkg/ccl/backupccl/full_cluster_backup_restore_test.go index 04ae6cce272d..7d68157ce7f4 100644 --- a/pkg/ccl/backupccl/full_cluster_backup_restore_test.go +++ b/pkg/ccl/backupccl/full_cluster_backup_restore_test.go @@ -557,7 +557,7 @@ func TestRestoreFromFullClusterBackup(t *testing.T) { t.Run("system tables", func(t *testing.T) { sqlDB.Exec(t, `CREATE DATABASE temp_sys`) - sqlDB.Exec(t, `RESTORE system.users FROM $1 WITH into_db='temp_sys'`, localFoo) + sqlDB.Exec(t, `RESTORE system.users, system.role_id_seq FROM $1 WITH into_db='temp_sys'`, localFoo) sqlDB.CheckQueryResults(t, "SELECT * FROM temp_sys.users", sqlDB.QueryStr(t, "SELECT * FROM system.users")) }) } @@ -642,6 +642,7 @@ func TestClusterRestoreFailCleanup(t *testing.T) { {"database_role_settings"}, {"external_connections"}, {"locations"}, + {"role_id_seq"}, {"role_members"}, {"role_options"}, {"scheduled_jobs"}, @@ -733,6 +734,7 @@ func TestClusterRestoreFailCleanup(t *testing.T) { {"database_role_settings"}, {"external_connections"}, {"locations"}, + {"role_id_seq"}, {"role_members"}, {"role_options"}, {"scheduled_jobs"}, @@ -1110,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"}, + }) +} diff --git a/pkg/ccl/backupccl/restore_job.go b/pkg/ccl/backupccl/restore_job.go index ba4deb83fabe..ecf9607b99f3 100644 --- a/pkg/ccl/backupccl/restore_job.go +++ b/pkg/ccl/backupccl/restore_job.go @@ -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" @@ -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 } } diff --git a/pkg/ccl/backupccl/restore_old_versions_test.go b/pkg/ccl/backupccl/restore_old_versions_test.go index 2eba95834351..2baa01871024 100644 --- a/pkg/ccl/backupccl/restore_old_versions_test.go +++ b/pkg/ccl/backupccl/restore_old_versions_test.go @@ -74,6 +74,7 @@ func TestRestoreOldVersions(t *testing.T) { privilegeDirs = testdataBase + "/privileges" multiRegionDirs = testdataBase + "/multi-region" publicSchemaDirs = testdataBase + "/public-schema-remap" + systemUsersDirs = testdataBase + "/system-users-restore" ) t.Run("table-restore", func(t *testing.T) { @@ -300,6 +301,28 @@ ORDER BY object_type, object_name`, [][]string{ t.Run(dir.Name(), restoreSyntheticPublicSchemaNamespaceEntryCleanupOnFail(exportDir)) } }) + + t.Run("system-users-restore", func(t *testing.T) { + dirs, err := ioutil.ReadDir(systemUsersDirs) + require.NoError(t, err) + for _, dir := range dirs { + require.True(t, dir.IsDir()) + exportDir, err := filepath.Abs(filepath.Join(systemUsersDirs, dir.Name())) + require.NoError(t, err) + t.Run(dir.Name(), restoreSystemUsersWithoutIDs(exportDir)) + } + }) + + t.Run("full-cluster-restore-users-without-ids", func(t *testing.T) { + dirs, err := ioutil.ReadDir(systemUsersDirs) + require.NoError(t, err) + for _, dir := range dirs { + require.True(t, dir.IsDir()) + exportDir, err := filepath.Abs(filepath.Join(systemUsersDirs, dir.Name())) + require.NoError(t, err) + t.Run(dir.Name(), fullClusterRestoreUsersWithoutIDs(exportDir)) + } + }) } func restoreOldVersionTestWithInterleave(exportDir string) func(t *testing.T) { @@ -1112,3 +1135,119 @@ func restoreSyntheticPublicSchemaNamespaceEntryCleanupOnFail(exportDir string) f sqlDB.CheckQueryResults(t, `SELECT id FROM system.namespace WHERE name = 'public' AND id=29 AND "parentID"!=1`, [][]string{}) } } + +func fullClusterRestoreUsersWithoutIDs(exportDir string) func(t *testing.T) { + return func(t *testing.T) { + const numAccounts = 1000 + _, _, tmpDir, cleanupFn := backupRestoreTestSetup(t, multiNode, numAccounts, InitManualReplication) + defer cleanupFn() + + _, sqlDB, cleanup := backupRestoreTestSetupEmpty(t, singleNode, tmpDir, + InitManualReplication, base.TestClusterArgs{ + ServerArgs: base.TestServerArgs{ + Knobs: base.TestingKnobs{ + JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(), + }, + }}) + defer cleanup() + err := os.Symlink(exportDir, filepath.Join(tmpDir, "foo")) + require.NoError(t, err) + + sqlDB.Exec(t, fmt.Sprintf("RESTORE FROM '%s'", localFoo)) + + sqlDB.CheckQueryResults(t, `SELECT username, "hashedPassword", "isRole", user_id FROM system.users`, [][]string{ + {"admin", "", "true", "2"}, + {"root", "", "false", "1"}, + {"testrole", "NULL", "true", "100"}, + {"testuser", "NULL", "false", "101"}, + {"testuser2", "NULL", "false", "102"}, + {"testuser3", "NULL", "false", "103"}, + }) + + // Verify that the next user we create uses the next biggest ID. + sqlDB.Exec(t, "CREATE USER testuser4") + + sqlDB.CheckQueryResults(t, `SELECT username, "hashedPassword", "isRole", user_id FROM system.users`, [][]string{ + {"admin", "", "true", "2"}, + {"root", "", "false", "1"}, + {"testrole", "NULL", "true", "100"}, + {"testuser", "NULL", "false", "101"}, + {"testuser2", "NULL", "false", "102"}, + {"testuser3", "NULL", "false", "103"}, + {"testuser4", "NULL", "false", "104"}, + }) + } +} + +func restoreSystemUsersWithoutIDs(exportDir string) func(t *testing.T) { + return func(t *testing.T) { + const numAccounts = 1000 + _, _, tmpDir, cleanupFn := backupRestoreTestSetup(t, multiNode, numAccounts, InitManualReplication) + defer cleanupFn() + + _, sqlDB, cleanup := backupRestoreTestSetupEmpty(t, singleNode, tmpDir, + InitManualReplication, base.TestClusterArgs{ + ServerArgs: base.TestServerArgs{ + Knobs: base.TestingKnobs{ + JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(), + }, + }}) + defer cleanup() + err := os.Symlink(exportDir, filepath.Join(tmpDir, "foo")) + require.NoError(t, err) + + sqlDB.Exec(t, fmt.Sprintf("RESTORE SYSTEM USERS FROM '%s'", localFoo)) + + sqlDB.CheckQueryResults(t, `SELECT username, "hashedPassword", "isRole", user_id FROM system.users`, [][]string{ + {"admin", "", "true", "2"}, + {"root", "", "false", "1"}, + {"testrole", "NULL", "true", "100"}, + {"testuser", "NULL", "false", "101"}, + {"testuser2", "NULL", "false", "102"}, + {"testuser3", "NULL", "false", "103"}, + }) + + // Verify that the next user we create uses the next biggest ID. + sqlDB.Exec(t, "CREATE USER testuser4") + + sqlDB.CheckQueryResults(t, `SELECT username, "hashedPassword", "isRole", user_id FROM system.users`, [][]string{ + {"admin", "", "true", "2"}, + {"root", "", "false", "1"}, + {"testrole", "NULL", "true", "100"}, + {"testuser", "NULL", "false", "101"}, + {"testuser2", "NULL", "false", "102"}, + {"testuser3", "NULL", "false", "103"}, + {"testuser4", "NULL", "false", "104"}, + }) + + // Drop some users and try restoring again. + sqlDB.Exec(t, "DROP ROLE testrole") + sqlDB.Exec(t, "DROP ROLE testuser2") + sqlDB.Exec(t, "DROP ROLE testuser3") + sqlDB.Exec(t, "DROP ROLE testuser4") + + sqlDB.Exec(t, fmt.Sprintf("RESTORE SYSTEM USERS FROM '%s'", localFoo)) + + // testrole, testuser2, testuser3 should be reassigned higher ids. + sqlDB.CheckQueryResults(t, `SELECT username, "hashedPassword", "isRole", user_id FROM system.users`, [][]string{ + {"admin", "", "true", "2"}, + {"root", "", "false", "1"}, + {"testrole", "NULL", "true", "105"}, + {"testuser", "NULL", "false", "101"}, + {"testuser2", "NULL", "false", "106"}, + {"testuser3", "NULL", "false", "107"}, + }) + + // Verify that the next user we create uses the next biggest ID. + sqlDB.Exec(t, "CREATE USER testuser4") + sqlDB.CheckQueryResults(t, `SELECT username, "hashedPassword", "isRole", user_id FROM system.users`, [][]string{ + {"admin", "", "true", "2"}, + {"root", "", "false", "1"}, + {"testrole", "NULL", "true", "105"}, + {"testuser", "NULL", "false", "101"}, + {"testuser2", "NULL", "false", "106"}, + {"testuser3", "NULL", "false", "107"}, + {"testuser4", "NULL", "false", "108"}, + }) + } +} diff --git a/pkg/ccl/backupccl/system_schema.go b/pkg/ccl/backupccl/system_schema.go index 8de78575cb52..24686d6db2d2 100644 --- a/pkg/ccl/backupccl/system_schema.go +++ b/pkg/ccl/backupccl/system_schema.go @@ -14,14 +14,18 @@ import ( "math" "sort" + "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" + "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descidgen" descpb "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" "github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/errors" ) @@ -83,6 +87,10 @@ type systemBackupConfiguration struct { expectMissingInSystemTenant bool } +// roleIDSequenceRestoreOrder is set to 1 since it must be after system.users +// which has the default 0. +const roleIDSequenceRestoreOrder = 1 + // defaultSystemTableRestoreFunc is how system table data is restored. This can // be overwritten with the system table's // systemBackupConfiguration.customRestoreFunc. @@ -110,6 +118,7 @@ func defaultSystemTableRestoreFunc( if _, err := executor.Exec(ctx, opName, txn, restoreQuery); err != nil { return errors.Wrapf(err, "inserting data to system.%s", systemTableName) } + return nil } @@ -152,6 +161,84 @@ func queryTableRowCount( return int64(*count), nil } +func usersRestoreFunc( + ctx context.Context, + execCfg *sql.ExecutorConfig, + txn *kv.Txn, + systemTableName, tempTableName string, +) error { + if !execCfg.Settings.Version.IsActive(ctx, clusterversion.AddSystemUserIDColumn) { + return defaultSystemTableRestoreFunc( + ctx, execCfg, txn, systemTableName, tempTableName, + ) + } + + executor := execCfg.InternalExecutor + hasIDColumnQuery := fmt.Sprintf( + `SELECT EXISTS (SELECT 1 FROM [SHOW COLUMNS FROM %s] WHERE column_name = 'user_id')`, tempTableName) + row, err := executor.QueryRow(ctx, "has-id-column", txn, hasIDColumnQuery) + if err != nil { + return err + } + hasIDColumn := tree.MustBeDBool(row[0]) + if hasIDColumn { + return defaultSystemTableRestoreFunc( + ctx, execCfg, txn, systemTableName, tempTableName, + ) + } + + deleteQuery := fmt.Sprintf("DELETE FROM system.%s WHERE true", systemTableName) + opName := systemTableName + "-data-deletion" + log.Eventf(ctx, "clearing data from system table %s with query %q", + systemTableName, deleteQuery) + + _, err = executor.Exec(ctx, opName, txn, deleteQuery) + if err != nil { + return errors.Wrapf(err, "deleting data from system.%s", systemTableName) + } + + it, err := executor.QueryIteratorEx(ctx, "query-system-users-in-backup", + txn, sessiondata.NodeUserSessionDataOverride, + fmt.Sprintf(`SELECT * FROM %s`, tempTableName)) + if err != nil { + return err + } + + for { + ok, err := it.Next(ctx) + if err != nil { + return err + } + if !ok { + break + } + + username := tree.MustBeDString(it.Cur()[0]) + password := it.Cur()[1] + isRole := tree.MustBeDBool(it.Cur()[2]) + + var id int64 + if username == "root" { + id = 1 + } else if username == "admin" { + id = 2 + } else { + id, err = descidgen.GenerateUniqueRoleID(ctx, execCfg.DB, execCfg.Codec) + if err != nil { + return err + } + } + + restoreQuery := fmt.Sprintf("INSERT INTO system.%s VALUES ($1, $2, $3, $4)", + systemTableName) + opName = systemTableName + "-data-insert" + if _, err := executor.Exec(ctx, opName, txn, restoreQuery, username, password, isRole, id); err != nil { + return errors.Wrapf(err, "inserting data to system.%s", systemTableName) + } + } + return nil +} + // When restoring the settings table, we want to make sure to not override the // version. func settingsRestoreFunc( @@ -181,6 +268,28 @@ func settingsRestoreFunc( return nil } +func roleIDSeqRestoreFunc( + ctx context.Context, + execCfg *sql.ExecutorConfig, + txn *kv.Txn, + systemTableName, tempTableName string, +) error { + if execCfg.Settings.Version.IsActive(ctx, clusterversion.AddSystemUserIDColumn) { + datums, err := execCfg.InternalExecutor.QueryRowEx( + ctx, "role-id-seq-custom-restore", txn, + sessiondata.NodeUserSessionDataOverride, + `SELECT max(user_id) FROM system.users`, + ) + if err != nil { + return err + } + max := tree.MustBeDOid(datums[0]) + return execCfg.DB.Put(ctx, execCfg.Codec.SequenceKey(keys.RoleIDSequenceID), max.Oid+1) + } + // Nothing to be done since no user ids have been assigned. + return nil +} + // systemTableBackupConfiguration is a map from every systemTable present in the // cluster to a configuration struct which specifies how it should be treated by // backup. Every system table should have a specification defined here, enforced @@ -188,6 +297,7 @@ func settingsRestoreFunc( var systemTableBackupConfiguration = map[string]systemBackupConfiguration{ systemschema.UsersTable.GetName(): { shouldIncludeInClusterBackup: optInToClusterBackup, // No desc ID columns. + customRestoreFunc: usersRestoreFunc, }, systemschema.ZonesTable.GetName(): { shouldIncludeInClusterBackup: optInToClusterBackup, // ID in "id". @@ -350,6 +460,11 @@ var systemTableBackupConfiguration = map[string]systemBackupConfiguration{ systemschema.SystemExternalConnectionsTable.GetName(): { shouldIncludeInClusterBackup: optInToClusterBackup, // No desc ID columns. }, + systemschema.RoleIDSequence.GetName(): { + shouldIncludeInClusterBackup: optInToClusterBackup, + customRestoreFunc: roleIDSeqRestoreFunc, + restoreInOrder: roleIDSequenceRestoreOrder, + }, } func rekeySystemTable( diff --git a/pkg/ccl/backupccl/testdata/backup-restore/external-connections b/pkg/ccl/backupccl/testdata/backup-restore/external-connections index bdbf112550fa..b99b8ad31707 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/external-connections +++ b/pkg/ccl/backupccl/testdata/backup-restore/external-connections @@ -37,6 +37,7 @@ public schema full public schema full public schema full public schema full +role_id_seq table full role_members table full role_options table full scheduled_jobs table full diff --git a/pkg/ccl/backupccl/testdata/backup-restore/feature-flags b/pkg/ccl/backupccl/testdata/backup-restore/feature-flags index 98433489620e..bc2e34597508 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/feature-flags +++ b/pkg/ccl/backupccl/testdata/backup-restore/feature-flags @@ -81,7 +81,6 @@ RESTORE TABLE d.t FROM 'nodelocal://1/deprecated'; ---- NOTICE: The `RESTORE FROM ` syntax will be removed in a future release, please switch over to using `RESTORE FROM IN ` to restore a particular backup from a collection: https://www.cockroachlabs.com/docs/stable/restore.html#view-the-backup-subdirectories - exec-sql RESTORE SYSTEM USERS FROM 'nodelocal://1/deprecated'; ---- diff --git a/pkg/ccl/backupccl/testdata/backup-restore/rangekeys b/pkg/ccl/backupccl/testdata/backup-restore/rangekeys index ba9be196e212..26677222a2c4 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/rangekeys +++ b/pkg/ccl/backupccl/testdata/backup-restore/rangekeys @@ -81,4 +81,3 @@ SELECT count(*) from orig1.baz 1 - diff --git a/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/2022/07/13-152350.46/BACKUP-LOCK-778749314546466817 b/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/2022/07/13-152350.46/BACKUP-LOCK-778749314546466817 new file mode 100644 index 000000000000..19104f172a34 --- /dev/null +++ b/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/2022/07/13-152350.46/BACKUP-LOCK-778749314546466817 @@ -0,0 +1 @@ +lock \ No newline at end of file diff --git a/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/2022/07/13-152350.46/BACKUP-STATISTICS b/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/2022/07/13-152350.46/BACKUP-STATISTICS new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/2022/07/13-152350.46/BACKUP_MANIFEST b/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/2022/07/13-152350.46/BACKUP_MANIFEST new file mode 100644 index 000000000000..446274ecf5c8 Binary files /dev/null and b/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/2022/07/13-152350.46/BACKUP_MANIFEST differ diff --git a/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/2022/07/13-152350.46/BACKUP_MANIFEST-CHECKSUM b/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/2022/07/13-152350.46/BACKUP_MANIFEST-CHECKSUM new file mode 100644 index 000000000000..29cf52bd46c2 --- /dev/null +++ b/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/2022/07/13-152350.46/BACKUP_MANIFEST-CHECKSUM @@ -0,0 +1 @@ +–θΙA \ No newline at end of file diff --git a/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/2022/07/13-152350.46/data/778749316061200385.sst b/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/2022/07/13-152350.46/data/778749316061200385.sst new file mode 100644 index 000000000000..7f5c92283c4b Binary files /dev/null and b/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/2022/07/13-152350.46/data/778749316061200385.sst differ diff --git a/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/2022/07/13-152350.46/data/778749316187160577.sst b/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/2022/07/13-152350.46/data/778749316187160577.sst new file mode 100644 index 000000000000..ec3ac90009ae Binary files /dev/null and b/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/2022/07/13-152350.46/data/778749316187160577.sst differ diff --git a/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/2022/07/13-152350.46/data/778749316254040065.sst b/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/2022/07/13-152350.46/data/778749316254040065.sst new file mode 100644 index 000000000000..d81a70864cbc Binary files /dev/null and b/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/2022/07/13-152350.46/data/778749316254040065.sst differ diff --git a/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/2022/07/13-152350.46/fileinfo.sst b/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/2022/07/13-152350.46/fileinfo.sst new file mode 100644 index 000000000000..8b761e3f1c50 Binary files /dev/null and b/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/2022/07/13-152350.46/fileinfo.sst differ diff --git a/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/2022/07/13-152350.46/metadata.sst b/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/2022/07/13-152350.46/metadata.sst new file mode 100644 index 000000000000..814e201b490f Binary files /dev/null and b/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/2022/07/13-152350.46/metadata.sst differ diff --git a/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/2022/07/13-152350.46/progress/BACKUP-CHECKPOINT-13cdcfcdcdd2cfc8d2ceccdfcecac5cdccc5cacfd1c6c9c9cbdfd4cfcfcfcfdfaaabbcfffe b/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/2022/07/13-152350.46/progress/BACKUP-CHECKPOINT-13cdcfcdcdd2cfc8d2ceccdfcecac5cdccc5cacfd1c6c9c9cbdfd4cfcfcfcfdfaaabbcfffe new file mode 100644 index 000000000000..49b1e216643a Binary files /dev/null and b/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/2022/07/13-152350.46/progress/BACKUP-CHECKPOINT-13cdcfcdcdd2cfc8d2ceccdfcecac5cdccc5cacfd1c6c9c9cbdfd4cfcfcfcfdfaaabbcfffe differ diff --git a/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/2022/07/13-152350.46/progress/BACKUP-CHECKPOINT-13cdcfcdcdd2cfc8d2ceccdfcecac5cdccc5cacfd1c6c9c9cbdfd4cfcfcfcfdfaaabbcfffe-CHECKSUM b/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/2022/07/13-152350.46/progress/BACKUP-CHECKPOINT-13cdcfcdcdd2cfc8d2ceccdfcecac5cdccc5cacfd1c6c9c9cbdfd4cfcfcfcfdfaaabbcfffe-CHECKSUM new file mode 100644 index 000000000000..9ffafb9bdf9e --- /dev/null +++ b/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/2022/07/13-152350.46/progress/BACKUP-CHECKPOINT-13cdcfcdcdd2cfc8d2ceccdfcecac5cdccc5cacfd1c6c9c9cbdfd4cfcfcfcfdfaaabbcfffe-CHECKSUM @@ -0,0 +1 @@ +œ‹μ \ No newline at end of file diff --git a/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/2022/07/13-152350.46/progress/BACKUP-CHECKPOINT-13cdcfcdcdd2cfc8d2ceccdfcecac5cdccc5cacfd1c8c6c7cdcdcbdfd4cfcfcfcfdfaaabbcfffe b/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/2022/07/13-152350.46/progress/BACKUP-CHECKPOINT-13cdcfcdcdd2cfc8d2ceccdfcecac5cdccc5cacfd1c8c6c7cdcdcbdfd4cfcfcfcfdfaaabbcfffe new file mode 100644 index 000000000000..f46f9785f2c2 Binary files /dev/null and b/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/2022/07/13-152350.46/progress/BACKUP-CHECKPOINT-13cdcfcdcdd2cfc8d2ceccdfcecac5cdccc5cacfd1c8c6c7cdcdcbdfd4cfcfcfcfdfaaabbcfffe differ diff --git a/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/2022/07/13-152350.46/progress/BACKUP-CHECKPOINT-13cdcfcdcdd2cfc8d2ceccdfcecac5cdccc5cacfd1c8c6c7cdcdcbdfd4cfcfcfcfdfaaabbcfffe-CHECKSUM b/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/2022/07/13-152350.46/progress/BACKUP-CHECKPOINT-13cdcfcdcdd2cfc8d2ceccdfcecac5cdccc5cacfd1c8c6c7cdcdcbdfd4cfcfcfcfdfaaabbcfffe-CHECKSUM new file mode 100644 index 000000000000..cc7de46ef007 --- /dev/null +++ b/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/2022/07/13-152350.46/progress/BACKUP-CHECKPOINT-13cdcfcdcdd2cfc8d2ceccdfcecac5cdccc5cacfd1c8c6c7cdcdcbdfd4cfcfcfcfdfaaabbcfffe-CHECKSUM @@ -0,0 +1 @@ +ΖΎΒ: \ No newline at end of file diff --git a/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/BACKUP-LOCK-782466617872318465 b/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/BACKUP-LOCK-782466617872318465 new file mode 100644 index 000000000000..19104f172a34 --- /dev/null +++ b/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/BACKUP-LOCK-782466617872318465 @@ -0,0 +1 @@ +lock \ No newline at end of file diff --git a/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/BACKUP-STATISTICS b/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/BACKUP-STATISTICS new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/BACKUP_MANIFEST b/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/BACKUP_MANIFEST new file mode 100644 index 000000000000..d0098cbf4b66 Binary files /dev/null and b/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/BACKUP_MANIFEST differ diff --git a/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/BACKUP_MANIFEST-CHECKSUM b/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/BACKUP_MANIFEST-CHECKSUM new file mode 100644 index 000000000000..724758f736b6 --- /dev/null +++ b/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/BACKUP_MANIFEST-CHECKSUM @@ -0,0 +1 @@ +y<΅γ \ No newline at end of file diff --git a/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/data/782466619668725761.sst b/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/data/782466619668725761.sst new file mode 100644 index 000000000000..c30b1142c49e Binary files /dev/null and b/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/data/782466619668725761.sst differ diff --git a/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/data/782466619788066817.sst b/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/data/782466619788066817.sst new file mode 100644 index 000000000000..c377a503da00 Binary files /dev/null and b/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/data/782466619788066817.sst differ diff --git a/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/data/782466619853340673.sst b/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/data/782466619853340673.sst new file mode 100644 index 000000000000..8726c4c9b5c1 Binary files /dev/null and b/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/data/782466619853340673.sst differ diff --git a/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/data/782466620157657089.sst b/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/data/782466620157657089.sst new file mode 100644 index 000000000000..1ab4086ff803 Binary files /dev/null and b/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/data/782466620157657089.sst differ diff --git a/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/fileinfo.sst b/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/fileinfo.sst new file mode 100644 index 000000000000..7f904c1b2679 Binary files /dev/null and b/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/fileinfo.sst differ diff --git a/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/metadata.sst b/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/metadata.sst new file mode 100644 index 000000000000..f7ad05aada0f Binary files /dev/null and b/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/metadata.sst differ diff --git a/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/metadata/latest/LATEST-13cdcfcdcdd2cfc8d2ceccdfcecac5cdccc5caced1cdc8cdcac7dfd4cfcfcfcfdfaaabbcfffe b/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/metadata/latest/LATEST-13cdcfcdcdd2cfc8d2ceccdfcecac5cdccc5caced1cdc8cdcac7dfd4cfcfcfcfdfaaabbcfffe new file mode 100644 index 000000000000..87e8647205b0 --- /dev/null +++ b/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/metadata/latest/LATEST-13cdcfcdcdd2cfc8d2ceccdfcecac5cdccc5caced1cdc8cdcac7dfd4cfcfcfcfdfaaabbcfffe @@ -0,0 +1 @@ +/2022/07/13-152350.46 \ No newline at end of file diff --git a/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/progress/BACKUP-CHECKPOINT-13cdcfcdcdd2cfc8d2cdc9dfcec7c5cccec5cfced1c6c6c8cecbcadfd4cfcfcfcfdfaaabbcfffe b/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/progress/BACKUP-CHECKPOINT-13cdcfcdcdd2cfc8d2cdc9dfcec7c5cccec5cfced1c6c6c8cecbcadfd4cfcfcfcfdfaaabbcfffe new file mode 100644 index 000000000000..4e38e6e64bff Binary files /dev/null and b/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/progress/BACKUP-CHECKPOINT-13cdcfcdcdd2cfc8d2cdc9dfcec7c5cccec5cfced1c6c6c8cecbcadfd4cfcfcfcfdfaaabbcfffe differ diff --git a/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/progress/BACKUP-CHECKPOINT-13cdcfcdcdd2cfc8d2cdc9dfcec7c5cccec5cfced1c6c6c8cecbcadfd4cfcfcfcfdfaaabbcfffe-CHECKSUM b/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/progress/BACKUP-CHECKPOINT-13cdcfcdcdd2cfc8d2cdc9dfcec7c5cccec5cfced1c6c6c8cecbcadfd4cfcfcfcfdfaaabbcfffe-CHECKSUM new file mode 100644 index 000000000000..9c31c2d3e05f --- /dev/null +++ b/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/progress/BACKUP-CHECKPOINT-13cdcfcdcdd2cfc8d2cdc9dfcec7c5cccec5cfced1c6c6c8cecbcadfd4cfcfcfcfdfaaabbcfffe-CHECKSUM @@ -0,0 +1 @@ +οYΗS \ No newline at end of file diff --git a/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/progress/BACKUP-CHECKPOINT-13cdcfcdcdd2cfc8d2cdc9dfcec7c5cccec5cfced1c8c7c9ccc7c7dfd4cfcfcfcfdfaaabbcfffe b/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/progress/BACKUP-CHECKPOINT-13cdcfcdcdd2cfc8d2cdc9dfcec7c5cccec5cfced1c8c7c9ccc7c7dfd4cfcfcfcfdfaaabbcfffe new file mode 100644 index 000000000000..5cb3f8b9ee6b Binary files /dev/null and b/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/progress/BACKUP-CHECKPOINT-13cdcfcdcdd2cfc8d2cdc9dfcec7c5cccec5cfced1c8c7c9ccc7c7dfd4cfcfcfcfdfaaabbcfffe differ diff --git a/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/progress/BACKUP-CHECKPOINT-13cdcfcdcdd2cfc8d2cdc9dfcec7c5cccec5cfced1c8c7c9ccc7c7dfd4cfcfcfcfdfaaabbcfffe-CHECKSUM b/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/progress/BACKUP-CHECKPOINT-13cdcfcdcdd2cfc8d2cdc9dfcec7c5cccec5cfced1c8c7c9ccc7c7dfd4cfcfcfcfdfaaabbcfffe-CHECKSUM new file mode 100644 index 000000000000..5c61472c663c --- /dev/null +++ b/pkg/ccl/backupccl/testdata/restore_old_versions/system-users-restore/22.1.3/progress/BACKUP-CHECKPOINT-13cdcfcdcdd2cfc8d2cdc9dfcec7c5cccec5cfced1c8c7c9ccc7c7dfd4cfcfcfcfdfaaabbcfffe-CHECKSUM @@ -0,0 +1 @@ +ž MΕ \ No newline at end of file diff --git a/pkg/ccl/spanconfigccl/spanconfigreconcilerccl/testdata/basic b/pkg/ccl/spanconfigccl/spanconfigreconcilerccl/testdata/basic index 0b3a8b281b40..ba1186ef88e4 100644 --- a/pkg/ccl/spanconfigccl/spanconfigreconcilerccl/testdata/basic +++ b/pkg/ccl/spanconfigccl/spanconfigreconcilerccl/testdata/basic @@ -54,6 +54,7 @@ upsert /Table/4{4-5} database system (host) upsert /Table/4{5-6} ttl_seconds=7200 ignore_strict_gc=true num_replicas=5 rangefeed_enabled=true upsert /Table/4{6-7} database system (host) upsert /Table/4{7-8} database system (host) +upsert /Table/4{8-9} database system (host) upsert /Table/5{0-1} database system (host) upsert /Table/5{1-2} database system (host) upsert /Table/5{2-3} database system (host) @@ -93,6 +94,7 @@ upsert /Table/11{2-3} num_replicas=7 state offset=47 ---- ... +/Table/4{8-9} database system (host) /Table/5{0-1} database system (host) /Table/5{1-2} database system (host) /Table/5{2-3} database system (host) @@ -185,6 +187,8 @@ delete /Table/4{6-7} upsert /Table/4{6-7} ttl_seconds=100 ignore_strict_gc=true num_replicas=5 rangefeed_enabled=true delete /Table/4{7-8} upsert /Table/4{7-8} ttl_seconds=100 ignore_strict_gc=true num_replicas=5 rangefeed_enabled=true +delete /Table/4{8-9} +upsert /Table/4{8-9} ttl_seconds=100 ignore_strict_gc=true num_replicas=5 rangefeed_enabled=true delete /Table/5{0-1} upsert /Table/5{0-1} ttl_seconds=100 ignore_strict_gc=true num_replicas=5 rangefeed_enabled=true delete /Table/5{1-2} diff --git a/pkg/ccl/spanconfigccl/spanconfigreconcilerccl/testdata/indexes b/pkg/ccl/spanconfigccl/spanconfigreconcilerccl/testdata/indexes index 3bd72c374648..b2b4a719300c 100644 --- a/pkg/ccl/spanconfigccl/spanconfigreconcilerccl/testdata/indexes +++ b/pkg/ccl/spanconfigccl/spanconfigreconcilerccl/testdata/indexes @@ -21,6 +21,7 @@ upsert /Table/10{6-7} range default state offset=47 ---- ... +/Table/4{8-9} database system (host) /Table/5{0-1} database system (host) /Table/5{1-2} database system (host) /Table/5{2-3} database system (host) @@ -45,6 +46,7 @@ upsert /Table/10{6/3-7} num_replicas=7 state offset=47 ---- ... +/Table/4{8-9} database system (host) /Table/5{0-1} database system (host) /Table/5{1-2} database system (host) /Table/5{2-3} database system (host) @@ -71,6 +73,7 @@ upsert /Table/10{6/3-7} ttl_seconds=3600 num_replicas=7 state offset=47 ---- ... +/Table/4{8-9} database system (host) /Table/5{0-1} database system (host) /Table/5{1-2} database system (host) /Table/5{2-3} database system (host) @@ -87,6 +90,7 @@ ALTER TABLE db.t CONFIGURE ZONE USING num_replicas = 9 state offset=47 ---- ... +/Table/4{8-9} database system (host) /Table/5{0-1} database system (host) /Table/5{1-2} database system (host) /Table/5{2-3} database system (host) @@ -117,6 +121,7 @@ state offset=46 ---- ... /Table/4{7-8} database system (host) +/Table/4{8-9} database system (host) /Table/5{0-1} database system (host) /Table/5{1-2} database system (host) /Table/5{2-3} database system (host) diff --git a/pkg/ccl/spanconfigccl/spanconfigreconcilerccl/testdata/multitenant/basic b/pkg/ccl/spanconfigccl/spanconfigreconcilerccl/testdata/multitenant/basic index 66e73483778c..4b1d656994dd 100644 --- a/pkg/ccl/spanconfigccl/spanconfigreconcilerccl/testdata/multitenant/basic +++ b/pkg/ccl/spanconfigccl/spanconfigreconcilerccl/testdata/multitenant/basic @@ -22,6 +22,7 @@ mutations state offset=47 ---- ... +/Table/4{8-9} database system (host) /Table/5{0-1} database system (host) /Table/5{1-2} database system (host) /Table/5{2-3} database system (host) @@ -69,6 +70,7 @@ upsert /Tenant/10/Table/4{2-3} database system (tenant) upsert /Tenant/10/Table/4{3-4} database system (tenant) upsert /Tenant/10/Table/4{4-5} database system (tenant) upsert /Tenant/10/Table/4{6-7} database system (tenant) +upsert /Tenant/10/Table/4{8-9} database system (tenant) upsert /Tenant/10/Table/5{0-1} database system (tenant) upsert /Tenant/10/Table/5{1-2} database system (tenant) upsert /Tenant/10/Table/5{2-3} database system (tenant) @@ -76,6 +78,7 @@ upsert /Tenant/10/Table/5{2-3} database system (tenant) state offset=47 ---- ... +/Table/4{8-9} database system (host) /Table/5{0-1} database system (host) /Table/5{1-2} database system (host) /Table/5{2-3} database system (host) @@ -113,6 +116,7 @@ state offset=47 /Tenant/10/Table/4{3-4} database system (tenant) /Tenant/10/Table/4{4-5} database system (tenant) /Tenant/10/Table/4{6-7} database system (tenant) +/Tenant/10/Table/4{8-9} database system (tenant) /Tenant/10/Table/5{0-1} database system (tenant) /Tenant/10/Table/5{1-2} database system (tenant) /Tenant/10/Table/5{2-3} database system (tenant) @@ -139,9 +143,11 @@ upsert /Tenant/10/Table/11{3-4} range default state offset=81 ---- ... +/Tenant/10/Table/4{2-3} database system (tenant) /Tenant/10/Table/4{3-4} database system (tenant) /Tenant/10/Table/4{4-5} database system (tenant) /Tenant/10/Table/4{6-7} database system (tenant) +/Tenant/10/Table/4{8-9} database system (tenant) /Tenant/10/Table/5{0-1} database system (tenant) /Tenant/10/Table/5{1-2} database system (tenant) /Tenant/10/Table/5{2-3} database system (tenant) diff --git a/pkg/ccl/spanconfigccl/spanconfigreconcilerccl/testdata/multitenant/protectedts b/pkg/ccl/spanconfigccl/spanconfigreconcilerccl/testdata/multitenant/protectedts index 25d2d433fbc4..f00e4d4bd6f9 100644 --- a/pkg/ccl/spanconfigccl/spanconfigreconcilerccl/testdata/multitenant/protectedts +++ b/pkg/ccl/spanconfigccl/spanconfigreconcilerccl/testdata/multitenant/protectedts @@ -19,6 +19,7 @@ mutations state offset=47 ---- ... +/Table/4{8-9} database system (host) /Table/5{0-1} database system (host) /Table/5{1-2} database system (host) /Table/5{2-3} database system (host) @@ -85,6 +86,7 @@ upsert /Tenant/10/Table/4{2-3} database system (tenant) upsert /Tenant/10/Table/4{3-4} database system (tenant) upsert /Tenant/10/Table/4{4-5} database system (tenant) upsert /Tenant/10/Table/4{6-7} database system (tenant) +upsert /Tenant/10/Table/4{8-9} database system (tenant) upsert /Tenant/10/Table/5{0-1} database system (tenant) upsert /Tenant/10/Table/5{1-2} database system (tenant) upsert /Tenant/10/Table/5{2-3} database system (tenant) diff --git a/pkg/ccl/spanconfigccl/spanconfigreconcilerccl/testdata/named_zones b/pkg/ccl/spanconfigccl/spanconfigreconcilerccl/testdata/named_zones index ff8dbf01f799..524d528b74f5 100644 --- a/pkg/ccl/spanconfigccl/spanconfigreconcilerccl/testdata/named_zones +++ b/pkg/ccl/spanconfigccl/spanconfigreconcilerccl/testdata/named_zones @@ -124,6 +124,7 @@ state offset=46 ---- ... /Table/4{7-8} database system (host) +/Table/4{8-9} database system (host) /Table/5{0-1} database system (host) /Table/5{1-2} database system (host) /Table/5{2-3} database system (host) @@ -142,6 +143,7 @@ state offset=46 ---- ... /Table/4{7-8} database system (host) +/Table/4{8-9} database system (host) /Table/5{0-1} database system (host) /Table/5{1-2} database system (host) /Table/5{2-3} database system (host) diff --git a/pkg/ccl/spanconfigccl/spanconfigreconcilerccl/testdata/partitions b/pkg/ccl/spanconfigccl/spanconfigreconcilerccl/testdata/partitions index 98eaf61f1ba7..319fe919fd64 100644 --- a/pkg/ccl/spanconfigccl/spanconfigreconcilerccl/testdata/partitions +++ b/pkg/ccl/spanconfigccl/spanconfigreconcilerccl/testdata/partitions @@ -13,6 +13,7 @@ mutations discard state offset=47 ---- ... +/Table/4{8-9} database system (host) /Table/5{0-1} database system (host) /Table/5{1-2} database system (host) /Table/5{2-3} database system (host) @@ -35,6 +36,7 @@ upsert /Table/10{6-7} range default state offset=47 ---- ... +/Table/4{8-9} database system (host) /Table/5{0-1} database system (host) /Table/5{1-2} database system (host) /Table/5{2-3} database system (host) @@ -54,6 +56,7 @@ upsert /Table/10{6-7} num_replicas=7 num_voters=5 state offset=47 ---- ... +/Table/4{8-9} database system (host) /Table/5{0-1} database system (host) /Table/5{1-2} database system (host) /Table/5{2-3} database system (host) @@ -78,6 +81,7 @@ upsert /Table/10{6/1/3-7} num_replicas=7 num_voters=5 state offset=47 ---- ... +/Table/4{8-9} database system (host) /Table/5{0-1} database system (host) /Table/5{1-2} database system (host) /Table/5{2-3} database system (host) @@ -102,6 +106,7 @@ upsert /Table/10{6/1/5-7} num_replicas=7 num_voters=5 state offset=47 ---- ... +/Table/4{8-9} database system (host) /Table/5{0-1} database system (host) /Table/5{1-2} database system (host) /Table/5{2-3} database system (host) @@ -140,6 +145,7 @@ upsert /Table/10{6/2-7} num_replicas=7 num_voters=5 state offset=47 ---- ... +/Table/4{8-9} database system (host) /Table/5{0-1} database system (host) /Table/5{1-2} database system (host) /Table/5{2-3} database system (host) @@ -176,6 +182,7 @@ upsert /Table/10{6/2-7} num_replicas=7 state offset=47 ---- ... +/Table/4{8-9} database system (host) /Table/5{0-1} database system (host) /Table/5{1-2} database system (host) /Table/5{2-3} database system (host) @@ -206,6 +213,7 @@ delete /Table/10{6/2-7} state offset=47 ---- ... +/Table/4{8-9} database system (host) /Table/5{0-1} database system (host) /Table/5{1-2} database system (host) /Table/5{2-3} database system (host) diff --git a/pkg/ccl/spanconfigccl/spanconfigreconcilerccl/testdata/protectedts b/pkg/ccl/spanconfigccl/spanconfigreconcilerccl/testdata/protectedts index 16e018af8b72..00fd5c5dcb35 100644 --- a/pkg/ccl/spanconfigccl/spanconfigreconcilerccl/testdata/protectedts +++ b/pkg/ccl/spanconfigccl/spanconfigreconcilerccl/testdata/protectedts @@ -77,6 +77,7 @@ state limit=3 state offset=51 ---- ... +/Table/5{0-1} database system (host) /Table/5{1-2} database system (host) /Table/5{2-3} database system (host) /Table/10{6-7} protection_policies=[{ts: 3} {ts: 4}] diff --git a/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/full_translate b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/full_translate index a5a8403e1d18..5e749ded3631 100644 --- a/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/full_translate +++ b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/full_translate @@ -70,6 +70,7 @@ full-translate /Table/4{5-6} ttl_seconds=7200 ignore_strict_gc=true num_replicas=5 rangefeed_enabled=true /Table/4{6-7} database system (host) /Table/4{7-8} database system (host) +/Table/4{8-9} database system (host) /Table/5{0-1} database system (host) /Table/5{1-2} database system (host) /Table/5{2-3} database system (host) diff --git a/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/full_translate_named_zones_deleted b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/full_translate_named_zones_deleted index ef2edd3f8edb..aba602352d93 100644 --- a/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/full_translate_named_zones_deleted +++ b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/full_translate_named_zones_deleted @@ -83,6 +83,7 @@ full-translate /Table/4{5-6} ttl_seconds=7200 ignore_strict_gc=true num_replicas=5 rangefeed_enabled=true /Table/4{6-7} database system (host) /Table/4{7-8} database system (host) +/Table/4{8-9} database system (host) /Table/5{0-1} database system (host) /Table/5{1-2} database system (host) /Table/5{2-3} database system (host) diff --git a/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/system_database b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/system_database index 586320e07206..8c7f48f69a75 100644 --- a/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/system_database +++ b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/system_database @@ -46,6 +46,7 @@ translate database=system /Table/4{5-6} ttl_seconds=7200 ignore_strict_gc=true num_replicas=5 rangefeed_enabled=true /Table/4{6-7} database system (host) /Table/4{7-8} database system (host) +/Table/4{8-9} database system (host) /Table/5{0-1} database system (host) /Table/5{1-2} database system (host) /Table/5{2-3} database system (host) @@ -99,6 +100,7 @@ translate database=system /Table/4{5-6} ttl_seconds=7200 ignore_strict_gc=true num_replicas=7 rangefeed_enabled=true /Table/4{6-7} ignore_strict_gc=true num_replicas=7 rangefeed_enabled=true /Table/4{7-8} ignore_strict_gc=true num_replicas=7 rangefeed_enabled=true +/Table/4{8-9} ignore_strict_gc=true num_replicas=7 rangefeed_enabled=true /Table/5{0-1} ignore_strict_gc=true num_replicas=7 rangefeed_enabled=true /Table/5{1-2} ignore_strict_gc=true num_replicas=7 rangefeed_enabled=true /Table/5{2-3} ignore_strict_gc=true num_replicas=7 rangefeed_enabled=true @@ -158,6 +160,7 @@ full-translate /Table/4{5-6} ttl_seconds=7200 ignore_strict_gc=true num_replicas=7 rangefeed_enabled=true /Table/4{6-7} ignore_strict_gc=true num_replicas=7 rangefeed_enabled=true /Table/4{7-8} ignore_strict_gc=true num_replicas=7 rangefeed_enabled=true +/Table/4{8-9} ignore_strict_gc=true num_replicas=7 rangefeed_enabled=true /Table/5{0-1} ignore_strict_gc=true num_replicas=7 rangefeed_enabled=true /Table/5{1-2} ignore_strict_gc=true num_replicas=7 rangefeed_enabled=true /Table/5{2-3} ignore_strict_gc=true num_replicas=7 rangefeed_enabled=true diff --git a/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/tenant/full_translate b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/tenant/full_translate index 84e7ffa98b62..466e15e2dd12 100644 --- a/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/tenant/full_translate +++ b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/tenant/full_translate @@ -51,6 +51,7 @@ full-translate /Tenant/10/Table/4{3-4} database system (tenant) /Tenant/10/Table/4{4-5} database system (tenant) /Tenant/10/Table/4{6-7} database system (tenant) +/Tenant/10/Table/4{8-9} database system (tenant) /Tenant/10/Table/5{0-1} database system (tenant) /Tenant/10/Table/5{1-2} database system (tenant) /Tenant/10/Table/5{2-3} database system (tenant) @@ -95,6 +96,7 @@ translate named-zone=default /Tenant/10/Table/4{3-4} database system (tenant) /Tenant/10/Table/4{4-5} database system (tenant) /Tenant/10/Table/4{6-7} database system (tenant) +/Tenant/10/Table/4{8-9} database system (tenant) /Tenant/10/Table/5{0-1} database system (tenant) /Tenant/10/Table/5{1-2} database system (tenant) /Tenant/10/Table/5{2-3} database system (tenant) diff --git a/pkg/ccl/streamingccl/streamclient/random_stream_client.go b/pkg/ccl/streamingccl/streamclient/random_stream_client.go index e413513f8259..397f75941252 100644 --- a/pkg/ccl/streamingccl/streamclient/random_stream_client.go +++ b/pkg/ccl/streamingccl/streamclient/random_stream_client.go @@ -350,6 +350,8 @@ func (m *randomStreamClient) getDescriptorAndNamespaceKVForTableID( tableID, fmt.Sprintf(RandomStreamSchemaPlaceholder, tableName), catpb.NewBasePrivilegeDescriptor(username.RootUserName()), + nil, /* txn */ + nil, /* collection */ ) if err != nil { return nil, nil, err diff --git a/pkg/cli/doctor_test.go b/pkg/cli/doctor_test.go index 2ec01e99ee28..9b145291169d 100644 --- a/pkg/cli/doctor_test.go +++ b/pkg/cli/doctor_test.go @@ -31,7 +31,7 @@ func TestDoctorCluster(t *testing.T) { "CREATE TABLE to_drop (id INT)", "DROP TABLE to_drop", "CREATE TABLE foo (id INT)", - "INSERT INTO system.users VALUES ('node', NULL, true)", + "INSERT INTO system.users VALUES ('node', NULL, true, 3)", "GRANT node TO root", "DELETE FROM system.namespace WHERE name = 'foo'", }, ";\n"), diff --git a/pkg/cli/testdata/doctor/test_examine_cluster b/pkg/cli/testdata/doctor/test_examine_cluster index e7bb31961f0b..1f148121d88e 100644 --- a/pkg/cli/testdata/doctor/test_examine_cluster +++ b/pkg/cli/testdata/doctor/test_examine_cluster @@ -1,7 +1,7 @@ debug doctor examine cluster ---- debug doctor examine cluster -Examining 46 descriptors and 45 namespace entries... +Examining 47 descriptors and 46 namespace entries... ParentID 100, ParentSchemaID 101: relation "foo" (105): expected matching namespace entry, found none Examining 4 jobs... ERROR: validation failed diff --git a/pkg/cli/testdata/zip/partial1 b/pkg/cli/testdata/zip/partial1 index 3081e23a6725..0cbb3ff8cca4 100644 --- a/pkg/cli/testdata/zip/partial1 +++ b/pkg/cli/testdata/zip/partial1 @@ -10,7 +10,7 @@ debug zip --concurrency=1 --cpu-profile-duration=0s /dev/null [cluster] requesting data for debug/settings... received response... converting to JSON... writing binary output: debug/settings.json... done [cluster] requesting data for debug/reports/problemranges... received response... converting to JSON... writing binary output: debug/reports/problemranges.json... done [cluster] retrieving list of system tables... done -[cluster] 39 system tables found +[cluster] 40 system tables found [cluster] retrieving SQL data for crdb_internal.cluster_contention_events... writing output: debug/crdb_internal.cluster_contention_events.txt... done [cluster] retrieving SQL data for crdb_internal.cluster_distsql_flows... writing output: debug/crdb_internal.cluster_distsql_flows.txt... done [cluster] retrieving SQL data for crdb_internal.cluster_database_privileges... writing output: debug/crdb_internal.cluster_database_privileges.txt... done @@ -53,6 +53,7 @@ debug zip --concurrency=1 --cpu-profile-duration=0s /dev/null [cluster] retrieving SQL data for system.replication_critical_localities... writing output: debug/system.replication_critical_localities.txt... done [cluster] retrieving SQL data for system.replication_stats... writing output: debug/system.replication_stats.txt... done [cluster] retrieving SQL data for system.reports_meta... writing output: debug/system.reports_meta.txt... done +[cluster] retrieving SQL data for system.role_id_seq... writing output: debug/system.role_id_seq.txt... done [cluster] retrieving SQL data for system.role_members... writing output: debug/system.role_members.txt... done [cluster] retrieving SQL data for system.role_options... writing output: debug/system.role_options.txt... done [cluster] retrieving SQL data for system.scheduled_jobs... writing output: debug/system.scheduled_jobs.txt... done @@ -109,7 +110,7 @@ debug zip --concurrency=1 --cpu-profile-duration=0s /dev/null [node 1] 1 log file ... [node 1] [log file ... [node 1] requesting ranges... received response... done -[node 1] 51 ranges found +[node 1] 52 ranges found [node 1] writing range 1... converting to JSON... writing binary output: debug/nodes/1/ranges/1.json... done [node 1] writing range 2... converting to JSON... writing binary output: debug/nodes/1/ranges/2.json... done [node 1] writing range 3... converting to JSON... writing binary output: debug/nodes/1/ranges/3.json... done @@ -161,6 +162,7 @@ debug zip --concurrency=1 --cpu-profile-duration=0s /dev/null [node 1] writing range 49... converting to JSON... writing binary output: debug/nodes/1/ranges/49.json... done [node 1] writing range 50... converting to JSON... writing binary output: debug/nodes/1/ranges/50.json... done [node 1] writing range 51... converting to JSON... writing binary output: debug/nodes/1/ranges/51.json... done +[node 1] writing range 52... converting to JSON... writing binary output: debug/nodes/1/ranges/52.json... done [node 2] node status... converting to JSON... writing binary output: debug/nodes/2/status.json... done [node 2] using SQL connection URL: postgresql://... [node 2] retrieving SQL data for crdb_internal.feature_usage... writing output: debug/nodes/2/crdb_internal.feature_usage.txt... @@ -291,7 +293,7 @@ debug zip --concurrency=1 --cpu-profile-duration=0s /dev/null [node 3] 1 log file ... [node 3] [log file ... [node 3] requesting ranges... received response... done -[node 3] 51 ranges found +[node 3] 52 ranges found [node 3] writing range 1... converting to JSON... writing binary output: debug/nodes/3/ranges/1.json... done [node 3] writing range 2... converting to JSON... writing binary output: debug/nodes/3/ranges/2.json... done [node 3] writing range 3... converting to JSON... writing binary output: debug/nodes/3/ranges/3.json... done @@ -343,6 +345,7 @@ debug zip --concurrency=1 --cpu-profile-duration=0s /dev/null [node 3] writing range 49... converting to JSON... writing binary output: debug/nodes/3/ranges/49.json... done [node 3] writing range 50... converting to JSON... writing binary output: debug/nodes/3/ranges/50.json... done [node 3] writing range 51... converting to JSON... writing binary output: debug/nodes/3/ranges/51.json... done +[node 3] writing range 52... converting to JSON... writing binary output: debug/nodes/3/ranges/52.json... done [cluster] pprof summary script... writing binary output: debug/pprof-summary.sh... done [cluster] hot range summary script... writing binary output: debug/hot-ranges.sh... done [cluster] tenant hot range summary script... writing binary output: debug/hot-ranges-tenant.sh... done diff --git a/pkg/cli/testdata/zip/partial1_excluded b/pkg/cli/testdata/zip/partial1_excluded index 8bca312e1578..3d8d58896d38 100644 --- a/pkg/cli/testdata/zip/partial1_excluded +++ b/pkg/cli/testdata/zip/partial1_excluded @@ -10,7 +10,7 @@ debug zip /dev/null --concurrency=1 --exclude-nodes=2 --cpu-profile-duration=0 [cluster] requesting data for debug/settings... received response... converting to JSON... writing binary output: debug/settings.json... done [cluster] requesting data for debug/reports/problemranges... received response... converting to JSON... writing binary output: debug/reports/problemranges.json... done [cluster] retrieving list of system tables... done -[cluster] 39 system tables found +[cluster] 40 system tables found [cluster] retrieving SQL data for crdb_internal.cluster_contention_events... writing output: debug/crdb_internal.cluster_contention_events.txt... done [cluster] retrieving SQL data for crdb_internal.cluster_distsql_flows... writing output: debug/crdb_internal.cluster_distsql_flows.txt... done [cluster] retrieving SQL data for crdb_internal.cluster_database_privileges... writing output: debug/crdb_internal.cluster_database_privileges.txt... done @@ -53,6 +53,7 @@ debug zip /dev/null --concurrency=1 --exclude-nodes=2 --cpu-profile-duration=0 [cluster] retrieving SQL data for system.replication_critical_localities... writing output: debug/system.replication_critical_localities.txt... done [cluster] retrieving SQL data for system.replication_stats... writing output: debug/system.replication_stats.txt... done [cluster] retrieving SQL data for system.reports_meta... writing output: debug/system.reports_meta.txt... done +[cluster] retrieving SQL data for system.role_id_seq... writing output: debug/system.role_id_seq.txt... done [cluster] retrieving SQL data for system.role_members... writing output: debug/system.role_members.txt... done [cluster] retrieving SQL data for system.role_options... writing output: debug/system.role_options.txt... done [cluster] retrieving SQL data for system.scheduled_jobs... writing output: debug/system.scheduled_jobs.txt... done @@ -109,7 +110,7 @@ debug zip /dev/null --concurrency=1 --exclude-nodes=2 --cpu-profile-duration=0 [node 1] 1 log file ... [node 1] [log file ... [node 1] requesting ranges... received response... done -[node 1] 51 ranges found +[node 1] 52 ranges found [node 1] writing range 1... converting to JSON... writing binary output: debug/nodes/1/ranges/1.json... done [node 1] writing range 2... converting to JSON... writing binary output: debug/nodes/1/ranges/2.json... done [node 1] writing range 3... converting to JSON... writing binary output: debug/nodes/1/ranges/3.json... done @@ -161,6 +162,7 @@ debug zip /dev/null --concurrency=1 --exclude-nodes=2 --cpu-profile-duration=0 [node 1] writing range 49... converting to JSON... writing binary output: debug/nodes/1/ranges/49.json... done [node 1] writing range 50... converting to JSON... writing binary output: debug/nodes/1/ranges/50.json... done [node 1] writing range 51... converting to JSON... writing binary output: debug/nodes/1/ranges/51.json... done +[node 1] writing range 52... converting to JSON... writing binary output: debug/nodes/1/ranges/52.json... done [node 2] skipping node... writing binary output: debug/nodes/2.skipped... done [node 3] node status... converting to JSON... writing binary output: debug/nodes/3/status.json... done [node 3] using SQL connection URL: postgresql://... @@ -200,7 +202,7 @@ debug zip /dev/null --concurrency=1 --exclude-nodes=2 --cpu-profile-duration=0 [node 3] 1 log file ... [node 3] [log file ... [node 3] requesting ranges... received response... done -[node 3] 51 ranges found +[node 3] 52 ranges found [node 3] writing range 1... converting to JSON... writing binary output: debug/nodes/3/ranges/1.json... done [node 3] writing range 2... converting to JSON... writing binary output: debug/nodes/3/ranges/2.json... done [node 3] writing range 3... converting to JSON... writing binary output: debug/nodes/3/ranges/3.json... done @@ -252,6 +254,7 @@ debug zip /dev/null --concurrency=1 --exclude-nodes=2 --cpu-profile-duration=0 [node 3] writing range 49... converting to JSON... writing binary output: debug/nodes/3/ranges/49.json... done [node 3] writing range 50... converting to JSON... writing binary output: debug/nodes/3/ranges/50.json... done [node 3] writing range 51... converting to JSON... writing binary output: debug/nodes/3/ranges/51.json... done +[node 3] writing range 52... converting to JSON... writing binary output: debug/nodes/3/ranges/52.json... done [cluster] pprof summary script... writing binary output: debug/pprof-summary.sh... done [cluster] hot range summary script... writing binary output: debug/hot-ranges.sh... done [cluster] tenant hot range summary script... writing binary output: debug/hot-ranges-tenant.sh... done diff --git a/pkg/cli/testdata/zip/partial2 b/pkg/cli/testdata/zip/partial2 index c8af3ab08cf4..aab22367bcf1 100644 --- a/pkg/cli/testdata/zip/partial2 +++ b/pkg/cli/testdata/zip/partial2 @@ -10,7 +10,7 @@ debug zip --concurrency=1 --cpu-profile-duration=0 /dev/null [cluster] requesting data for debug/settings... received response... converting to JSON... writing binary output: debug/settings.json... done [cluster] requesting data for debug/reports/problemranges... received response... converting to JSON... writing binary output: debug/reports/problemranges.json... done [cluster] retrieving list of system tables... done -[cluster] 39 system tables found +[cluster] 40 system tables found [cluster] retrieving SQL data for crdb_internal.cluster_contention_events... writing output: debug/crdb_internal.cluster_contention_events.txt... done [cluster] retrieving SQL data for crdb_internal.cluster_distsql_flows... writing output: debug/crdb_internal.cluster_distsql_flows.txt... done [cluster] retrieving SQL data for crdb_internal.cluster_database_privileges... writing output: debug/crdb_internal.cluster_database_privileges.txt... done @@ -53,6 +53,7 @@ debug zip --concurrency=1 --cpu-profile-duration=0 /dev/null [cluster] retrieving SQL data for system.replication_critical_localities... writing output: debug/system.replication_critical_localities.txt... done [cluster] retrieving SQL data for system.replication_stats... writing output: debug/system.replication_stats.txt... done [cluster] retrieving SQL data for system.reports_meta... writing output: debug/system.reports_meta.txt... done +[cluster] retrieving SQL data for system.role_id_seq... writing output: debug/system.role_id_seq.txt... done [cluster] retrieving SQL data for system.role_members... writing output: debug/system.role_members.txt... done [cluster] retrieving SQL data for system.role_options... writing output: debug/system.role_options.txt... done [cluster] retrieving SQL data for system.scheduled_jobs... writing output: debug/system.scheduled_jobs.txt... done @@ -109,7 +110,7 @@ debug zip --concurrency=1 --cpu-profile-duration=0 /dev/null [node 1] 1 log file ... [node 1] [log file ... [node 1] requesting ranges... received response... done -[node 1] 51 ranges found +[node 1] 52 ranges found [node 1] writing range 1... converting to JSON... writing binary output: debug/nodes/1/ranges/1.json... done [node 1] writing range 2... converting to JSON... writing binary output: debug/nodes/1/ranges/2.json... done [node 1] writing range 3... converting to JSON... writing binary output: debug/nodes/1/ranges/3.json... done @@ -161,6 +162,7 @@ debug zip --concurrency=1 --cpu-profile-duration=0 /dev/null [node 1] writing range 49... converting to JSON... writing binary output: debug/nodes/1/ranges/49.json... done [node 1] writing range 50... converting to JSON... writing binary output: debug/nodes/1/ranges/50.json... done [node 1] writing range 51... converting to JSON... writing binary output: debug/nodes/1/ranges/51.json... done +[node 1] writing range 52... converting to JSON... writing binary output: debug/nodes/1/ranges/52.json... done [node 3] node status... converting to JSON... writing binary output: debug/nodes/3/status.json... done [node 3] using SQL connection URL: postgresql://... [node 3] retrieving SQL data for crdb_internal.feature_usage... writing output: debug/nodes/3/crdb_internal.feature_usage.txt... done @@ -199,7 +201,7 @@ debug zip --concurrency=1 --cpu-profile-duration=0 /dev/null [node 3] 1 log file ... [node 3] [log file ... [node 3] requesting ranges... received response... done -[node 3] 51 ranges found +[node 3] 52 ranges found [node 3] writing range 1... converting to JSON... writing binary output: debug/nodes/3/ranges/1.json... done [node 3] writing range 2... converting to JSON... writing binary output: debug/nodes/3/ranges/2.json... done [node 3] writing range 3... converting to JSON... writing binary output: debug/nodes/3/ranges/3.json... done @@ -251,6 +253,7 @@ debug zip --concurrency=1 --cpu-profile-duration=0 /dev/null [node 3] writing range 49... converting to JSON... writing binary output: debug/nodes/3/ranges/49.json... done [node 3] writing range 50... converting to JSON... writing binary output: debug/nodes/3/ranges/50.json... done [node 3] writing range 51... converting to JSON... writing binary output: debug/nodes/3/ranges/51.json... done +[node 3] writing range 52... converting to JSON... writing binary output: debug/nodes/3/ranges/52.json... done [cluster] pprof summary script... writing binary output: debug/pprof-summary.sh... done [cluster] hot range summary script... writing binary output: debug/hot-ranges.sh... done [cluster] tenant hot range summary script... writing binary output: debug/hot-ranges-tenant.sh... done diff --git a/pkg/cli/testdata/zip/specialnames b/pkg/cli/testdata/zip/specialnames index 0eaab6ebdc4c..1912ee9f9db5 100644 --- a/pkg/cli/testdata/zip/specialnames +++ b/pkg/cli/testdata/zip/specialnames @@ -1,7 +1,7 @@ zip ---- [cluster] retrieving list of system tables... done -[cluster] 39 system tables found +[cluster] 40 system tables found [cluster] retrieving SQL data for crdb_internal.cluster_database_privileges... writing output: debug/crdb_internal.cluster_database_privileges.txt... done [cluster] retrieving SQL data for crdb_internal.table_indexes... writing output: debug/crdb_internal.table_indexes.txt... done [cluster] retrieving SQL data for system.database_role_settings... writing output: debug/system.database_role_settings.txt... done diff --git a/pkg/cli/testdata/zip/testzip b/pkg/cli/testdata/zip/testzip index 44e22b86e737..4fd94a90cb86 100644 --- a/pkg/cli/testdata/zip/testzip +++ b/pkg/cli/testdata/zip/testzip @@ -10,7 +10,7 @@ debug zip --concurrency=1 --cpu-profile-duration=1s /dev/null [cluster] requesting data for debug/settings... received response... converting to JSON... writing binary output: debug/settings.json... done [cluster] requesting data for debug/reports/problemranges... received response... converting to JSON... writing binary output: debug/reports/problemranges.json... done [cluster] retrieving list of system tables... done -[cluster] 39 system tables found +[cluster] 40 system tables found [cluster] retrieving SQL data for crdb_internal.cluster_contention_events... writing output: debug/crdb_internal.cluster_contention_events.txt... done [cluster] retrieving SQL data for crdb_internal.cluster_distsql_flows... writing output: debug/crdb_internal.cluster_distsql_flows.txt... done [cluster] retrieving SQL data for crdb_internal.cluster_database_privileges... writing output: debug/crdb_internal.cluster_database_privileges.txt... done @@ -53,6 +53,7 @@ debug zip --concurrency=1 --cpu-profile-duration=1s /dev/null [cluster] retrieving SQL data for system.replication_critical_localities... writing output: debug/system.replication_critical_localities.txt... done [cluster] retrieving SQL data for system.replication_stats... writing output: debug/system.replication_stats.txt... done [cluster] retrieving SQL data for system.reports_meta... writing output: debug/system.reports_meta.txt... done +[cluster] retrieving SQL data for system.role_id_seq... writing output: debug/system.role_id_seq.txt... done [cluster] retrieving SQL data for system.role_members... writing output: debug/system.role_members.txt... done [cluster] retrieving SQL data for system.role_options... writing output: debug/system.role_options.txt... done [cluster] retrieving SQL data for system.scheduled_jobs... writing output: debug/system.scheduled_jobs.txt... done @@ -109,7 +110,7 @@ debug zip --concurrency=1 --cpu-profile-duration=1s /dev/null [node 1] requesting log file ... [node 1] 0 log file ... [node 1] requesting ranges... received response... done -[node 1] 51 ranges found +[node 1] 52 ranges found [node 1] writing range 1... converting to JSON... writing binary output: debug/nodes/1/ranges/1.json... done [node 1] writing range 2... converting to JSON... writing binary output: debug/nodes/1/ranges/2.json... done [node 1] writing range 3... converting to JSON... writing binary output: debug/nodes/1/ranges/3.json... done @@ -161,6 +162,7 @@ debug zip --concurrency=1 --cpu-profile-duration=1s /dev/null [node 1] writing range 49... converting to JSON... writing binary output: debug/nodes/1/ranges/49.json... done [node 1] writing range 50... converting to JSON... writing binary output: debug/nodes/1/ranges/50.json... done [node 1] writing range 51... converting to JSON... writing binary output: debug/nodes/1/ranges/51.json... done +[node 1] writing range 52... converting to JSON... writing binary output: debug/nodes/1/ranges/52.json... done [cluster] pprof summary script... writing binary output: debug/pprof-summary.sh... done [cluster] hot range summary script... writing binary output: debug/hot-ranges.sh... done [cluster] tenant hot range summary script... writing binary output: debug/hot-ranges-tenant.sh... done diff --git a/pkg/cli/testdata/zip/testzip_concurrent b/pkg/cli/testdata/zip/testzip_concurrent index b5d92d314119..e74618f3087b 100644 --- a/pkg/cli/testdata/zip/testzip_concurrent +++ b/pkg/cli/testdata/zip/testzip_concurrent @@ -1,6 +1,6 @@ zip ---- -[cluster] 39 system tables found +[cluster] 40 system tables found [cluster] creating output file /dev/null... [cluster] creating output file /dev/null: done [cluster] establishing RPC connection to ... @@ -172,6 +172,9 @@ zip [cluster] retrieving SQL data for system.reports_meta... [cluster] retrieving SQL data for system.reports_meta: done [cluster] retrieving SQL data for system.reports_meta: writing output: debug/system.reports_meta.txt... +[cluster] retrieving SQL data for system.role_id_seq... +[cluster] retrieving SQL data for system.role_id_seq: done +[cluster] retrieving SQL data for system.role_id_seq: writing output: debug/system.role_id_seq.txt... [cluster] retrieving SQL data for system.role_members... [cluster] retrieving SQL data for system.role_members: done [cluster] retrieving SQL data for system.role_members: writing output: debug/system.role_members.txt... @@ -222,7 +225,7 @@ zip [cluster] using SQL address: ... [cluster] using SQL address: ... [node 1] 1 log file ... -[node 1] 51 ranges found +[node 1] 52 ranges found [node 1] [log file ... [node 1] [log file ... [node 1] [log file ... @@ -520,6 +523,10 @@ zip [node 1] writing range 51: converting to JSON... [node 1] writing range 51: done [node 1] writing range 51: writing binary output: debug/nodes/1/ranges/51.json... +[node 1] writing range 52... +[node 1] writing range 52: converting to JSON... +[node 1] writing range 52: done +[node 1] writing range 52: writing binary output: debug/nodes/1/ranges/52.json... [node 1] writing range 5: converting to JSON... [node 1] writing range 5: done [node 1] writing range 5: writing binary output: debug/nodes/1/ranges/5.json... @@ -540,7 +547,7 @@ zip [node 1] writing range 9: done [node 1] writing range 9: writing binary output: debug/nodes/1/ranges/9.json... [node 2] 1 log file ... -[node 2] 51 ranges found +[node 2] 52 ranges found [node 2] [log file ... [node 2] [log file ... [node 2] [log file ... @@ -838,6 +845,10 @@ zip [node 2] writing range 51: converting to JSON... [node 2] writing range 51: done [node 2] writing range 51: writing binary output: debug/nodes/2/ranges/51.json... +[node 2] writing range 52... +[node 2] writing range 52: converting to JSON... +[node 2] writing range 52: done +[node 2] writing range 52: writing binary output: debug/nodes/2/ranges/52.json... [node 2] writing range 5: converting to JSON... [node 2] writing range 5: done [node 2] writing range 5: writing binary output: debug/nodes/2/ranges/5.json... @@ -858,7 +869,7 @@ zip [node 2] writing range 9: done [node 2] writing range 9: writing binary output: debug/nodes/2/ranges/9.json... [node 3] 1 log file ... -[node 3] 51 ranges found +[node 3] 52 ranges found [node 3] [log file ... [node 3] [log file ... [node 3] [log file ... @@ -1156,6 +1167,10 @@ zip [node 3] writing range 51: converting to JSON... [node 3] writing range 51: done [node 3] writing range 51: writing binary output: debug/nodes/3/ranges/51.json... +[node 3] writing range 52... +[node 3] writing range 52: converting to JSON... +[node 3] writing range 52: done +[node 3] writing range 52: writing binary output: debug/nodes/3/ranges/52.json... [node 3] writing range 5: converting to JSON... [node 3] writing range 5: done [node 3] writing range 5: writing binary output: debug/nodes/3/ranges/5.json... diff --git a/pkg/cli/testdata/zip/testzip_tenant b/pkg/cli/testdata/zip/testzip_tenant index 07191d2eef3b..a966e66d6d49 100644 --- a/pkg/cli/testdata/zip/testzip_tenant +++ b/pkg/cli/testdata/zip/testzip_tenant @@ -18,7 +18,7 @@ debug zip --concurrency=1 --cpu-profile-duration=1s /dev/null [cluster] requesting data for debug/reports/problemranges: last request failed: rpc error: ... [cluster] requesting data for debug/reports/problemranges: creating error output: debug/reports/problemranges.json.err.txt... done [cluster] retrieving list of system tables... done -[cluster] 37 system tables found +[cluster] 38 system tables found [cluster] retrieving SQL data for crdb_internal.cluster_contention_events... writing output: debug/crdb_internal.cluster_contention_events.txt... done [cluster] retrieving SQL data for crdb_internal.cluster_distsql_flows... writing output: debug/crdb_internal.cluster_distsql_flows.txt... done [cluster] retrieving SQL data for crdb_internal.cluster_database_privileges... writing output: debug/crdb_internal.cluster_database_privileges.txt... done @@ -68,6 +68,7 @@ debug zip --concurrency=1 --cpu-profile-duration=1s /dev/null [cluster] retrieving SQL data for system.replication_critical_localities... writing output: debug/system.replication_critical_localities.txt... done [cluster] retrieving SQL data for system.replication_stats... writing output: debug/system.replication_stats.txt... done [cluster] retrieving SQL data for system.reports_meta... writing output: debug/system.reports_meta.txt... done +[cluster] retrieving SQL data for system.role_id_seq... writing output: debug/system.role_id_seq.txt... done [cluster] retrieving SQL data for system.role_members... writing output: debug/system.role_members.txt... done [cluster] retrieving SQL data for system.role_options... writing output: debug/system.role_options.txt... done [cluster] retrieving SQL data for system.scheduled_jobs... writing output: debug/system.scheduled_jobs.txt... done diff --git a/pkg/clusterversion/cockroach_versions.go b/pkg/clusterversion/cockroach_versions.go index 72ec94d32ec7..7ff502942dbd 100644 --- a/pkg/clusterversion/cockroach_versions.go +++ b/pkg/clusterversion/cockroach_versions.go @@ -327,6 +327,16 @@ const ( // AlterSystemStatementStatisticsAddIndexRecommendations adds an // index_recommendations column to the system.statement_statistics table. AlterSystemStatementStatisticsAddIndexRecommendations + // RoleIDSequence is the version where the system.role_id_sequence exists. + RoleIDSequence + // AddSystemUserIDColumn is the version where the system.users table has + // a user_id column for writes only. + AddSystemUserIDColumn + // UsersHaveIDs is the version where all users in the system.users table + // have ids. + UsersHaveIDs + // SetUserIDNotNull sets the user_id column in system.users to not null. + SetUserIDNotNull // ************************************************* // Step (1): Add new versions here. @@ -564,7 +574,22 @@ var versionsSingleton = keyedVersions{ Key: AlterSystemStatementStatisticsAddIndexRecommendations, Version: roachpb.Version{Major: 22, Minor: 1, Internal: 32}, }, - + { + Key: RoleIDSequence, + Version: roachpb.Version{Major: 22, Minor: 1, Internal: 34}, + }, + { + Key: AddSystemUserIDColumn, + Version: roachpb.Version{Major: 22, Minor: 1, Internal: 36}, + }, + { + Key: UsersHaveIDs, + Version: roachpb.Version{Major: 22, Minor: 1, Internal: 38}, + }, + { + Key: SetUserIDNotNull, + Version: roachpb.Version{Major: 22, Minor: 1, Internal: 40}, + }, // ************************************************* // Step (2): Add new versions here. // Do not add new versions to a patch release. diff --git a/pkg/clusterversion/key_string.go b/pkg/clusterversion/key_string.go index 2087c66a50a7..5291251e37aa 100644 --- a/pkg/clusterversion/key_string.go +++ b/pkg/clusterversion/key_string.go @@ -58,11 +58,15 @@ func _() { _ = x[AlterSystemSQLInstancesAddLocality-47] _ = x[SystemExternalConnectionsTable-48] _ = x[AlterSystemStatementStatisticsAddIndexRecommendations-49] + _ = x[RoleIDSequence-50] + _ = x[AddSystemUserIDColumn-51] + _ = x[UsersHaveIDs-52] + _ = x[SetUserIDNotNull-53] } -const _Key_name = "V21_2Start22_1PebbleFormatBlockPropertyCollectorProbeRequestPublicSchemasWithDescriptorsEnsureSpanConfigReconciliationEnsureSpanConfigSubscriptionEnableSpanConfigStoreSCRAMAuthenticationUnsafeLossOfQuorumRecoveryRangeLogAlterSystemProtectedTimestampAddColumnEnableProtectedTimestampsForTenantDeleteCommentsWithDroppedIndexesRemoveIncompatibleDatabasePrivilegesAddRaftAppliedIndexTermMigrationPostAddRaftAppliedIndexTermMigrationDontProposeWriteTimestampForLeaseTransfersEnablePebbleFormatVersionBlockPropertiesMVCCIndexBackfillerEnableLeaseHolderRemovalLooselyCoupledRaftLogTruncationChangefeedIdlenessEnableDeclarativeSchemaChangerRowLevelTTLPebbleFormatSplitUserKeysMarkedEnableNewStoreRebalancerClusterLocksVirtualTableAutoStatsTableSettingsSuperRegionsEnableNewChangefeedOptionsSpanCountTablePreSeedSpanCountTableSeedSpanCountTableV22_1Start22_2LocalTimestampsPebbleFormatSplitUserKeysMarkedCompactedEnsurePebbleFormatVersionRangeKeysEnablePebbleFormatVersionRangeKeysTrigramInvertedIndexesRemoveGrantPrivilegeMVCCRangeTombstonesUpgradeSequenceToBeReferencedByIDSampledStmtDiagReqsAddSSTableTombstonesSystemPrivilegesTableEnablePredicateProjectionChangefeedAlterSystemSQLInstancesAddLocalitySystemExternalConnectionsTableAlterSystemStatementStatisticsAddIndexRecommendations" +const _Key_name = "V21_2Start22_1PebbleFormatBlockPropertyCollectorProbeRequestPublicSchemasWithDescriptorsEnsureSpanConfigReconciliationEnsureSpanConfigSubscriptionEnableSpanConfigStoreSCRAMAuthenticationUnsafeLossOfQuorumRecoveryRangeLogAlterSystemProtectedTimestampAddColumnEnableProtectedTimestampsForTenantDeleteCommentsWithDroppedIndexesRemoveIncompatibleDatabasePrivilegesAddRaftAppliedIndexTermMigrationPostAddRaftAppliedIndexTermMigrationDontProposeWriteTimestampForLeaseTransfersEnablePebbleFormatVersionBlockPropertiesMVCCIndexBackfillerEnableLeaseHolderRemovalLooselyCoupledRaftLogTruncationChangefeedIdlenessEnableDeclarativeSchemaChangerRowLevelTTLPebbleFormatSplitUserKeysMarkedEnableNewStoreRebalancerClusterLocksVirtualTableAutoStatsTableSettingsSuperRegionsEnableNewChangefeedOptionsSpanCountTablePreSeedSpanCountTableSeedSpanCountTableV22_1Start22_2LocalTimestampsPebbleFormatSplitUserKeysMarkedCompactedEnsurePebbleFormatVersionRangeKeysEnablePebbleFormatVersionRangeKeysTrigramInvertedIndexesRemoveGrantPrivilegeMVCCRangeTombstonesUpgradeSequenceToBeReferencedByIDSampledStmtDiagReqsAddSSTableTombstonesSystemPrivilegesTableEnablePredicateProjectionChangefeedAlterSystemSQLInstancesAddLocalitySystemExternalConnectionsTableAlterSystemStatementStatisticsAddIndexRecommendationsRoleIDSequenceAddSystemUserIDColumnUsersHaveIDsSetUserIDNotNull" -var _Key_index = [...]uint16{0, 5, 14, 48, 60, 88, 118, 146, 167, 186, 220, 258, 292, 324, 360, 392, 428, 470, 510, 529, 553, 584, 602, 632, 643, 674, 698, 722, 744, 756, 782, 796, 817, 835, 840, 849, 864, 904, 938, 972, 994, 1014, 1033, 1066, 1085, 1105, 1126, 1161, 1195, 1225, 1278} +var _Key_index = [...]uint16{0, 5, 14, 48, 60, 88, 118, 146, 167, 186, 220, 258, 292, 324, 360, 392, 428, 470, 510, 529, 553, 584, 602, 632, 643, 674, 698, 722, 744, 756, 782, 796, 817, 835, 840, 849, 864, 904, 938, 972, 994, 1014, 1033, 1066, 1085, 1105, 1126, 1161, 1195, 1225, 1278, 1292, 1313, 1325, 1341} func (i Key) String() string { if i < 0 || i >= Key(len(_Key_index)-1) { diff --git a/pkg/keys/constants.go b/pkg/keys/constants.go index 4db296eb3b44..4390db95ef57 100644 --- a/pkg/keys/constants.go +++ b/pkg/keys/constants.go @@ -460,6 +460,7 @@ const ( TenantUsageTableID = 45 SQLInstancesTableID = 46 SpanConfigurationsTableID = 47 + RoleIDSequenceID = 48 ) // CommentType the type of the schema object on which a comment has been diff --git a/pkg/security/username/username.go b/pkg/security/username/username.go index 59429c0703ca..4197a72bd30c 100644 --- a/pkg/security/username/username.go +++ b/pkg/security/username/username.go @@ -65,6 +65,9 @@ type SQLUsername struct { // NodeUser is used by nodes for intra-cluster traffic. const NodeUser = "node" +// NodeUserID is the ID for NodeUser. +const NodeUserID = 3 + // NodeUserName is the SQLUsername for NodeUser. func NodeUserName() SQLUsername { return SQLUsername{NodeUser} } @@ -74,6 +77,9 @@ func (s SQLUsername) IsNodeUser() bool { return s.u == NodeUser } // RootUser is the default cluster administrator. const RootUser = "root" +// RootUserID is the ID for RootUser. +const RootUserID = 1 + // RootUserName is the SQLUsername for RootUser. func RootUserName() SQLUsername { return SQLUsername{RootUser} } @@ -83,6 +89,9 @@ func (s SQLUsername) IsRootUser() bool { return s.u == RootUser } // AdminRole is the default (and non-droppable) role with superuser privileges. const AdminRole = "admin" +// AdminRoleID is the ID for admin. +const AdminRoleID = 2 + // AdminRoleName is the SQLUsername for AdminRole. func AdminRoleName() SQLUsername { return SQLUsername{AdminRole} } @@ -95,6 +104,25 @@ func (s SQLUsername) IsAdminRole() bool { return s.u == AdminRole } // It can be granted privileges, implicitly granting them to all users (current and future). const PublicRole = "public" +// PublicRoleID is the ID for public role. +const PublicRoleID = 4 + +// This map is immutable and should always hold. +// Right now this should always hold as we cannot rename any of the +// roles defined in this map. +// TODO(richardjcai): Add checks to ensure that this mapping always holds. +var roleNameToID = map[SQLUsername]int{ + RootUserName(): RootUserID, + AdminRoleName(): AdminRoleID, + NodeUserName(): NodeUserID, + PublicRoleName(): PublicRoleID, +} + +// GetDefaultRoleNameToID returns a role id for default roles. +func GetDefaultRoleNameToID(username SQLUsername) int { + return roleNameToID[username] +} + // NoneRole is a special role. // It is primarily used in SET ROLE, where "none" symbolizes a reset. const NoneRole = "none" diff --git a/pkg/server/admin_test.go b/pkg/server/admin_test.go index 2927e5f76b7f..fbec9541db01 100644 --- a/pkg/server/admin_test.go +++ b/pkg/server/admin_test.go @@ -48,6 +48,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/idxusage" + "github.com/cockroachdb/cockroach/pkg/sql/sem/catconstants" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb" "github.com/cockroachdb/cockroach/pkg/testutils" @@ -582,6 +583,11 @@ func TestRangeCount(t *testing.T) { } m[tableName] = tblResp.RangeCount } + // Hardcode the single range used by the role_id_seq, the above + // request does not return sequences. + // TODO(richardjcai): Maybe update the request to return + // sequences as well? + m[fmt.Sprintf("public.%s", catconstants.RoleIDSequenceName)] = 1 return m } @@ -970,8 +976,8 @@ func TestAdminAPIUsers(t *testing.T) { // Create sample users. query := ` -INSERT INTO system.users (username, "hashedPassword") -VALUES ('adminUser', 'abc'), ('bob', 'xyz')` +INSERT INTO system.users (username, "hashedPassword", user_id) +VALUES ('adminUser', 'abc', 200), ('bob', 'xyz', 201)` if _, err := db.Exec(query); err != nil { t.Fatal(err) } diff --git a/pkg/sql/catalog/bootstrap/metadata.go b/pkg/sql/catalog/bootstrap/metadata.go index cf85afffa79e..41067d73a301 100644 --- a/pkg/sql/catalog/bootstrap/metadata.go +++ b/pkg/sql/catalog/bootstrap/metadata.go @@ -172,6 +172,20 @@ func (ms MetadataSchema) GetInitialValues() ([]roachpb.KeyValue, []roachpb.RKey) add(catalogkeys.MakeSchemaNameKey(ms.codec, desc.GetID(), tree.PublicSchema), publicSchemaValue) } + // Set initial sequence values. + if tbl, ok := desc.(catalog.TableDescriptor); ok && tbl.IsSequence() && tbl.GetID() != keys.DescIDSequenceID { + // Note that we skip over the DescIDSequence here, + // the value is initialized earlier in this function. + // DescIDSequence is special cased such that there + // is a special "descIDGenerator" for the system tenant. + // Because of this, there is no DescIDSequence for + // the system tenant and thus this loop over descriptors + // will not initialize the value for the system tenant. + value := roachpb.Value{} + value.SetInt(tbl.GetSequenceOpts().Start) + add(ms.codec.SequenceKey(uint32(tbl.GetID())), value) + } + // Create descriptor metadata key. descValue := roachpb.Value{} if err := descValue.SetProto(desc.DescriptorProto()); err != nil { @@ -325,6 +339,7 @@ func addSystemDescriptorsToSchema(target *MetadataSchema) { // Tables introduced in 22.2. target.AddDescriptor(systemschema.SystemPrivilegeTable) target.AddDescriptor(systemschema.SystemExternalConnectionsTable) + target.AddDescriptor(systemschema.RoleIDSequence) // Adding a new system table? It should be added here to the metadata schema, // and also created as a migration for older clusters. diff --git a/pkg/sql/catalog/catprivilege/system.go b/pkg/sql/catalog/catprivilege/system.go index 4fe6ea55d0e5..ca5985970ef7 100644 --- a/pkg/sql/catalog/catprivilege/system.go +++ b/pkg/sql/catalog/catprivilege/system.go @@ -73,6 +73,9 @@ var ( // TODO(adityamaru,dt): Remove once we fix the handling of dynamic system // table IDs during restore. RestoreCopySystemTablePrefix = "crdb_internal_copy" + readWriteSystemSequences = []catconstants.SystemTableName{ + catconstants.RoleIDSequenceName, + } systemSuperuserPrivileges = func() map[descpb.NameInfo]privilege.List { m := make(map[descpb.NameInfo]privilege.List) @@ -88,6 +91,10 @@ var ( tableKey.Name = string(r) m[tableKey] = privilege.ReadData } + for _, r := range readWriteSystemSequences { + tableKey.Name = string(r) + m[tableKey] = privilege.ReadWriteSequenceData + } m[descpb.NameInfo{Name: catconstants.SystemDatabaseName}] = privilege.List{privilege.CONNECT} return m }() diff --git a/pkg/sql/catalog/descidgen/generate_id.go b/pkg/sql/catalog/descidgen/generate_id.go index 4d68b08fa1e2..112d6bd6f05f 100644 --- a/pkg/sql/catalog/descidgen/generate_id.go +++ b/pkg/sql/catalog/descidgen/generate_id.go @@ -83,3 +83,23 @@ func PeekNextUniqueDescID(ctx context.Context, db *kv.DB, codec keys.SQLCodec) ( } return descpb.ID(v.ValueInt()), nil } + +// GenerateUniqueRoleID returns the next available Role ID and increments +// the counter. The incrementing is non-transactional, and the counter could be +// incremented multiple times because of retries. +func GenerateUniqueRoleID(ctx context.Context, db *kv.DB, codec keys.SQLCodec) (int64, error) { + return IncrementUniqueRoleID(ctx, db, codec, 1) +} + +// IncrementUniqueRoleID returns the next available Role ID and increments +// the counter by inc. The incrementing is non-transactional, and the counter +// could be incremented multiple times because of retries. +func IncrementUniqueRoleID( + ctx context.Context, db *kv.DB, codec keys.SQLCodec, inc int64, +) (int64, error) { + newVal, err := kv.IncrementValRetryable(ctx, db, codec.SequenceKey(keys.RoleIDSequenceID), inc) + if err != nil { + return 0, err + } + return newVal - inc, nil +} diff --git a/pkg/sql/catalog/descs/collection.go b/pkg/sql/catalog/descs/collection.go index a40f45a9e31f..e4316ee1e40e 100644 --- a/pkg/sql/catalog/descs/collection.go +++ b/pkg/sql/catalog/descs/collection.go @@ -434,3 +434,13 @@ func (tc *Collection) AddDeletedDescriptor(id descpb.ID) { func (tc *Collection) SetSession(session sqlliveness.Session) { tc.sqlLivenessSession = session } + +// MakeTestCollection makes a Collection that can be used for tests. +func MakeTestCollection(ctx context.Context, leaseManager *lease.Manager) Collection { + settings := cluster.MakeTestingClusterSettings() + return Collection{ + settings: settings, + version: settings.Version.ActiveVersion(ctx), + leased: makeLeasedDescriptors(leaseManager), + } +} diff --git a/pkg/sql/catalog/systemschema/system.go b/pkg/sql/catalog/systemschema/system.go index 36c7761c3aea..b23a6435bd98 100644 --- a/pkg/sql/catalog/systemschema/system.go +++ b/pkg/sql/catalog/systemschema/system.go @@ -55,14 +55,17 @@ CREATE TABLE system.descriptor ( CONSTRAINT "primary" PRIMARY KEY (id) );` + // UsersTableSchema represents the system.users table. UsersTableSchema = ` CREATE TABLE system.users ( - username STRING, - "hashedPassword" BYTES, + username STRING NOT NULL, + "hashedPassword" BYTES NULL, "isRole" BOOL NOT NULL DEFAULT false, - CONSTRAINT "primary" PRIMARY KEY (username) + user_id OID NOT NULL, + CONSTRAINT "primary" PRIMARY KEY (username), + UNIQUE INDEX users_user_id_idx (user_id ASC), + FAMILY "primary" (username, user_id) );` - RoleOptionsTableSchema = ` CREATE TABLE system.role_options ( username STRING NOT NULL, @@ -101,6 +104,11 @@ CREATE TABLE system.tenants ( CONSTRAINT "primary" PRIMARY KEY (id), FAMILY "primary" (id, active, info) );` + + // RoleIDSequenceSchema starts at 100 so we have reserved IDs for special + // roles such as root and admin. + RoleIDSequenceSchema = ` +CREATE SEQUENCE system.role_id_seq START 100 MINVALUE 100 MAXVALUE 2147483647;` ) // These system tables are not part of the system config. @@ -758,8 +766,6 @@ func systemTable( NextMutationID: 1, NextConstraintID: 1, } - tbl.PrimaryIndex.ConstraintID = tbl.NextConstraintID - tbl.NextConstraintID++ for _, col := range columns { if tbl.NextColumnID <= col.ID { tbl.NextColumnID = col.ID + 1 @@ -775,11 +781,16 @@ func systemTable( tbl.NextIndexID = idx.ID + 1 } // Only assigned constraint IDs to unique non-primary indexes. - if idx.Unique && i > 1 { + if idx.Unique && i >= 1 { tbl.Indexes[i-1].ConstraintID = tbl.NextConstraintID tbl.NextConstraintID++ } } + + // When creating tables normally, unique index constraint ids are + // assigned before the primary index. + tbl.PrimaryIndex.ConstraintID = tbl.NextConstraintID + tbl.NextConstraintID++ return tbl } @@ -900,13 +911,24 @@ var ( {Name: "username", ID: 1, Type: types.String}, {Name: "hashedPassword", ID: 2, Type: types.Bytes, Nullable: true}, {Name: "isRole", ID: 3, Type: types.Bool, DefaultExpr: &falseBoolString}, + {Name: "user_id", ID: 4, Type: types.Oid}, }, []descpb.ColumnFamilyDescriptor{ - {Name: "primary", ID: 0, ColumnNames: []string{"username"}, ColumnIDs: singleID1}, + {Name: "primary", ID: 0, ColumnNames: []string{"username", "user_id"}, ColumnIDs: []descpb.ColumnID{1, 4}, DefaultColumnID: 4}, {Name: "fam_2_hashedPassword", ID: 2, ColumnNames: []string{"hashedPassword"}, ColumnIDs: []descpb.ColumnID{2}, DefaultColumnID: 2}, {Name: "fam_3_isRole", ID: 3, ColumnNames: []string{"isRole"}, ColumnIDs: []descpb.ColumnID{3}, DefaultColumnID: 3}, }, pk("username"), + descpb.IndexDescriptor{ + Name: "users_user_id_idx", + ID: 2, + Unique: true, + KeyColumnNames: []string{"user_id"}, + KeyColumnDirections: []catpb.IndexColumn_Direction{catpb.IndexColumn_ASC}, + KeyColumnIDs: []descpb.ColumnID{4}, + KeySuffixColumnIDs: []descpb.ColumnID{1}, + Version: descpb.StrictIndexColumnIDGuaranteesVersion, + }, )) // ZonesTable is the descriptor for the zones table. @@ -1002,6 +1024,51 @@ var ( }, ) + // RoleIDSequence is the descriptor for the role id sequence. + RoleIDSequence = registerSystemTable( + RoleIDSequenceSchema, + systemTable( + catconstants.RoleIDSequenceName, + keys.RoleIDSequenceID, + []descpb.ColumnDescriptor{ + {Name: tabledesc.SequenceColumnName, ID: tabledesc.SequenceColumnID, Type: types.Int}, + }, + []descpb.ColumnFamilyDescriptor{{ + Name: "primary", + ID: keys.SequenceColumnFamilyID, + ColumnNames: []string{tabledesc.SequenceColumnName}, + ColumnIDs: []descpb.ColumnID{tabledesc.SequenceColumnID}, + DefaultColumnID: tabledesc.SequenceColumnID, + }}, + descpb.IndexDescriptor{ + ID: keys.SequenceIndexID, + Name: tabledesc.LegacyPrimaryKeyIndexName, + KeyColumnIDs: []descpb.ColumnID{tabledesc.SequenceColumnID}, + KeyColumnNames: []string{tabledesc.SequenceColumnName}, + KeyColumnDirections: []catpb.IndexColumn_Direction{catpb.IndexColumn_ASC}, + }, + ), + func(tbl *descpb.TableDescriptor) { + opts := &descpb.TableDescriptor_SequenceOpts{ + Increment: 1, + MinValue: 100, + MaxValue: math.MaxInt32, + Start: 100, + CacheSize: 1, + } + tbl.SequenceOpts = opts + tbl.NextColumnID = 0 + tbl.NextFamilyID = 0 + tbl.NextIndexID = 0 + tbl.NextMutationID = 0 + // Sequences never exposed their internal constraints, + // so all IDs will be left at zero. CREATE SEQUENCE has + // the same behaviour. + tbl.NextConstraintID = 0 + tbl.PrimaryIndex.ConstraintID = 0 + }, + ) + TenantsTable = registerSystemTable( TenantsTableSchema, systemTable( diff --git a/pkg/sql/catalog/systemschema_test/testdata/bootstrap b/pkg/sql/catalog/systemschema_test/testdata/bootstrap index 1b1e73c8fb7d..b75a8b7610e0 100644 --- a/pkg/sql/catalog/systemschema_test/testdata/bootstrap +++ b/pkg/sql/catalog/systemschema_test/testdata/bootstrap @@ -12,8 +12,10 @@ CREATE TABLE public.users ( username STRING NOT NULL, "hashedPassword" BYTES NULL, "isRole" BOOL NOT NULL DEFAULT false, + user_id OID NOT NULL, CONSTRAINT "primary" PRIMARY KEY (username ASC), - FAMILY "primary" (username), + UNIQUE INDEX users_user_id_idx (user_id ASC), + FAMILY "primary" (username, user_id), FAMILY "fam_2_hashedPassword" ("hashedPassword"), FAMILY "fam_3_isRole" ("isRole") ); @@ -357,6 +359,7 @@ CREATE TABLE public.span_configurations ( CONSTRAINT "primary" PRIMARY KEY (start_key ASC), CONSTRAINT check_bounds CHECK (start_key < end_key) ); +CREATE SEQUENCE public.role_id_seq MINVALUE 100 MAXVALUE 2147483647 INCREMENT 1 START 100; CREATE TABLE public.tenant_settings ( tenant_id INT8 NOT NULL, name STRING NOT NULL, diff --git a/pkg/sql/catalog/tabledesc/safe_format_test.go b/pkg/sql/catalog/tabledesc/safe_format_test.go index 7e037cc56667..26e976abbea9 100644 --- a/pkg/sql/catalog/tabledesc/safe_format_test.go +++ b/pkg/sql/catalog/tabledesc/safe_format_test.go @@ -274,6 +274,8 @@ func TestSafeMessage(t *testing.T) { tc.id, tc.schema, catpb.NewBasePrivilegeDescriptor(username.RootUserName()), + nil, + nil, ) require.NoError(t, err) var td catalog.TableDescriptor diff --git a/pkg/sql/crdb_internal_test.go b/pkg/sql/crdb_internal_test.go index fff4059d9ade..6318a09354b3 100644 --- a/pkg/sql/crdb_internal_test.go +++ b/pkg/sql/crdb_internal_test.go @@ -439,7 +439,7 @@ CREATE TABLE nojob (k INT8); // Now introduce some inconsistencies. tdb.Exec(t, fmt.Sprintf(` -INSERT INTO system.users VALUES ('node', NULL, true); +INSERT INTO system.users VALUES ('node', NULL, true, 3); GRANT node TO root; DELETE FROM system.descriptor WHERE id = %d; DELETE FROM system.descriptor WHERE id = %d; diff --git a/pkg/sql/create_role.go b/pkg/sql/create_role.go index 5f33766a32cf..1af8cfa45800 100644 --- a/pkg/sql/create_role.go +++ b/pkg/sql/create_role.go @@ -15,9 +15,11 @@ import ( "fmt" "strings" + "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/security/password" "github.com/cockroachdb/cockroach/pkg/security/username" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descidgen" "github.com/cockroachdb/cockroach/pkg/sql/decodeusername" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" @@ -150,14 +152,22 @@ func (n *CreateRoleNode) startExec(params runParams) error { } // TODO(richardjcai): move hashedPassword column to system.role_options. + stmt := fmt.Sprintf("INSERT INTO %s VALUES ($1, $2, $3)", sessioninit.UsersTableName) + args := append(make([]interface{}, 0, 4), n.roleName, hashedPassword, n.isRole) + if params.ExecCfg().Settings.Version.IsActive(params.ctx, clusterversion.AddSystemUserIDColumn) { + stmt = fmt.Sprintf("INSERT INTO %s VALUES ($1, $2, $3, $4)", sessioninit.UsersTableName) + roleID, err := descidgen.GenerateUniqueRoleID(params.ctx, params.ExecCfg().DB, params.ExecCfg().Codec) + if err != nil { + return err + } + args = append(args, roleID) + } rowsAffected, err := params.extendedEvalCtx.ExecCfg.InternalExecutor.Exec( params.ctx, opName, params.p.txn, - fmt.Sprintf("insert into %s values ($1, $2, $3)", sessioninit.UsersTableName), - n.roleName, - hashedPassword, - n.isRole, + stmt, + args..., ) if err != nil { diff --git a/pkg/sql/exec_util.go b/pkg/sql/exec_util.go index 3d7fdfdb4a8b..623adc16f930 100644 --- a/pkg/sql/exec_util.go +++ b/pkg/sql/exec_util.go @@ -1723,6 +1723,14 @@ func golangFillQueryArguments(args ...interface{}) (tree.Datums, error) { } } d = a + case val.Type().Elem().Kind() == reflect.Int: + a := tree.NewDArray(types.Int) + for v := 0; v < val.Len(); v++ { + if err := a.Append(tree.NewDInt(tree.DInt(val.Index(v).Int()))); err != nil { + return nil, err + } + } + d = a case val.Type().Elem().Kind() == reflect.Uint8: d = tree.NewDBytes(tree.DBytes(val.Bytes())) } diff --git a/pkg/sql/execinfrapb/flow_diagram_external_test.go b/pkg/sql/execinfrapb/flow_diagram_external_test.go index 87658bbe1c49..f5b03f5a4dca 100644 --- a/pkg/sql/execinfrapb/flow_diagram_external_test.go +++ b/pkg/sql/execinfrapb/flow_diagram_external_test.go @@ -54,6 +54,8 @@ func TestPlanDiagramTableReaderWrapColumns(t *testing.T) { z INT )`, catpb.NewBasePrivilegeDescriptor(username.NodeUserName()), + nil, + nil, ) if err != nil { t.Fatal(err) diff --git a/pkg/sql/logictest/logic.go b/pkg/sql/logictest/logic.go index de74aed4d626..8e30021d774a 100644 --- a/pkg/sql/logictest/logic.go +++ b/pkg/sql/logictest/logic.go @@ -1612,8 +1612,19 @@ CREATE DATABASE test; USE test; t.Fatal(err) } - if _, err := t.db.Exec(fmt.Sprintf("CREATE USER %s;", username.TestUser)); err != nil { - t.Fatal(err) + if !t.cfg.BootstrapVersion.Equal(roachpb.Version{}) && t.cfg.BootstrapVersion.Less(roachpb.Version{Major: 22, Minor: 2}) { + // Hacky way to create user with an ID if we're on a + // bootstrapped binary less than 22.2. The version gate + // causes the regular CREATE USER to fail since it will not + // insert an ID. + if _, err := t.db.Exec(`INSERT INTO system.users VALUES ($1, '', false, $2);`, + username.TestUser, 100); err != nil { + t.Fatal(err) + } + } else { + if _, err := t.db.Exec(fmt.Sprintf("CREATE USER %s;", username.TestUser)); err != nil { + t.Fatal(err) + } } t.labelMap = make(map[string]string) diff --git a/pkg/sql/logictest/testdata/logic_test/grant_database b/pkg/sql/logictest/testdata/logic_test/grant_database index 27811a28d134..a2564b9f4127 100644 --- a/pkg/sql/logictest/testdata/logic_test/grant_database +++ b/pkg/sql/logictest/testdata/logic_test/grant_database @@ -22,7 +22,7 @@ statement error pq: user or role "test-user" does not exist GRANT ALL ON DATABASE a TO readwrite, "test-user" statement ok -INSERT INTO system.users VALUES('test-user',''); +INSERT INTO system.users VALUES('test-user','',false,3); statement ok GRANT ALL PRIVILEGES ON DATABASE a TO readwrite, "test-user" WITH GRANT OPTION diff --git a/pkg/sql/logictest/testdata/logic_test/grant_table b/pkg/sql/logictest/testdata/logic_test/grant_table index 4da62a62a2b3..35c033eee115 100644 --- a/pkg/sql/logictest/testdata/logic_test/grant_table +++ b/pkg/sql/logictest/testdata/logic_test/grant_table @@ -913,6 +913,12 @@ system public span_configurations root DELETE system public span_configurations root INSERT true system public span_configurations root SELECT true system public span_configurations root UPDATE true +system public role_id_seq admin SELECT true +system public role_id_seq admin UPDATE true +system public role_id_seq admin USAGE true +system public role_id_seq root SELECT true +system public role_id_seq root UPDATE true +system public role_id_seq root USAGE true system public tenant_settings admin DELETE true system public tenant_settings admin INSERT true system public tenant_settings admin SELECT true @@ -1351,6 +1357,9 @@ system public reports_meta root DELETE system public reports_meta root INSERT true system public reports_meta root SELECT true system public reports_meta root UPDATE true +system public role_id_seq root SELECT true +system public role_id_seq root UPDATE true +system public role_id_seq root USAGE true system public role_members root DELETE true system public role_members root INSERT true system public role_members root SELECT true @@ -1536,7 +1545,7 @@ a public t admin ALL true a public t root ALL true statement ok -INSERT INTO system.users VALUES('test-user',''); +INSERT INTO system.users VALUES('test-user','',false,200); statement ok GRANT ALL ON t TO readwrite, "test-user" diff --git a/pkg/sql/logictest/testdata/logic_test/information_schema b/pkg/sql/logictest/testdata/logic_test/information_schema index feaeb09410ac..4688f856689a 100644 --- a/pkg/sql/logictest/testdata/logic_test/information_schema +++ b/pkg/sql/logictest/testdata/logic_test/information_schema @@ -1565,6 +1565,8 @@ system public primary system public 630200280_28_1_not_null system public reports_meta CHECK NO NO system public 630200280_28_2_not_null system public reports_meta CHECK NO NO system public primary system public reports_meta PRIMARY KEY NO NO +system public 630200280_48_1_not_null system public role_id_seq CHECK NO NO +system public primary system public role_id_seq PRIMARY KEY NO NO system public 630200280_23_1_not_null system public role_members CHECK NO NO system public 630200280_23_2_not_null system public role_members CHECK NO NO system public 630200280_23_3_not_null system public role_members CHECK NO NO @@ -1659,7 +1661,9 @@ system public 630200280_14_3_not_null system public primary system public ui PRIMARY KEY NO NO system public 630200280_4_1_not_null system public users CHECK NO NO system public 630200280_4_3_not_null system public users CHECK NO NO +system public 630200280_4_4_not_null system public users CHECK NO NO system public primary system public users PRIMARY KEY NO NO +system public users_user_id_idx system public users UNIQUE NO NO system public 630200280_19_1_not_null system public web_sessions CHECK NO NO system public 630200280_19_2_not_null system public web_sessions CHECK NO NO system public 630200280_19_3_not_null system public web_sessions CHECK NO NO @@ -1824,8 +1828,10 @@ system public 630200280_46_1_not_null system public 630200280_47_1_not_null start_key IS NOT NULL system public 630200280_47_2_not_null end_key IS NOT NULL system public 630200280_47_3_not_null config IS NOT NULL +system public 630200280_48_1_not_null value IS NOT NULL system public 630200280_4_1_not_null username IS NOT NULL system public 630200280_4_3_not_null isRole IS NOT NULL +system public 630200280_4_4_not_null user_id IS NOT NULL system public 630200280_50_1_not_null tenant_id IS NOT NULL system public 630200280_50_2_not_null name IS NOT NULL system public 630200280_50_3_not_null value IS NOT NULL @@ -1899,6 +1905,7 @@ system public replication_critical_localities zone_id system public replication_stats subzone_id system public primary system public replication_stats zone_id system public primary system public reports_meta id system public primary +system public role_id_seq value system public primary system public role_members member system public primary system public role_members role system public primary system public role_options option system public primary @@ -1936,6 +1943,7 @@ system public transaction_statistics crdb_internal_aggr system public transaction_statistics fingerprint_id system public primary system public transaction_statistics node_id system public primary system public ui key system public primary +system public users user_id system public users_user_id_idx system public users username system public primary system public web_sessions id system public primary system public zones id system public primary @@ -2135,6 +2143,7 @@ system public replication_stats under_replicated_r system public replication_stats zone_id 1 system public reports_meta generated 2 system public reports_meta id 1 +system public role_id_seq value 1 system public role_members isAdmin 3 system public role_members member 2 system public role_members role 1 @@ -2243,6 +2252,7 @@ system public ui lastUpdated system public ui value 2 system public users hashedPassword 2 system public users isRole 3 +system public users user_id 4 system public users username 1 system public web_sessions auditInfo 8 system public web_sessions createdAt 4 @@ -3139,6 +3149,12 @@ NULL root system public reports_meta NULL root system public reports_meta INSERT YES NO NULL root system public reports_meta SELECT YES YES NULL root system public reports_meta UPDATE YES NO +NULL admin system public role_id_seq SELECT YES YES +NULL admin system public role_id_seq UPDATE YES NO +NULL admin system public role_id_seq USAGE YES NO +NULL root system public role_id_seq SELECT YES YES +NULL root system public role_id_seq UPDATE YES NO +NULL root system public role_id_seq USAGE YES NO NULL admin system public role_members DELETE YES NO NULL admin system public role_members INSERT YES NO NULL admin system public role_members SELECT YES YES @@ -3821,6 +3837,12 @@ NULL root system public span_configurations NULL root system public span_configurations INSERT YES NO NULL root system public span_configurations SELECT YES YES NULL root system public span_configurations UPDATE YES NO +NULL admin system public role_id_seq SELECT YES YES +NULL admin system public role_id_seq UPDATE YES NO +NULL admin system public role_id_seq USAGE YES NO +NULL root system public role_id_seq SELECT YES YES +NULL root system public role_id_seq UPDATE YES NO +NULL root system public role_id_seq USAGE YES NO NULL admin system public tenant_settings DELETE YES NO NULL admin system public tenant_settings INSERT YES NO NULL admin system public tenant_settings SELECT YES YES diff --git a/pkg/sql/logictest/testdata/logic_test/pg_catalog b/pkg/sql/logictest/testdata/logic_test/pg_catalog index ac49bf536977..2007bfaa0c91 100644 --- a/pkg/sql/logictest/testdata/logic_test/pg_catalog +++ b/pkg/sql/logictest/testdata/logic_test/pg_catalog @@ -1107,6 +1107,7 @@ ORDER BY indexrelid ---- indexrelid indrelid indnatts indisunique indisprimary indisexclusion indimmediate indisclustered indisvalid indcheckxmin indisready indislive indisreplident indkey indcollation indclass indoption indexprs indpred indnkeyatts 144368028 32 1 true true false true false true false false true false 1 0 0 2 NULL NULL 1 +190763692 48 1 false true false false false true false false true false 1 0 0 2 NULL NULL 1 404104299 39 1 true true false true false true false false true false 1 0 0 2 NULL NULL 1 543291288 23 1 false false false false false true false false true false 1 3403232968 0 2 NULL NULL 1 543291289 23 1 false false false false false true false false true false 2 3403232968 0 2 NULL NULL 1 @@ -1148,6 +1149,7 @@ indexrelid indrelid indnatts indisunique indisprimary indisexclusion indim 3353994584 36 1 true true false true false true false false true false 1 0 0 2 NULL NULL 1 3400390248 52 1 true true false true false true false false true false 1 3403232968 0 2 NULL NULL 1 3446785912 4 1 true true false true false true false false true false 1 3403232968 0 2 NULL NULL 1 +3446785915 4 1 true false false true false true false false true false 4 0 0 2 NULL NULL 1 3493181576 20 2 true true false true false true false false true false 1 2 0 0 0 0 2 2 NULL NULL 2 3613730852 43 1 false false false false false true false false true false 2 0 0 2 NULL NULL 1 3613730855 43 4 true true false true false true false false true false 1 2 3 4 0 0 3403232968 0 0 0 0 0 2 2 2 2 NULL NULL 4 @@ -1170,6 +1172,7 @@ ORDER BY indexrelid, operator_argument_position ---- indexrelid operator_argument_type_oid operator_argument_position 144368028 0 1 +190763692 0 1 404104299 0 1 543291288 0 1 543291289 0 1 @@ -1239,6 +1242,7 @@ indexrelid operator_argument_type_oid operator_argument_position 3353994584 0 1 3400390248 0 1 3446785912 0 1 +3446785915 0 1 3493181576 0 1 3493181576 0 2 3613730852 0 1 diff --git a/pkg/sql/logictest/testdata/logic_test/ranges b/pkg/sql/logictest/testdata/logic_test/ranges index 6413550e9a82..8dfa5f8132af 100644 Binary files a/pkg/sql/logictest/testdata/logic_test/ranges and b/pkg/sql/logictest/testdata/logic_test/ranges differ diff --git a/pkg/sql/logictest/testdata/logic_test/show_source b/pkg/sql/logictest/testdata/logic_test/show_source index c066430c5309..0fdb1eb5be67 100644 --- a/pkg/sql/logictest/testdata/logic_test/show_source +++ b/pkg/sql/logictest/testdata/logic_test/show_source @@ -237,94 +237,97 @@ query TT colnames SELECT * FROM [SHOW SEQUENCES FROM system] ---- sequence_schema sequence_name +public role_id_seq query TTTTT colnames,rowsort SELECT schema_name, table_name, type, owner, locality FROM [SHOW TABLES FROM system] ---- -schema_name table_name type owner locality -public descriptor table NULL NULL -public external_connections table NULL NULL -public privileges table NULL NULL -public tenant_settings table NULL NULL -public span_configurations table NULL NULL -public sql_instances table NULL NULL -public tenant_usage table NULL NULL -public database_role_settings table NULL NULL -public transaction_statistics table NULL NULL -public statement_statistics table NULL NULL -public join_tokens table NULL NULL -public migrations table NULL NULL -public sqlliveness table NULL NULL -public scheduled_jobs table NULL NULL -public statement_diagnostics table NULL NULL -public statement_diagnostics_requests table NULL NULL -public statement_bundle_chunks table NULL NULL -public role_options table NULL NULL -public protected_ts_records table NULL NULL -public protected_ts_meta table NULL NULL -public namespace table NULL NULL -public reports_meta table NULL NULL -public replication_stats table NULL NULL -public replication_critical_localities table NULL NULL -public replication_constraint_stats table NULL NULL -public comments table NULL NULL -public role_members table NULL NULL -public locations table NULL NULL -public table_statistics table NULL NULL -public web_sessions table NULL NULL -public jobs table NULL NULL -public ui table NULL NULL -public rangelog table NULL NULL -public eventlog table NULL NULL -public lease table NULL NULL -public tenants table NULL NULL -public settings table NULL NULL -public zones table NULL NULL -public users table NULL NULL +schema_name table_name type owner locality +public descriptor table NULL NULL +public external_connections table NULL NULL +public privileges table NULL NULL +public tenant_settings table NULL NULL +public role_id_seq sequence NULL NULL +public span_configurations table NULL NULL +public sql_instances table NULL NULL +public tenant_usage table NULL NULL +public database_role_settings table NULL NULL +public transaction_statistics table NULL NULL +public statement_statistics table NULL NULL +public join_tokens table NULL NULL +public migrations table NULL NULL +public sqlliveness table NULL NULL +public scheduled_jobs table NULL NULL +public statement_diagnostics table NULL NULL +public statement_diagnostics_requests table NULL NULL +public statement_bundle_chunks table NULL NULL +public role_options table NULL NULL +public protected_ts_records table NULL NULL +public protected_ts_meta table NULL NULL +public namespace table NULL NULL +public reports_meta table NULL NULL +public replication_stats table NULL NULL +public replication_critical_localities table NULL NULL +public replication_constraint_stats table NULL NULL +public comments table NULL NULL +public role_members table NULL NULL +public locations table NULL NULL +public table_statistics table NULL NULL +public web_sessions table NULL NULL +public jobs table NULL NULL +public ui table NULL NULL +public rangelog table NULL NULL +public eventlog table NULL NULL +public lease table NULL NULL +public tenants table NULL NULL +public settings table NULL NULL +public zones table NULL NULL +public users table NULL NULL query TTTTTT colnames,rowsort SELECT schema_name, table_name, type, owner, locality, comment FROM [SHOW TABLES FROM system WITH COMMENT] ---- -schema_name table_name type owner locality comment -public descriptor table NULL NULL Β· -public external_connections table NULL NULL Β· -public privileges table NULL NULL Β· -public tenant_settings table NULL NULL Β· -public span_configurations table NULL NULL Β· -public sql_instances table NULL NULL Β· -public tenant_usage table NULL NULL Β· -public database_role_settings table NULL NULL Β· -public transaction_statistics table NULL NULL Β· -public statement_statistics table NULL NULL Β· -public join_tokens table NULL NULL Β· -public migrations table NULL NULL Β· -public sqlliveness table NULL NULL Β· -public scheduled_jobs table NULL NULL Β· -public statement_diagnostics table NULL NULL Β· -public statement_diagnostics_requests table NULL NULL Β· -public statement_bundle_chunks table NULL NULL Β· -public role_options table NULL NULL Β· -public protected_ts_records table NULL NULL Β· -public protected_ts_meta table NULL NULL Β· -public namespace table NULL NULL Β· -public reports_meta table NULL NULL Β· -public replication_stats table NULL NULL Β· -public replication_critical_localities table NULL NULL Β· -public replication_constraint_stats table NULL NULL Β· -public comments table NULL NULL Β· -public role_members table NULL NULL Β· -public locations table NULL NULL Β· -public table_statistics table NULL NULL Β· -public web_sessions table NULL NULL Β· -public jobs table NULL NULL Β· -public ui table NULL NULL Β· -public rangelog table NULL NULL Β· -public eventlog table NULL NULL Β· -public lease table NULL NULL Β· -public tenants table NULL NULL Β· -public settings table NULL NULL Β· -public zones table NULL NULL Β· -public users table NULL NULL Β· +schema_name table_name type owner locality comment +public lease table NULL NULL Β· +public span_configurations table NULL NULL Β· +public database_role_settings table NULL NULL Β· +public join_tokens table NULL NULL Β· +public scheduled_jobs table NULL NULL Β· +public locations table NULL NULL Β· +public jobs table NULL NULL Β· +public ui table NULL NULL Β· +public eventlog table NULL NULL Β· +public tenants table NULL NULL Β· +public settings table NULL NULL Β· +public zones table NULL NULL Β· +public users table NULL NULL Β· +public descriptor table NULL NULL Β· +public external_connections table NULL NULL Β· +public role_id_seq sequence NULL NULL Β· +public tenant_usage table NULL NULL Β· +public statement_diagnostics_requests table NULL NULL Β· +public role_options table NULL NULL Β· +public protected_ts_records table NULL NULL Β· +public namespace table NULL NULL Β· +public replication_critical_localities table NULL NULL Β· +public web_sessions table NULL NULL Β· +public rangelog table NULL NULL Β· +public tenant_settings table NULL NULL Β· +public reports_meta table NULL NULL Β· +public replication_stats table NULL NULL Β· +public table_statistics table NULL NULL Β· +public privileges table NULL NULL Β· +public sql_instances table NULL NULL Β· +public transaction_statistics table NULL NULL Β· +public statement_statistics table NULL NULL Β· +public migrations table NULL NULL Β· +public sqlliveness table NULL NULL Β· +public statement_diagnostics table NULL NULL Β· +public statement_bundle_chunks table NULL NULL Β· +public protected_ts_meta table NULL NULL Β· +public replication_constraint_stats table NULL NULL Β· +public comments table NULL NULL Β· +public role_members table NULL NULL Β· query ITTT colnames SELECT node_id, user_name, application_name, active_queries diff --git a/pkg/sql/logictest/testdata/logic_test/system b/pkg/sql/logictest/testdata/logic_test/system index bc1584d7817f..51388eabb48f 100644 --- a/pkg/sql/logictest/testdata/logic_test/system +++ b/pkg/sql/logictest/testdata/logic_test/system @@ -12,45 +12,46 @@ skipif config 3node-tenant-default-configs query TTTTT SELECT schema_name, table_name, type, owner, locality FROM [SHOW TABLES FROM system] ORDER BY 2 ---- -public comments table NULL NULL -public database_role_settings table NULL NULL -public descriptor table NULL NULL -public eventlog table NULL NULL -public external_connections table NULL NULL -public jobs table NULL NULL -public join_tokens table NULL NULL -public lease table NULL NULL -public locations table NULL NULL -public migrations table NULL NULL -public namespace table NULL NULL -public privileges table NULL NULL -public protected_ts_meta table NULL NULL -public protected_ts_records table NULL NULL -public rangelog table NULL NULL -public replication_constraint_stats table NULL NULL -public replication_critical_localities table NULL NULL -public replication_stats table NULL NULL -public reports_meta table NULL NULL -public role_members table NULL NULL -public role_options table NULL NULL -public scheduled_jobs table NULL NULL -public settings table NULL NULL -public span_configurations table NULL NULL -public sql_instances table NULL NULL -public sqlliveness table NULL NULL -public statement_bundle_chunks table NULL NULL -public statement_diagnostics table NULL NULL -public statement_diagnostics_requests table NULL NULL -public statement_statistics table NULL NULL -public table_statistics table NULL NULL -public tenant_settings table NULL NULL -public tenant_usage table NULL NULL -public tenants table NULL NULL -public transaction_statistics table NULL NULL -public ui table NULL NULL -public users table NULL NULL -public web_sessions table NULL NULL -public zones table NULL NULL +public comments table NULL NULL +public database_role_settings table NULL NULL +public descriptor table NULL NULL +public eventlog table NULL NULL +public external_connections table NULL NULL +public jobs table NULL NULL +public join_tokens table NULL NULL +public lease table NULL NULL +public locations table NULL NULL +public migrations table NULL NULL +public namespace table NULL NULL +public privileges table NULL NULL +public protected_ts_meta table NULL NULL +public protected_ts_records table NULL NULL +public rangelog table NULL NULL +public replication_constraint_stats table NULL NULL +public replication_critical_localities table NULL NULL +public replication_stats table NULL NULL +public reports_meta table NULL NULL +public role_id_seq sequence NULL NULL +public role_members table NULL NULL +public role_options table NULL NULL +public scheduled_jobs table NULL NULL +public settings table NULL NULL +public span_configurations table NULL NULL +public sql_instances table NULL NULL +public sqlliveness table NULL NULL +public statement_bundle_chunks table NULL NULL +public statement_diagnostics table NULL NULL +public statement_diagnostics_requests table NULL NULL +public statement_statistics table NULL NULL +public table_statistics table NULL NULL +public tenant_settings table NULL NULL +public tenant_usage table NULL NULL +public tenants table NULL NULL +public transaction_statistics table NULL NULL +public ui table NULL NULL +public users table NULL NULL +public web_sessions table NULL NULL +public zones table NULL NULL onlyif config 3node-tenant-default-configs query TTTTT @@ -76,6 +77,7 @@ public replication_constraint_stats table NULL NULL public replication_critical_localities table NULL NULL public replication_stats table NULL NULL public reports_meta table NULL NULL +public role_id_seq sequence NULL NULL public role_members table NULL NULL public role_options table NULL NULL public scheduled_jobs table NULL NULL @@ -138,6 +140,7 @@ SELECT id FROM system.descriptor 45 46 47 +48 50 51 52 @@ -187,6 +190,7 @@ SELECT id FROM system.descriptor 43 44 46 +48 50 51 52 @@ -224,9 +228,10 @@ descriptor BYTES true NULL Β· {primary} false query TTBTTTB SHOW COLUMNS FROM system.users ---- -username STRING false NULL Β· {primary} false -hashedPassword BYTES true NULL Β· {primary} false -isRole BOOL false false Β· {primary} false +username STRING false NULL Β· {primary,users_user_id_idx} false +hashedPassword BYTES true NULL Β· {primary} false +isRole BOOL false false Β· {primary} false +user_id OID false NULL Β· {primary,users_user_id_idx} false query TTBTTTB SHOW COLUMNS FROM system.zones @@ -443,6 +448,12 @@ system public reports_meta root DELETE true system public reports_meta root INSERT true system public reports_meta root SELECT true system public reports_meta root UPDATE true +system public role_id_seq admin SELECT true +system public role_id_seq admin UPDATE true +system public role_id_seq admin USAGE true +system public role_id_seq root SELECT true +system public role_id_seq root UPDATE true +system public role_id_seq root USAGE true system public role_members admin DELETE true system public role_members admin INSERT true system public role_members admin SELECT true @@ -721,6 +732,12 @@ system public reports_meta root DELETE true system public reports_meta root INSERT true system public reports_meta root SELECT true system public reports_meta root UPDATE true +system public role_id_seq admin SELECT true +system public role_id_seq admin UPDATE true +system public role_id_seq admin USAGE true +system public role_id_seq root SELECT true +system public role_id_seq root UPDATE true +system public role_id_seq root USAGE true system public role_members admin DELETE true system public role_members admin INSERT true system public role_members admin SELECT true diff --git a/pkg/sql/logictest/testdata/logic_test/system_namespace b/pkg/sql/logictest/testdata/logic_test/system_namespace index 4f6f4a4d4bff..9fc090a9844b 100644 --- a/pkg/sql/logictest/testdata/logic_test/system_namespace +++ b/pkg/sql/logictest/testdata/logic_test/system_namespace @@ -26,6 +26,7 @@ SELECT * FROM system.namespace 1 29 replication_critical_localities 26 1 29 replication_stats 27 1 29 reports_meta 28 +1 29 role_id_seq 48 1 29 role_members 23 1 29 role_options 33 1 29 scheduled_jobs 37 @@ -82,6 +83,7 @@ SELECT * FROM system.namespace 1 29 replication_critical_localities 26 1 29 replication_stats 27 1 29 reports_meta 28 +1 29 role_id_seq 48 1 29 role_members 23 1 29 role_options 33 1 29 scheduled_jobs 37 diff --git a/pkg/sql/opt/exec/execbuilder/testdata/autocommit b/pkg/sql/opt/exec/execbuilder/testdata/autocommit index fc2bf1d280e6..4108e0c21f29 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/autocommit +++ b/pkg/sql/opt/exec/execbuilder/testdata/autocommit @@ -43,7 +43,7 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r51: sending batch 1 CPut, 1 EndTxn to (n1,s1):1 +dist sender send r52: sending batch 1 CPut, 1 EndTxn to (n1,s1):1 # Multi-row insert should auto-commit. query B @@ -66,7 +66,7 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r51: sending batch 2 CPut, 1 EndTxn to (n1,s1):1 +dist sender send r52: sending batch 2 CPut, 1 EndTxn to (n1,s1):1 # No auto-commit inside a transaction. statement ok @@ -90,7 +90,7 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r51: sending batch 2 CPut to (n1,s1):1 +dist sender send r52: sending batch 2 CPut to (n1,s1):1 statement ok ROLLBACK @@ -116,7 +116,7 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r51: sending batch 2 CPut, 1 EndTxn to (n1,s1):1 +dist sender send r52: sending batch 2 CPut, 1 EndTxn to (n1,s1):1 query B SELECT count(*) > 0 FROM [ @@ -139,7 +139,7 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r51: sending batch 2 CPut, 1 EndTxn to (n1,s1):1 +dist sender send r52: sending batch 2 CPut, 1 EndTxn to (n1,s1):1 # Insert with RETURNING statement with side-effects should not auto-commit. # In this case division can (in principle) error out. @@ -164,8 +164,8 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r51: sending batch 2 CPut to (n1,s1):1 -dist sender send r51: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r52: sending batch 2 CPut to (n1,s1):1 +dist sender send r52: sending batch 1 EndTxn to (n1,s1):1 # Another way to test the scenario above: generate an error and ensure that the # mutation was not committed. @@ -202,7 +202,7 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r51: sending batch 1 Put, 1 EndTxn to (n1,s1):1 +dist sender send r52: sending batch 1 Put, 1 EndTxn to (n1,s1):1 # Multi-row upsert should auto-commit. query B @@ -225,7 +225,7 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r51: sending batch 2 Put, 1 EndTxn to (n1,s1):1 +dist sender send r52: sending batch 2 Put, 1 EndTxn to (n1,s1):1 # No auto-commit inside a transaction. statement ok @@ -249,7 +249,7 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r51: sending batch 2 Put to (n1,s1):1 +dist sender send r52: sending batch 2 Put to (n1,s1):1 statement ok ROLLBACK @@ -275,7 +275,7 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r51: sending batch 2 Put, 1 EndTxn to (n1,s1):1 +dist sender send r52: sending batch 2 Put, 1 EndTxn to (n1,s1):1 # TODO(radu): allow non-side-effecting projections. query B @@ -299,8 +299,8 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r51: sending batch 2 Put to (n1,s1):1 -dist sender send r51: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r52: sending batch 2 Put to (n1,s1):1 +dist sender send r52: sending batch 1 EndTxn to (n1,s1):1 # Upsert with RETURNING statement with side-effects should not auto-commit. # In this case division can (in principle) error out. @@ -325,8 +325,8 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r51: sending batch 2 Put to (n1,s1):1 -dist sender send r51: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r52: sending batch 2 Put to (n1,s1):1 +dist sender send r52: sending batch 1 EndTxn to (n1,s1):1 # Another way to test the scenario above: generate an error and ensure that the # mutation was not committed. @@ -363,8 +363,8 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r51: sending batch 1 Scan to (n1,s1):1 -dist sender send r51: sending batch 2 Put, 1 EndTxn to (n1,s1):1 +dist sender send r52: sending batch 1 Scan to (n1,s1):1 +dist sender send r52: sending batch 2 Put, 1 EndTxn to (n1,s1):1 # No auto-commit inside a transaction. statement ok @@ -388,8 +388,8 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r51: sending batch 1 Scan to (n1,s1):1 -dist sender send r51: sending batch 2 Put to (n1,s1):1 +dist sender send r52: sending batch 1 Scan to (n1,s1):1 +dist sender send r52: sending batch 2 Put to (n1,s1):1 statement ok ROLLBACK @@ -415,8 +415,8 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r51: sending batch 1 Scan to (n1,s1):1 -dist sender send r51: sending batch 2 Put, 1 EndTxn to (n1,s1):1 +dist sender send r52: sending batch 1 Scan to (n1,s1):1 +dist sender send r52: sending batch 2 Put, 1 EndTxn to (n1,s1):1 # TODO(radu): allow non-side-effecting projections. query B @@ -440,9 +440,9 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r51: sending batch 1 Scan to (n1,s1):1 -dist sender send r51: sending batch 2 Put to (n1,s1):1 -dist sender send r51: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r52: sending batch 1 Scan to (n1,s1):1 +dist sender send r52: sending batch 2 Put to (n1,s1):1 +dist sender send r52: sending batch 1 EndTxn to (n1,s1):1 # Update with RETURNING statement with side-effects should not auto-commit. # In this case division can (in principle) error out. @@ -467,9 +467,9 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r51: sending batch 1 Scan to (n1,s1):1 -dist sender send r51: sending batch 2 Put to (n1,s1):1 -dist sender send r51: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r52: sending batch 1 Scan to (n1,s1):1 +dist sender send r52: sending batch 2 Put to (n1,s1):1 +dist sender send r52: sending batch 1 EndTxn to (n1,s1):1 # Another way to test the scenario above: generate an error and ensure that the # mutation was not committed. @@ -506,7 +506,7 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r51: sending batch 1 DelRng, 1 EndTxn to (n1,s1):1 +dist sender send r52: sending batch 1 DelRng, 1 EndTxn to (n1,s1):1 # Multi-row delete should auto-commit. query B @@ -529,7 +529,7 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r51: sending batch 1 DelRng, 1 EndTxn to (n1,s1):1 +dist sender send r52: sending batch 1 DelRng, 1 EndTxn to (n1,s1):1 # No auto-commit inside a transaction. statement ok @@ -553,7 +553,7 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r51: sending batch 1 DelRng to (n1,s1):1 +dist sender send r52: sending batch 1 DelRng to (n1,s1):1 statement ok ROLLBACK @@ -579,8 +579,8 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r51: sending batch 1 Scan to (n1,s1):1 -dist sender send r51: sending batch 2 Del, 1 EndTxn to (n1,s1):1 +dist sender send r52: sending batch 1 Scan to (n1,s1):1 +dist sender send r52: sending batch 2 Del, 1 EndTxn to (n1,s1):1 # TODO(radu): allow non-side-effecting projections. query B @@ -604,9 +604,9 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r51: sending batch 1 Scan to (n1,s1):1 -dist sender send r51: sending batch 2 Del to (n1,s1):1 -dist sender send r51: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r52: sending batch 1 Scan to (n1,s1):1 +dist sender send r52: sending batch 2 Del to (n1,s1):1 +dist sender send r52: sending batch 1 EndTxn to (n1,s1):1 # Insert with RETURNING statement with side-effects should not auto-commit. # In this case division can (in principle) error out. @@ -631,9 +631,9 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r51: sending batch 1 Scan to (n1,s1):1 -dist sender send r51: sending batch 2 Del to (n1,s1):1 -dist sender send r51: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r52: sending batch 1 Scan to (n1,s1):1 +dist sender send r52: sending batch 2 Del to (n1,s1):1 +dist sender send r52: sending batch 1 EndTxn to (n1,s1):1 statement ok INSERT INTO ab VALUES (12, 0); @@ -682,9 +682,9 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r51: sending batch 2 CPut to (n1,s1):1 -dist sender send r51: sending batch 2 Get to (n1,s1):1 -dist sender send r51: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r52: sending batch 2 CPut to (n1,s1):1 +dist sender send r52: sending batch 2 Get to (n1,s1):1 +dist sender send r52: sending batch 1 EndTxn to (n1,s1):1 query B SELECT count(*) > 0 FROM [ @@ -707,10 +707,10 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r51: sending batch 1 Scan to (n1,s1):1 -dist sender send r51: sending batch 1 Put to (n1,s1):1 -dist sender send r51: sending batch 1 Scan to (n1,s1):1 -dist sender send r51: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r52: sending batch 1 Scan to (n1,s1):1 +dist sender send r52: sending batch 1 Put to (n1,s1):1 +dist sender send r52: sending batch 1 Scan to (n1,s1):1 +dist sender send r52: sending batch 1 EndTxn to (n1,s1):1 query B SELECT count(*) > 0 FROM [ @@ -734,10 +734,10 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r51: sending batch 1 Get to (n1,s1):1 -dist sender send r51: sending batch 1 Del to (n1,s1):1 -dist sender send r51: sending batch 1 Scan to (n1,s1):1 -dist sender send r51: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r52: sending batch 1 Get to (n1,s1):1 +dist sender send r52: sending batch 1 Del to (n1,s1):1 +dist sender send r52: sending batch 1 Scan to (n1,s1):1 +dist sender send r52: sending batch 1 EndTxn to (n1,s1):1 # Test with a single cascade, which should use autocommit. statement ok @@ -767,9 +767,9 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r51: sending batch 1 DelRng to (n1,s1):1 -dist sender send r51: sending batch 1 Scan to (n1,s1):1 -dist sender send r51: sending batch 1 Del, 1 EndTxn to (n1,s1):1 +dist sender send r52: sending batch 1 DelRng to (n1,s1):1 +dist sender send r52: sending batch 1 Scan to (n1,s1):1 +dist sender send r52: sending batch 1 Del, 1 EndTxn to (n1,s1):1 # ----------------------- # Multiple mutation tests @@ -799,9 +799,9 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r51: sending batch 2 CPut to (n1,s1):1 -dist sender send r51: sending batch 2 CPut to (n1,s1):1 -dist sender send r51: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r52: sending batch 2 CPut to (n1,s1):1 +dist sender send r52: sending batch 2 CPut to (n1,s1):1 +dist sender send r52: sending batch 1 EndTxn to (n1,s1):1 query B SELECT count(*) > 0 FROM [ @@ -826,9 +826,9 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r51: sending batch 2 CPut to (n1,s1):1 -dist sender send r51: sending batch 2 CPut to (n1,s1):1 -dist sender send r51: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r52: sending batch 2 CPut to (n1,s1):1 +dist sender send r52: sending batch 2 CPut to (n1,s1):1 +dist sender send r52: sending batch 1 EndTxn to (n1,s1):1 # Check that the statement can still be auto-committed when the txn rows written # erring guardrail is enabled. @@ -861,7 +861,7 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r51: sending batch 1 CPut, 1 EndTxn to (n1,s1):1 +dist sender send r52: sending batch 1 CPut, 1 EndTxn to (n1,s1):1 query error pq: txn has written 2 rows, which is above the limit INSERT INTO guardrails VALUES (2), (3) diff --git a/pkg/sql/opt/exec/execbuilder/testdata/delete b/pkg/sql/opt/exec/execbuilder/testdata/delete index 8b6b205c0aa3..bc5eb13c1067 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/delete +++ b/pkg/sql/opt/exec/execbuilder/testdata/delete @@ -231,9 +231,9 @@ SELECT operation, message FROM [SHOW KV TRACE FOR SESSION] WHERE message LIKE '%DelRange%' OR message LIKE '%DelRng%' ---- batch flow coordinator DelRange /Table/110/1 - /Table/110/2 -dist sender send r51: sending batch 1 DelRng to (n1,s1):1 +dist sender send r52: sending batch 1 DelRng to (n1,s1):1 batch flow coordinator DelRange /Table/110/1/601/0 - /Table/110/2 -dist sender send r51: sending batch 1 DelRng to (n1,s1):1 +dist sender send r52: sending batch 1 DelRng to (n1,s1):1 # Ensure that DelRange requests are autocommitted when DELETE FROM happens on a # chunk of fewer than 600 keys. @@ -252,7 +252,7 @@ SELECT operation, message FROM [SHOW KV TRACE FOR SESSION] WHERE message LIKE '%DelRange%' OR message LIKE '%sending batch%' ---- batch flow coordinator DelRange /Table/110/1/5 - /Table/110/1/6 -dist sender send r51: sending batch 1 DelRng, 1 EndTxn to (n1,s1):1 +dist sender send r52: sending batch 1 DelRng, 1 EndTxn to (n1,s1):1 statement ok CREATE TABLE xyz ( diff --git a/pkg/sql/opt/exec/execbuilder/testdata/explain b/pkg/sql/opt/exec/execbuilder/testdata/explain index 99c1ca23e327..b60176af6fb3 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/explain +++ b/pkg/sql/opt/exec/execbuilder/testdata/explain @@ -674,14 +674,14 @@ vectorized: true β”‚ └── β€’ render β”‚ β”‚ estimated row count: 3 β”‚ β”‚ - β”‚ └── β€’ merge join (left outer) + β”‚ └── β€’ hash join (left outer) β”‚ β”‚ estimated row count: 3 β”‚ β”‚ equality: (username) = (username) β”‚ β”‚ left cols are key β”‚ β”‚ β”‚ β”œβ”€β”€ β€’ scan β”‚ β”‚ estimated row count: 3 (100% of the table; stats collected ago) - β”‚ β”‚ table: users@primary + β”‚ β”‚ table: users@users_user_id_idx β”‚ β”‚ spans: FULL SCAN β”‚ β”‚ β”‚ └── β€’ scan diff --git a/pkg/sql/opt/exec/execbuilder/testdata/inverted_filter_geospatial_dist b/pkg/sql/opt/exec/execbuilder/testdata/inverted_filter_geospatial_dist index 4da455d7bb59..aa962ed194c5 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/inverted_filter_geospatial_dist +++ b/pkg/sql/opt/exec/execbuilder/testdata/inverted_filter_geospatial_dist @@ -250,7 +250,7 @@ query ITTTI colnames,rowsort SELECT range_id, start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM TABLE geo_table] ---- range_id start_key end_key replicas lease_holder -51 NULL NULL {2} 2 +52 NULL NULL {2} 2 # We should see a distributed execution (though need to retry to purge possibly # stale dist sender caches). diff --git a/pkg/sql/opt/exec/execbuilder/testdata/show_trace b/pkg/sql/opt/exec/execbuilder/testdata/show_trace index bc7497e198e6..6b5fd66430d2 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/show_trace +++ b/pkg/sql/opt/exec/execbuilder/testdata/show_trace @@ -227,7 +227,7 @@ SET tracing = off query T SELECT message FROM [SHOW TRACE FOR SESSION] WHERE message LIKE e'%1 CPut, 1 EndTxn%' AND message NOT LIKE e'%proposing command%' ---- -r52: sending batch 1 CPut, 1 EndTxn to (n1,s1):1 +r53: sending batch 1 CPut, 1 EndTxn to (n1,s1):1 node received request: 1 CPut, 1 EndTxn # Check that we can run set tracing regardless of the current tracing state. @@ -311,9 +311,9 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r52: sending batch 1 CPut to (n1,s1):1 -dist sender send r52: sending batch 1 EndTxn to (n1,s1):1 -dist sender send r52: sending batch 2 CPut, 1 EndTxn to (n1,s1):1 +dist sender send r53: sending batch 1 CPut to (n1,s1):1 +dist sender send r53: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r53: sending batch 2 CPut, 1 EndTxn to (n1,s1):1 # Make another session trace. statement ok @@ -342,9 +342,9 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r52: sending batch 4 CPut, 1 EndTxn to (n1,s1):1 -dist sender send r52: sending batch 5 CPut to (n1,s1):1 -dist sender send r52: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r53: sending batch 4 CPut, 1 EndTxn to (n1,s1):1 +dist sender send r53: sending batch 5 CPut to (n1,s1):1 +dist sender send r53: sending batch 1 EndTxn to (n1,s1):1 # make a table with some big strings in it. statement ok @@ -365,11 +365,11 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r52: sending batch 6 CPut to (n1,s1):1 -dist sender send r52: sending batch 6 CPut to (n1,s1):1 -dist sender send r52: sending batch 6 CPut to (n1,s1):1 -dist sender send r52: sending batch 6 CPut to (n1,s1):1 -dist sender send r52: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r53: sending batch 6 CPut to (n1,s1):1 +dist sender send r53: sending batch 6 CPut to (n1,s1):1 +dist sender send r53: sending batch 6 CPut to (n1,s1):1 +dist sender send r53: sending batch 6 CPut to (n1,s1):1 +dist sender send r53: sending batch 1 EndTxn to (n1,s1):1 statement ok CREATE TABLE streamer (pk INT PRIMARY KEY, attribute INT, blob TEXT, INDEX(attribute), FAMILY (pk, attribute, blob)); @@ -398,4 +398,4 @@ WHERE message LIKE '%r$rangeid: sending batch 42 Get%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r52: sending batch 42 Get to (n1,s1):1 +dist sender send r53: sending batch 42 Get to (n1,s1):1 diff --git a/pkg/sql/opt/exec/execbuilder/testdata/sql_activity_stats_compaction b/pkg/sql/opt/exec/execbuilder/testdata/sql_activity_stats_compaction index 56b014da27f8..c1620a0b6d7b 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/sql_activity_stats_compaction +++ b/pkg/sql/opt/exec/execbuilder/testdata/sql_activity_stats_compaction @@ -3,7 +3,7 @@ # Ensure we can run DELETE statement on system.statement_statistics and # system.transaction_statistics table. statement ok -INSERT INTO system.users VALUES ('node', NULL, true); +INSERT INTO system.users VALUES ('node', NULL, true, 3); statement ok GRANT node TO root; diff --git a/pkg/sql/pgwire/pgwire_test.go b/pkg/sql/pgwire/pgwire_test.go index 2e4ee5a77bca..ff228730c3e5 100644 --- a/pkg/sql/pgwire/pgwire_test.go +++ b/pkg/sql/pgwire/pgwire_test.go @@ -537,11 +537,12 @@ func TestPGPreparedQuery(t *testing.T) { {"SHOW DATABASE", []preparedQueryTest{ baseTest.Results("defaultdb"), }}, - {"SHOW COLUMNS FROM system.users", []preparedQueryTest{ + {sql: "SHOW COLUMNS FROM system.users", ptest: []preparedQueryTest{ baseTest. - Results("username", "STRING", false, gosql.NullBool{}, "", "{primary}", false). + Results("username", "STRING", false, gosql.NullBool{}, "", "{primary,users_user_id_idx}", false). Results("hashedPassword", "BYTES", true, gosql.NullBool{}, "", "{primary}", false). - Results("isRole", "BOOL", false, false, "", "{primary}", false), + Results("isRole", "BOOL", false, false, "", "{primary}", false). + Results("user_id", "OID", false, gosql.NullBool{}, "", "{primary,users_user_id_idx}", false), }}, {"SELECT database_name, owner FROM [SHOW DATABASES]", []preparedQueryTest{ baseTest.Results("d", username.RootUser). @@ -562,16 +563,20 @@ func TestPGPreparedQuery(t *testing.T) { {"SHOW INDEXES FROM system.users", []preparedQueryTest{ baseTest.Results("users", "primary", false, 1, "username", "ASC", false, false, true). Results("users", "primary", false, 2, "hashedPassword", "N/A", true, false, true). - Results("users", "primary", false, 3, "isRole", "N/A", true, false, true), + Results("users", "primary", false, 3, "isRole", "N/A", true, false, true). + Results("users", "primary", false, 4, "user_id", "N/A", true, false, true). + Results("users", "users_user_id_idx", false, 1, "user_id", "ASC", false, false, true). + Results("users", "users_user_id_idx", false, 2, "username", "ASC", true, true, true), }}, {"SHOW TABLES FROM system", []preparedQueryTest{ - baseTest.Results("public", "comments", "table", gosql.NullString{}, 0, gosql.NullString{}).Others(38), + baseTest.Results("public", "comments", "table", gosql.NullString{}, 0, gosql.NullString{}).Others(39), }}, {"SHOW SCHEMAS FROM system", []preparedQueryTest{ baseTest.Results("crdb_internal", gosql.NullString{}).Others(4), }}, {"SHOW CONSTRAINTS FROM system.users", []preparedQueryTest{ - baseTest.Results("users", "primary", "PRIMARY KEY", "PRIMARY KEY (username ASC)", true), + baseTest.Results("users", "primary", "PRIMARY KEY", "PRIMARY KEY (username ASC)", true). + Results("users", "users_user_id_idx", "UNIQUE", "UNIQUE (user_id ASC)", true), }}, {"SHOW TIME ZONE", []preparedQueryTest{ baseTest.Results("UTC"), diff --git a/pkg/sql/pgwire/testdata/auth/special_cases b/pkg/sql/pgwire/testdata/auth/special_cases index 6d726f9ef302..0abceb160b70 100644 --- a/pkg/sql/pgwire/testdata/auth/special_cases +++ b/pkg/sql/pgwire/testdata/auth/special_cases @@ -102,7 +102,7 @@ subtest user_has_null_hashed_password_column # handled empty hashedPassword values. See #48769. sql -INSERT INTO system.users (username, "hashedPassword") VALUES ('nopassword', NULL) +INSERT INTO system.users (username, "hashedPassword", user_id) VALUES ('nopassword', NULL, 4) ---- ok diff --git a/pkg/sql/privilege/privilege.go b/pkg/sql/privilege/privilege.go index e2942c11c468..f0d1c0e2050c 100644 --- a/pkg/sql/privilege/privilege.go +++ b/pkg/sql/privilege/privilege.go @@ -103,14 +103,15 @@ var isDescriptorBacked = map[ObjectType]bool{ // Predefined sets of privileges. var ( - AllPrivileges = List{ALL, CONNECT, CREATE, DROP, SELECT, INSERT, DELETE, UPDATE, USAGE, ZONECONFIG, EXECUTE} - ReadData = List{SELECT} - ReadWriteData = List{SELECT, INSERT, DELETE, UPDATE} - DBPrivileges = List{ALL, CONNECT, CREATE, DROP, ZONECONFIG} - TablePrivileges = List{ALL, CREATE, DROP, SELECT, INSERT, DELETE, UPDATE, ZONECONFIG} - SchemaPrivileges = List{ALL, CREATE, USAGE} - TypePrivileges = List{ALL, USAGE} - FunctionPrivileges = List{ALL, EXECUTE} + AllPrivileges = List{ALL, CONNECT, CREATE, DROP, SELECT, INSERT, DELETE, UPDATE, USAGE, ZONECONFIG, EXECUTE} + ReadData = List{SELECT} + ReadWriteData = List{SELECT, INSERT, DELETE, UPDATE} + ReadWriteSequenceData = List{SELECT, UPDATE, USAGE} + DBPrivileges = List{ALL, CONNECT, CREATE, DROP, ZONECONFIG} + TablePrivileges = List{ALL, CREATE, DROP, SELECT, INSERT, DELETE, UPDATE, ZONECONFIG} + SchemaPrivileges = List{ALL, CREATE, USAGE} + TypePrivileges = List{ALL, USAGE} + FunctionPrivileges = List{ALL, EXECUTE} // SequencePrivileges is appended with TablePrivileges as well. This is because // before v22.2 we treated Sequences the same as Tables. This is to avoid making // certain privileges unavailable after upgrade migration. diff --git a/pkg/sql/sem/catconstants/constants.go b/pkg/sql/sem/catconstants/constants.go index c41074df33bf..92c5005b18a0 100644 --- a/pkg/sql/sem/catconstants/constants.go +++ b/pkg/sql/sem/catconstants/constants.go @@ -83,6 +83,7 @@ const ( SpanCountTableName SystemTableName = "span_count" SystemPrivilegeTableName SystemTableName = "privileges" SystemExternalConnectionsTableName SystemTableName = "external_connections" + RoleIDSequenceName SystemTableName = "role_id_seq" ) // Oid for virtual database and table. diff --git a/pkg/sql/stats/automatic_stats_test.go b/pkg/sql/stats/automatic_stats_test.go index bda2420a8207..ac512b98c7e7 100644 --- a/pkg/sql/stats/automatic_stats_test.go +++ b/pkg/sql/stats/automatic_stats_test.go @@ -752,7 +752,7 @@ func TestAnalyzeSystemTables(t *testing.T) { for _, tableName := range tableNames { // Stats may not be collected on system.lease and system.table_statistics. if tableName == "lease" || tableName == "table_statistics" || - tableName == "jobs" || tableName == "scheduled_jobs" { + tableName == "jobs" || tableName == "scheduled_jobs" || tableName == "role_id_seq" { continue } sql := fmt.Sprintf("ANALYZE system.%s", tableName) diff --git a/pkg/sql/table_test.go b/pkg/sql/table_test.go index 0850547be9fb..338e386d2f56 100644 --- a/pkg/sql/table_test.go +++ b/pkg/sql/table_test.go @@ -189,7 +189,7 @@ func TestMakeTableDescColumns(t *testing.T) { for i, d := range testData { s := "CREATE TABLE foo.test (a " + d.sqlType + " PRIMARY KEY, b " + d.sqlType + ")" schema, err := CreateTestTableDescriptor(context.Background(), 1, 100, s, - catpb.NewBasePrivilegeDescriptor(username.AdminRoleName())) + catpb.NewBasePrivilegeDescriptor(username.AdminRoleName()), nil, nil) if err != nil { t.Fatalf("%d: %v", i, err) } @@ -319,7 +319,7 @@ func TestMakeTableDescIndexes(t *testing.T) { for i, d := range testData { s := "CREATE TABLE foo.test (" + d.sql + ")" schema, err := CreateTestTableDescriptor(context.Background(), 1, 100, s, - catpb.NewBasePrivilegeDescriptor(username.AdminRoleName())) + catpb.NewBasePrivilegeDescriptor(username.AdminRoleName()), nil, nil) if err != nil { t.Fatalf("%d (%s): %v", i, d.sql, err) } @@ -393,7 +393,7 @@ func TestMakeTableDescUniqueConstraints(t *testing.T) { for i, d := range testData { s := "CREATE TABLE foo.test (" + d.sql + ")" schema, err := CreateTestTableDescriptor(context.Background(), 1, 100, s, - catpb.NewBasePrivilegeDescriptor(username.AdminRoleName())) + catpb.NewBasePrivilegeDescriptor(username.AdminRoleName()), nil, nil) if err != nil { t.Fatalf("%d (%s): %v", i, d.sql, err) } @@ -412,7 +412,7 @@ func TestPrimaryKeyUnspecified(t *testing.T) { s := "CREATE TABLE foo.test (a INT, b INT, CONSTRAINT c UNIQUE (b))" ctx := context.Background() desc, err := CreateTestTableDescriptor(ctx, 1, 100, s, - catpb.NewBasePrivilegeDescriptor(username.AdminRoleName())) + catpb.NewBasePrivilegeDescriptor(username.AdminRoleName()), nil, nil) if err != nil { t.Fatal(err) } diff --git a/pkg/sql/tests/BUILD.bazel b/pkg/sql/tests/BUILD.bazel index 2fb69954417f..2ac8f075ccb5 100644 --- a/pkg/sql/tests/BUILD.bazel +++ b/pkg/sql/tests/BUILD.bazel @@ -80,6 +80,7 @@ go_test( "//pkg/sql/catalog/descpb", "//pkg/sql/catalog/descs", "//pkg/sql/catalog/desctestutils", + "//pkg/sql/catalog/lease", "//pkg/sql/catalog/systemschema", "//pkg/sql/catalog/tabledesc", "//pkg/sql/parser", diff --git a/pkg/sql/tests/system_table_test.go b/pkg/sql/tests/system_table_test.go index ce51a42ef88e..e407962c0d87 100644 --- a/pkg/sql/tests/system_table_test.go +++ b/pkg/sql/tests/system_table_test.go @@ -17,6 +17,7 @@ import ( "strings" "testing" + "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/keys" @@ -28,9 +29,12 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descbuilder" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/lease" "github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/datadriven" @@ -48,10 +52,10 @@ func TestInitialKeys(t *testing.T) { var nonDescKeys int if systemTenant { codec = keys.SystemSQLCodec - nonDescKeys = 10 + nonDescKeys = 11 } else { codec = keys.MakeSQLCodec(roachpb.MakeTenantID(5)) - nonDescKeys = 3 + nonDescKeys = 4 } ms := bootstrap.MakeMetadataSchema(codec, zonepb.DefaultZoneConfigRef(), zonepb.DefaultSystemZoneConfigRef()) @@ -68,6 +72,8 @@ func TestInitialKeys(t *testing.T) { descpb.ID(1000 /* suitably large descriptor ID */), "CREATE TABLE system.x (val INTEGER PRIMARY KEY)", catpb.NewBasePrivilegeDescriptor(username.NodeUserName()), + nil, + nil, ) if err != nil { t.Fatal(err) @@ -166,6 +172,14 @@ func TestSystemTableLiterals(t *testing.T) { pkg catalog.TableDescriptor } + ctx := context.Background() + tc := testcluster.StartTestCluster(t, 2, base.TestClusterArgs{ + ReplicationMode: base.ReplicationManual, // saves time + }) + defer tc.Stopper().Stop(ctx) + + s := tc.Servers[0] + testcases := make(map[string]testcase) for schema, desc := range systemschema.SystemTableDescriptors { if _, alreadyExists := testcases[desc.GetName()]; alreadyExists { @@ -177,38 +191,46 @@ func TestSystemTableLiterals(t *testing.T) { } } - const expectedNumberOfSystemTables = 41 + const expectedNumberOfSystemTables = 42 require.Equal(t, expectedNumberOfSystemTables, len(testcases)) - for name, test := range testcases { - t.Run(name, func(t *testing.T) { - privs := *test.pkg.GetPrivileges() - desc := test.pkg - // Allocate an ID to dynamically allocated system tables. - if desc.GetID() == 0 { - mut := desc.NewBuilder().BuildCreatedMutable().(*tabledesc.Mutable) - mut.ID = keys.MaxReservedDescID + 1 - desc = mut.ImmutableCopy().(catalog.TableDescriptor) - } + runTest := func(name string, test testcase) { + privs := *test.pkg.GetPrivileges() + desc := test.pkg + // Allocate an ID to dynamically allocated system tables. + if desc.GetID() == 0 { + mut := desc.NewBuilder().BuildCreatedMutable().(*tabledesc.Mutable) + mut.ID = keys.MaxReservedDescID + 1 + desc = mut.ImmutableCopy().(catalog.TableDescriptor) + } + leaseManager := s.LeaseManager().(*lease.Manager) + collection := descs.MakeTestCollection(ctx, leaseManager) - gen, err := sql.CreateTestTableDescriptor( - context.Background(), - keys.SystemDatabaseID, - desc.GetID(), - test.schema, - &privs, - ) - if err != nil { - t.Fatalf("test: %+v, err: %v", test, err) - } - require.NoError(t, descbuilder.ValidateSelf(gen, clusterversion.TestingClusterVersion)) + gen, err := sql.CreateTestTableDescriptor( + context.Background(), + keys.SystemDatabaseID, + desc.GetID(), + test.schema, + &privs, + s.DB().NewTxn(ctx, "create-test-table-desc"), + &collection, + ) + if err != nil { + t.Fatalf("test: %+v, err: %v", test, err) + } + require.NoError(t, descbuilder.ValidateSelf(gen, clusterversion.TestingClusterVersion)) - if desc.TableDesc().Equal(gen.TableDesc()) { - return - } - diff := strings.Join(pretty.Diff(desc.TableDesc(), gen.TableDesc()), "\n") - t.Errorf("%s table descriptor generated from CREATE TABLE statement does not match "+ - "hardcoded table descriptor:\n%s", desc.GetName(), diff) + if desc.TableDesc().Equal(gen.TableDesc()) { + return + } + diff := strings.Join(pretty.Diff(desc.TableDesc(), gen.TableDesc()), "\n") + t.Errorf("%s table descriptor generated from CREATE TABLE statement does not match "+ + "hardcoded table descriptor:\n%s", desc.GetName(), diff) + } + + for name, test := range testcases { + t.Run(name, func(t *testing.T) { + runTest(name, test) }) } } diff --git a/pkg/sql/tests/testdata/initial_keys b/pkg/sql/tests/testdata/initial_keys index a727c24c3404..dd9eb02d2b8a 100644 --- a/pkg/sql/tests/testdata/initial_keys +++ b/pkg/sql/tests/testdata/initial_keys @@ -1,6 +1,6 @@ initial-keys tenant=system ---- -90 keys: +93 keys: /System/"desc-idgen" /Table/3/1/1/2/1 /Table/3/1/3/2/1 @@ -39,6 +39,7 @@ initial-keys tenant=system /Table/3/1/45/2/1 /Table/3/1/46/2/1 /Table/3/1/47/2/1 + /Table/3/1/48/2/1 /Table/3/1/50/2/1 /Table/3/1/51/2/1 /Table/3/1/52/2/1 @@ -71,6 +72,7 @@ initial-keys tenant=system /NamespaceTable/30/1/1/29/"replication_critical_localities"/4/1 /NamespaceTable/30/1/1/29/"replication_stats"/4/1 /NamespaceTable/30/1/1/29/"reports_meta"/4/1 + /NamespaceTable/30/1/1/29/"role_id_seq"/4/1 /NamespaceTable/30/1/1/29/"role_members"/4/1 /NamespaceTable/30/1/1/29/"role_options"/4/1 /NamespaceTable/30/1/1/29/"scheduled_jobs"/4/1 @@ -91,7 +93,8 @@ initial-keys tenant=system /NamespaceTable/30/1/1/29/"users"/4/1 /NamespaceTable/30/1/1/29/"web_sessions"/4/1 /NamespaceTable/30/1/1/29/"zones"/4/1 -45 splits: + /Table/48/1/0/0 +46 splits: /Table/3 /Table/4 /Table/5 @@ -134,13 +137,14 @@ initial-keys tenant=system /Table/45 /Table/46 /Table/47 + /Table/48 /Table/50 /Table/51 /Table/52 initial-keys tenant=5 ---- -79 keys: +82 keys: /Tenant/5/Table/3/1/1/2/1 /Tenant/5/Table/3/1/3/2/1 /Tenant/5/Table/3/1/4/2/1 @@ -176,6 +180,7 @@ initial-keys tenant=5 /Tenant/5/Table/3/1/43/2/1 /Tenant/5/Table/3/1/44/2/1 /Tenant/5/Table/3/1/46/2/1 + /Tenant/5/Table/3/1/48/2/1 /Tenant/5/Table/3/1/50/2/1 /Tenant/5/Table/3/1/51/2/1 /Tenant/5/Table/3/1/52/2/1 @@ -203,6 +208,7 @@ initial-keys tenant=5 /Tenant/5/NamespaceTable/30/1/1/29/"replication_critical_localities"/4/1 /Tenant/5/NamespaceTable/30/1/1/29/"replication_stats"/4/1 /Tenant/5/NamespaceTable/30/1/1/29/"reports_meta"/4/1 + /Tenant/5/NamespaceTable/30/1/1/29/"role_id_seq"/4/1 /Tenant/5/NamespaceTable/30/1/1/29/"role_members"/4/1 /Tenant/5/NamespaceTable/30/1/1/29/"role_options"/4/1 /Tenant/5/NamespaceTable/30/1/1/29/"scheduled_jobs"/4/1 @@ -220,12 +226,13 @@ initial-keys tenant=5 /Tenant/5/NamespaceTable/30/1/1/29/"users"/4/1 /Tenant/5/NamespaceTable/30/1/1/29/"web_sessions"/4/1 /Tenant/5/NamespaceTable/30/1/1/29/"zones"/4/1 + /Tenant/5/Table/48/1/0/0 1 splits: /Tenant/5 initial-keys tenant=999 ---- -79 keys: +82 keys: /Tenant/999/Table/3/1/1/2/1 /Tenant/999/Table/3/1/3/2/1 /Tenant/999/Table/3/1/4/2/1 @@ -261,6 +268,7 @@ initial-keys tenant=999 /Tenant/999/Table/3/1/43/2/1 /Tenant/999/Table/3/1/44/2/1 /Tenant/999/Table/3/1/46/2/1 + /Tenant/999/Table/3/1/48/2/1 /Tenant/999/Table/3/1/50/2/1 /Tenant/999/Table/3/1/51/2/1 /Tenant/999/Table/3/1/52/2/1 @@ -288,6 +296,7 @@ initial-keys tenant=999 /Tenant/999/NamespaceTable/30/1/1/29/"replication_critical_localities"/4/1 /Tenant/999/NamespaceTable/30/1/1/29/"replication_stats"/4/1 /Tenant/999/NamespaceTable/30/1/1/29/"reports_meta"/4/1 + /Tenant/999/NamespaceTable/30/1/1/29/"role_id_seq"/4/1 /Tenant/999/NamespaceTable/30/1/1/29/"role_members"/4/1 /Tenant/999/NamespaceTable/30/1/1/29/"role_options"/4/1 /Tenant/999/NamespaceTable/30/1/1/29/"scheduled_jobs"/4/1 @@ -305,5 +314,6 @@ initial-keys tenant=999 /Tenant/999/NamespaceTable/30/1/1/29/"users"/4/1 /Tenant/999/NamespaceTable/30/1/1/29/"web_sessions"/4/1 /Tenant/999/NamespaceTable/30/1/1/29/"zones"/4/1 + /Tenant/999/Table/48/1/0/0 1 splits: /Tenant/999 diff --git a/pkg/sql/testutils.go b/pkg/sql/testutils.go index 6b8b8450d62d..9bc57e3b9e5d 100644 --- a/pkg/sql/testutils.go +++ b/pkg/sql/testutils.go @@ -14,11 +14,13 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/dbdesc" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" "github.com/cockroachdb/cockroach/pkg/sql/catalog/schemadesc" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/clusterunique" @@ -35,7 +37,12 @@ import ( // Will fail on complex tables where that operation requires e.g. looking up // other tables. func CreateTestTableDescriptor( - ctx context.Context, parentID, id descpb.ID, schema string, privileges *catpb.PrivilegeDescriptor, + ctx context.Context, + parentID, id descpb.ID, + schema string, + privileges *catpb.PrivilegeDescriptor, + txn *kv.Txn, + collection *descs.Collection, ) (*tabledesc.Mutable, error) { st := cluster.MakeTestingClusterSettings() stmt, err := parser.ParseOne(schema) @@ -44,13 +51,18 @@ func CreateTestTableDescriptor( } semaCtx := tree.MakeSemaContext() evalCtx := eval.MakeTestingEvalContext(st) + sessionData := &sessiondata.SessionData{ + LocalOnlySessionData: sessiondatapb.LocalOnlySessionData{ + EnableUniqueWithoutIndexConstraints: true, + }, + } switch n := stmt.AST.(type) { case *tree.CreateTable: db := dbdesc.NewInitial(parentID, "test", username.RootUserName()) desc, err := NewTableDesc( ctx, nil, /* txn */ - nil, /* vs */ + NewSkippingCacheSchemaResolver(collection, sessiondata.NewStack(sessionData), txn, nil), st, n, db, @@ -59,14 +71,10 @@ func CreateTestTableDescriptor( nil, /* regionConfig */ hlc.Timestamp{}, /* creationTime */ privileges, - nil, /* affected */ + make(map[descpb.ID]*tabledesc.Mutable), &semaCtx, &evalCtx, - &sessiondata.SessionData{ - LocalOnlySessionData: sessiondatapb.LocalOnlySessionData{ - EnableUniqueWithoutIndexConstraints: true, - }, - }, /* sessionData */ + sessionData, tree.PersistencePermanent, ) return desc, err diff --git a/pkg/startupmigrations/migrations.go b/pkg/startupmigrations/migrations.go index cea7ef0d2474..f0547eedcc0d 100644 --- a/pkg/startupmigrations/migrations.go +++ b/pkg/startupmigrations/migrations.go @@ -780,7 +780,7 @@ func populateVersionSetting(ctx context.Context, r runner) error { func addRootUser(ctx context.Context, r runner) error { // Upsert the root user into the table. We intentionally override any existing entry. const upsertRootStmt = ` - UPSERT INTO system.users (username, "hashedPassword", "isRole") VALUES ($1, '', false) + UPSERT INTO system.users (username, "hashedPassword", "isRole", "user_id") VALUES ($1, '', false, 1) ` return r.execAsRootWithRetry(ctx, "addRootUser", upsertRootStmt, username.RootUser) } @@ -788,7 +788,7 @@ func addRootUser(ctx context.Context, r runner) error { func addAdminRole(ctx context.Context, r runner) error { // Upsert the admin role into the table. We intentionally override any existing entry. const upsertAdminStmt = ` - UPSERT INTO system.users (username, "hashedPassword", "isRole") VALUES ($1, '', true) + UPSERT INTO system.users (username, "hashedPassword", "isRole", "user_id") VALUES ($1, '', true, 2) ` return r.execAsRootWithRetry(ctx, "addAdminRole", upsertAdminStmt, username.AdminRole) } diff --git a/pkg/startupmigrations/migrations_test.go b/pkg/startupmigrations/migrations_test.go index 8b322678da4d..26dd6dc10d57 100644 --- a/pkg/startupmigrations/migrations_test.go +++ b/pkg/startupmigrations/migrations_test.go @@ -545,8 +545,8 @@ func TestAdminUserExists(t *testing.T) { // Create a user named "admin". We have to do a manual insert as "CREATE USER" // knows about "isRole", but the migration hasn't run yet. - mt.sqlDB.Exec(t, `INSERT INTO system.users (username, "hashedPassword") VALUES ($1, '')`, - username.AdminRole) + mt.sqlDB.Exec(t, `INSERT INTO system.users (username, "hashedPassword", user_id) VALUES ($1, '', $2)`, + username.AdminRole, username.AdminRoleID) // The revised migration in v2.1 upserts the admin user, so this should succeed. if err := mt.runMigration(ctx, migration); err != nil { @@ -566,8 +566,8 @@ func TestPublicRoleExists(t *testing.T) { // Create a user (we check for user or role) named "public". // We have to do a manual insert as "CREATE USER" knows to disallow "public". - mt.sqlDB.Exec(t, `INSERT INTO system.users (username, "hashedPassword", "isRole") VALUES ($1, '', false)`, - username.PublicRole) + mt.sqlDB.Exec(t, `INSERT INTO system.users (username, "hashedPassword", "isRole", user_id) VALUES ($1, '', false, $2)`, + username.PublicRole, username.PublicRoleID) e := `found a user named public which is now a reserved name.` // The revised migration in v2.1 upserts the admin user, so this should succeed. @@ -577,8 +577,8 @@ func TestPublicRoleExists(t *testing.T) { // Now try with a role instead of a user. mt.sqlDB.Exec(t, `DELETE FROM system.users WHERE username = $1`, username.PublicRole) - mt.sqlDB.Exec(t, `INSERT INTO system.users (username, "hashedPassword", "isRole") VALUES ($1, '', true)`, - username.PublicRole) + mt.sqlDB.Exec(t, `INSERT INTO system.users (username, "hashedPassword", "isRole", user_id) VALUES ($1, '', true, $2)`, + username.PublicRole, username.PublicRoleID) e = `found a role named public which is now a reserved name.` // The revised migration in v2.1 upserts the admin user, so this should succeed. diff --git a/pkg/upgrade/upgrades/BUILD.bazel b/pkg/upgrade/upgrades/BUILD.bazel index 41501400ef02..d4671d8a5624 100644 --- a/pkg/upgrade/upgrades/BUILD.bazel +++ b/pkg/upgrade/upgrades/BUILD.bazel @@ -9,10 +9,12 @@ go_library( "descriptor_utils.go", "precondition_before_starting_an_upgrade.go", "remove_grant_migration.go", + "role_id_sequence_migration.go", "sampled_stmt_diagnostics_requests.go", "schema_changes.go", "system_external_connections.go", "system_privileges.go", + "system_users_role_id_migration.go", "upgrade_sequence_to_be_referenced_by_ID.go", "upgrades.go", ], @@ -60,6 +62,7 @@ go_test( "main_test.go", "precondition_before_starting_an_upgrade_external_test.go", "remove_grant_migration_test.go", + "role_id_migration_test.go", "sampled_stmt_diagnostics_requests_test.go", "system_privileges_test.go", "upgrade_sequence_to_be_referenced_by_ID_external_test.go", @@ -79,6 +82,7 @@ go_test( "//pkg/settings/cluster", "//pkg/sql", "//pkg/sql/catalog", + "//pkg/sql/catalog/catalogkeys", "//pkg/sql/catalog/catpb", "//pkg/sql/catalog/descbuilder", "//pkg/sql/catalog/descpb", diff --git a/pkg/upgrade/upgrades/descriptor_utils.go b/pkg/upgrade/upgrades/descriptor_utils.go index 5486167ba7e0..11d52ba55a5e 100644 --- a/pkg/upgrade/upgrades/descriptor_utils.go +++ b/pkg/upgrade/upgrades/descriptor_utils.go @@ -46,18 +46,15 @@ func CreateSystemTableInTxn( // have dynamically allocated IDs. This method supports that, but also continues // to support adding fixed ID system table descriptors. tKey := catalogkeys.EncodeNameKey(codec, desc) - - // If this descriptor doesn't have an ID, which happens for dynamic - // system tables, we need to allocate it an ID. + // If we're going to allocate an ID, make sure the table does not exist. + got, err := txn.Get(ctx, tKey) + if err != nil { + return descpb.InvalidID, false, err + } + if got.Value.IsPresent() { + return descpb.InvalidID, false, nil + } if desc.GetID() == descpb.InvalidID { - // If we're going to allocate an ID, make sure the table does not exist. - got, err := txn.Get(ctx, tKey) - if err != nil { - return descpb.InvalidID, false, err - } - if got.Value.IsPresent() { - return descpb.InvalidID, false, nil - } id, err := descidgen.NewTransactionalGenerator(codec, txn). GenerateUniqueDescID(ctx) if err != nil { @@ -71,6 +68,9 @@ func CreateSystemTableInTxn( b := txn.NewBatch() b.CPut(tKey, desc.GetID(), nil) b.CPut(catalogkeys.MakeDescMetadataKey(codec, desc.GetID()), desc.DescriptorProto(), nil) + if desc.IsSequence() { + b.InitPut(codec.SequenceKey(uint32(desc.GetID())), desc.GetSequenceOpts().Start, false /* failOnTombstones */) + } if err := txn.Run(ctx, b); err != nil { return descpb.InvalidID, false, err } diff --git a/pkg/upgrade/upgrades/role_id_migration_test.go b/pkg/upgrade/upgrades/role_id_migration_test.go new file mode 100644 index 000000000000..d8ae0a36df67 --- /dev/null +++ b/pkg/upgrade/upgrades/role_id_migration_test.go @@ -0,0 +1,247 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package upgrades_test + +import ( + "context" + "fmt" + "sync" + "testing" + + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/keys" + "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/sql/catalog/catalogkeys" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema" + "github.com/cockroachdb/cockroach/pkg/sql/privilege" + "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/testutils/skip" + "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" + "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" + "github.com/cockroachdb/cockroach/pkg/upgrade/upgrades" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/stretchr/testify/require" +) + +func runTestRoleIDMigration(t *testing.T, numUsers int) { + defer leaktest.AfterTest(t)() + ctx := context.Background() + + settings := cluster.MakeTestingClusterSettingsWithVersions( + clusterversion.TestingBinaryVersion, + clusterversion.ByKey(clusterversion.RoleIDSequence-1), + false, + ) + + tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{ + ServerArgs: base.TestServerArgs{ + Settings: settings, + Knobs: base.TestingKnobs{ + Server: &server.TestingKnobs{ + DisableAutomaticVersionUpgrade: make(chan struct{}), + BinaryVersionOverride: clusterversion.ByKey(clusterversion.RoleIDSequence - 1), + }, + }, + }, + }) + defer tc.Stopper().Stop(ctx) + + db := tc.ServerConn(0) + defer db.Close() + tdb := sqlutils.MakeSQLRunner(db) + s := tc.Server(0) + + // Delete system.role_id_seq. + tdb.Exec(t, `INSERT INTO system.users VALUES ('node', '', false, 0)`) + tdb.Exec(t, `GRANT node TO root`) + tdb.Exec(t, `DROP SEQUENCE system.role_id_seq`) + tdb.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) + + // Remove entries from system.users. + tdb.Exec(t, `DELETE FROM system.users WHERE username = 'root' OR username ='admin' OR username='node'`) + + tdb.CheckQueryResults(t, `SELECT * FROM system.users`, [][]string{}) + + // Inject the old copy of the descriptor. + upgrades.InjectLegacyTable(ctx, t, s, systemschema.UsersTable, + getDeprecatedSystemUsersTable) + + // Rewrite entries into system.users. + tdb.Exec(t, `INSERT INTO system.users VALUES ('root', '', false), ('admin', '', true)`) + + tdb.CheckQueryResults(t, `SELECT * FROM system.users`, [][]string{ + {"admin", "", "true"}, + {"root", "", "false"}, + }) + + _, err = tc.Conns[0].ExecContext(ctx, `SET CLUSTER SETTING version = $1`, + clusterversion.ByKey(clusterversion.RoleIDSequence).String()) + require.NoError(t, err) + + tdb.CheckQueryResults(t, `SELECT * FROM system.role_id_seq`, [][]string{ + {"100", "0", "true"}, + }) + + if numUsers > 100 { + // Always create first user as testuser0. + tdb.Exec(t, `INSERT INTO system.users VALUES ('testuser0', NULL, false)`) + numUsers += 1 + var wg sync.WaitGroup + wg.Add(100) + // Make creating users faster. + for i := 0; i < 100; i++ { + // Each goroutine creates 100 users. + go func(capI int) { + defer wg.Done() + // This is hacky but INSERT into is faster + // than CREATE USER due to not having schema + // changes. This really affect our migration + // so let's insert to go faster. + for j := 0; j < numUsers/100; j++ { + tdb.Exec(t, fmt.Sprintf(`INSERT INTO system.users VALUES ('testuser%dx%d', '', false)`, capI, j)) + } + }(i) + } + wg.Wait() + } else { + for i := 0; i < numUsers; i++ { + tdb.Exec(t, fmt.Sprintf(`CREATE USER testuser%d`, i)) + } + } + + var wg sync.WaitGroup + if numUsers > 100 { + wg.Add(1) + // Create other users in parallel while the migration is happening. + go func() { + for i := 0; i < 1000; i++ { + tdb.Exec(t, fmt.Sprintf(`CREATE USER parallel_user_creation_%d`, i)) + } + wg.Done() + }() + } + _, err = tc.Conns[0].ExecContext(ctx, `SET CLUSTER SETTING version = $1`, + clusterversion.ByKey(clusterversion.AddSystemUserIDColumn).String()) + require.NoError(t, err) + + _, err = tc.Conns[0].ExecContext(ctx, `SET CLUSTER SETTING version = $1`, + clusterversion.ByKey(clusterversion.UsersHaveIDs).String()) + require.NoError(t, err) + + _, err = tc.Conns[0].ExecContext(ctx, `SET CLUSTER SETTING version = $1`, + clusterversion.ByKey(clusterversion.SetUserIDNotNull).String()) + require.NoError(t, err) + + tdb.CheckQueryResults(t, `SELECT * FROM system.users WHERE user_id IS NULL`, [][]string{}) + tdb.Exec(t, `CREATE USER testuser_last`) + + if numUsers <= 100 { + tdb.CheckQueryResults(t, `SELECT * FROM system.users WHERE username IN ('admin', 'root', 'testuser0', 'testuser_last')`, [][]string{ + {"admin", "", "true", "2"}, + {"root", "", "false", "1"}, + {"testuser0", "NULL", "false", "100"}, + {"testuser_last", "NULL", "false", fmt.Sprint(100 + numUsers)}, + }) + } else { + // When we create more than 100 users, we also concurrency + // create some users which makes the id of testuser_last + // non-deterministic. + tdb.CheckQueryResults(t, `SELECT * FROM system.users WHERE username IN ('admin', 'root')`, [][]string{ + {"admin", "", "true", "2"}, + {"root", "", "false", "1"}, + }) + tdb.CheckQueryResults(t, `SELECT user_id > 100000 FROM system.users WHERE username = 'testuser_last'`, [][]string{ + {"true"}, + }) + } + + // Verify that the schema is correct. + expectedSchema := `CREATE TABLE public.users ( + username STRING NOT NULL, + "hashedPassword" BYTES NULL, + "isRole" BOOL NOT NULL DEFAULT false, + user_id OID NOT NULL, + CONSTRAINT "primary" PRIMARY KEY (username ASC), + UNIQUE INDEX users_user_id_idx (user_id ASC), + FAMILY "primary" (username, user_id), + FAMILY "fam_2_hashedPassword" ("hashedPassword"), + FAMILY "fam_3_isRole" ("isRole") +)` + row := tc.Conns[0].QueryRow(`SELECT create_statement FROM [SHOW CREATE TABLE system.users]`) + var actualSchema string + err = row.Scan(&actualSchema) + require.NoError(t, err) + + require.Equal(t, expectedSchema, actualSchema) + + // Wait for users to finish being created. + wg.Wait() +} + +func TestRoleIDMigration1User(t *testing.T) { + runTestRoleIDMigration(t, 1) +} + +func TestRoleIDMigration100User(t *testing.T) { + runTestRoleIDMigration(t, 100) +} + +func TestRoleIDMigration100000Users(t *testing.T) { + skip.UnderStress(t) + runTestRoleIDMigration(t, 100000) +} + +func getDeprecatedSystemUsersTable() *descpb.TableDescriptor { + falseBoolString := "false" + + return &descpb.TableDescriptor{ + Name: "users", + ID: keys.UsersTableID, + ParentID: keys.SystemDatabaseID, + UnexposedParentSchemaID: keys.PublicSchemaID, + Version: 1, + Columns: []descpb.ColumnDescriptor{ + {Name: "username", ID: 1, Type: types.String}, + {Name: "hashedPassword", ID: 2, Type: types.Bytes, Nullable: true}, + {Name: "isRole", ID: 3, Type: types.Bool, DefaultExpr: &falseBoolString}, + }, + NextColumnID: 4, + Families: []descpb.ColumnFamilyDescriptor{ + {Name: "primary", ID: 0, ColumnNames: []string{"username"}, ColumnIDs: []descpb.ColumnID{1}}, + {Name: "fam_2_hashedPassword", ID: 2, ColumnNames: []string{"hashedPassword"}, ColumnIDs: []descpb.ColumnID{2}, DefaultColumnID: 2}, + {Name: "fam_3_isRole", ID: 3, ColumnNames: []string{"isRole"}, ColumnIDs: []descpb.ColumnID{3}, DefaultColumnID: 3}, + }, + NextFamilyID: 4, + PrimaryIndex: descpb.IndexDescriptor{ + Name: "primary", + ID: 1, + Unique: true, + KeyColumnNames: []string{"username"}, + KeyColumnDirections: []catpb.IndexColumn_Direction{catpb.IndexColumn_ASC}, + KeyColumnIDs: []descpb.ColumnID{1}, + }, + NextIndexID: 2, + Privileges: catpb.NewCustomSuperuserPrivilegeDescriptor(privilege.ReadWriteData, username.NodeUserName()), + FormatVersion: descpb.InterleavedFormatVersion, + NextMutationID: 1, + NextConstraintID: 1, + } +} diff --git a/pkg/upgrade/upgrades/role_id_sequence_migration.go b/pkg/upgrade/upgrades/role_id_sequence_migration.go new file mode 100644 index 000000000000..978a6b6c59a7 --- /dev/null +++ b/pkg/upgrade/upgrades/role_id_sequence_migration.go @@ -0,0 +1,29 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package upgrades + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/jobs" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema" + "github.com/cockroachdb/cockroach/pkg/upgrade" +) + +// roleIDSequenceMigration creates the system.role_id_sequence sequence. +func roleIDSequenceMigration( + ctx context.Context, _ clusterversion.ClusterVersion, d upgrade.TenantDeps, _ *jobs.Job, +) error { + return createSystemTable( + ctx, d.DB, d.Codec, systemschema.RoleIDSequence, + ) +} diff --git a/pkg/upgrade/upgrades/system_privileges_test.go b/pkg/upgrade/upgrades/system_privileges_test.go index b85f6c246fae..529faad0aea8 100644 --- a/pkg/upgrade/upgrades/system_privileges_test.go +++ b/pkg/upgrade/upgrades/system_privileges_test.go @@ -54,7 +54,7 @@ func TestSystemPrivilegesMigration(t *testing.T) { tdb := sqlutils.MakeSQLRunner(db) // Delete system.role_id_seq. - tdb.Exec(t, `INSERT INTO system.users VALUES ('node', '', false)`) + tdb.Exec(t, `INSERT INTO system.users VALUES ('node', '', false, 3)`) tdb.Exec(t, `GRANT node TO root`) tdb.Exec(t, `DROP TABLE system.privileges`) tdb.Exec(t, `REVOKE node FROM root`) diff --git a/pkg/upgrade/upgrades/system_users_role_id_migration.go b/pkg/upgrade/upgrades/system_users_role_id_migration.go new file mode 100644 index 000000000000..ed65caabc863 --- /dev/null +++ b/pkg/upgrade/upgrades/system_users_role_id_migration.go @@ -0,0 +1,173 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package upgrades + +import ( + "context" + "fmt" + "strings" + + "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/jobs" + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/security/username" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descidgen" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" + "github.com/cockroachdb/cockroach/pkg/upgrade" +) + +// The migration is broken down into four steps. +// 1. Adding the "user_id" column as NULL. +// All newly created users will have an ID after step 1, this gives us a +// cut off point on which users we need to backfill. +// 2. Manually backfill the id column. +// 3. Set the column to not null. +// We need to do this because we cannot add a column with a nextval call as a +// default expression. +// It results in: unimplemented: cannot evaluate scalar expressions +// containing sequence operations in this context. +// The column family is specified such that the bootstrapped table +// definition and the migration end up with the same table state. +const addUserIDColumn = ` +ALTER TABLE system.users +ADD COLUMN IF NOT EXISTS "user_id" OID FAMILY "primary" +` + +const updateUserIDColumnSetNotNull = ` +ALTER TABLE system.users ALTER COLUMN "user_id" SET NOT NULL +` + +const addUserIDIndex = ` +CREATE UNIQUE INDEX users_user_id_idx ON system.users ("user_id" ASC) +` + +func alterSystemUsersAddUserIDColumnWithIndex( + ctx context.Context, cs clusterversion.ClusterVersion, d upgrade.TenantDeps, _ *jobs.Job, +) error { + for _, op := range []operation{ + { + name: "add-system-users-user-id-column", + schemaList: []string{"user_id"}, + query: addUserIDColumn, + schemaExistsFn: hasColumn, + }, + { + name: "alter-system-users-add-index", + schemaList: []string{"users_user_id_idx"}, + query: addUserIDIndex, + schemaExistsFn: hasIndex, + }, + } { + if err := migrateTable(ctx, cs, d, op, keys.UsersTableID, systemschema.UsersTable); err != nil { + return err + } + } + + return nil +} + +func backfillSystemUsersIDColumn( + ctx context.Context, cs clusterversion.ClusterVersion, d upgrade.TenantDeps, _ *jobs.Job, +) error { + var upsertRootStmt = ` + UPSERT INTO system.users (username, "isRole", "user_id") VALUES ($1, false, $2) + ` + + var upsertAdminStmt = ` + UPSERT INTO system.users (username, "isRole", "user_id") VALUES ($1, true, $2) + ` + + _, err := d.InternalExecutor.ExecEx(ctx, "upsert-root-user-in-role-id-migration", nil, + sessiondata.NodeUserSessionDataOverride, upsertRootStmt, username.RootUser, username.RootUserID) + if err != nil { + return err + } + + _, err = d.InternalExecutor.ExecEx(ctx, "upsert-admin-role-in-role-id-migration", nil, + sessiondata.NodeUserSessionDataOverride, upsertAdminStmt, username.AdminRole, username.AdminRoleID) + if err != nil { + return err + } + + row, err := d.InternalExecutor.QueryRowEx(ctx, `get-num-null-user-ids`, nil, sessiondata.NodeUserSessionDataOverride, + `SELECT count(1) FROM system.users WHERE user_id IS NULL`) + if err != nil { + return err + } + numUsersToUpdate := int(tree.MustBeDInt(row[0])) + const batchSize = 10000 + for i := 0; i < numUsersToUpdate; i += batchSize { + numIDs := batchSize + if numUsersToUpdate-i < batchSize { + numIDs = numUsersToUpdate - i + } + startID, err := descidgen.IncrementUniqueRoleID(ctx, d.DB, d.Codec, int64(numIDs)) + if err != nil { + return err + } + + ids := make([]int, numIDs) + for j := range ids { + ids[j] = int(startID) + j + } + + updateUserIDs := fmt.Sprintf(` +UPDATE system.users + SET user_id = id + FROM ( + SELECT * + FROM ( + SELECT row_number() OVER () AS rn, * + FROM system.users + WHERE user_id IS NULL + ) AS t1 + JOIN ( + SELECT row_number() OVER () AS rn, + unnest AS id + FROM ROWS FROM (unnest($1)) + ) AS t2 ON t1.rn = t2.rn + ) AS temp + WHERE system.users.username = temp.username + AND system.users.user_id IS NULL + LIMIT %d`, numIDs) + + _, err = d.InternalExecutor.ExecEx(ctx, "update-role-ids", nil, + sessiondata.NodeUserSessionDataOverride, updateUserIDs, ids) + if err != nil { + return err + } + } + return nil +} + +func setUserIDNotNull( + ctx context.Context, cs clusterversion.ClusterVersion, d upgrade.TenantDeps, _ *jobs.Job, +) error { + op := operation{ + name: "alter-system-users-user-id-column-not-null", + schemaList: []string{"user_id"}, + query: updateUserIDColumnSetNotNull, + schemaExistsFn: func(storedTable, _ catalog.TableDescriptor, colName string) (bool, error) { + storedCol, err := storedTable.FindColumnWithName(tree.Name(colName)) + if err != nil { + if strings.Contains(err.Error(), "does not exist") { + return false, nil + } + return false, err + } + return !storedCol.IsNullable(), nil + }, + } + return migrateTable(ctx, cs, d, op, keys.UsersTableID, systemschema.UsersTable) +} diff --git a/pkg/upgrade/upgrades/upgrades.go b/pkg/upgrade/upgrades/upgrades.go index 2d85e6d2e4a9..84e3b4cdf09f 100644 --- a/pkg/upgrade/upgrades/upgrades.go +++ b/pkg/upgrade/upgrades/upgrades.go @@ -96,6 +96,32 @@ var upgrades = []upgrade.Upgrade{ NoPrecondition, alterSystemStatementStatisticsAddIndexRecommendations, ), + upgrade.NewTenantUpgrade("add system.role_id_sequence", + toCV(clusterversion.RoleIDSequence), + NoPrecondition, + roleIDSequenceMigration, + ), + // Add user_id column, the column will not be backfilled. + // However, new users created from this point forward will be created + // with an ID. We cannot start using the IDs in this version + // as old users are not backfilled. The key here is that we have a cut + // off point where we know which users need to be backfilled and no + // more users can be created without ids. + upgrade.NewTenantUpgrade("alter system.users to include user_id column", + toCV(clusterversion.AddSystemUserIDColumn), + NoPrecondition, + alterSystemUsersAddUserIDColumnWithIndex, + ), + upgrade.NewTenantUpgrade("backfill users with ids and add an index on the id column", + toCV(clusterversion.UsersHaveIDs), + NoPrecondition, + backfillSystemUsersIDColumn, + ), + upgrade.NewTenantUpgrade("set user_id column to not null", + toCV(clusterversion.SetUserIDNotNull), + NoPrecondition, + setUserIDNotNull, + ), } func init() {