Skip to content

Commit

Permalink
Merge #98897 #98958
Browse files Browse the repository at this point in the history
98897: server: fix tenant auth for status server r=knz,THardy98 a=dhartunian

Previously, the authentication for gRPC endpoints that are exposed
via HTTP on the tenant was not implemented correctly. Because the HTTP
session is decoded into gRPC metadata, and that metadata was contained
in the Context object passed through to the Tenant Connector, the
username from the tenant could leak into the kv layer and be treated
as an authenticated username. If that username happened to match one
in the system tenant it would be accepted as valid.

Additinally, some endpoints were missing their authentication code.
This did not break functionality because a gRPC request without any
metadata is treated as an internal request with admin permissions.

*Warning*: If a request contains a validated username as part of gRPC
metadata and that metadata is preserved as the request is handed down
to the KV layer, it could be interpreted as a valid user on the system
tenant and cause an escalation of privileges.

This commit adds authentication to the HotRangesV2 endpoint and
SpanStats endpoints which were missing it, and contains tests that
ensure that the endpoints return errors when the user does not have
the correct permissions.

Epic: CRDB-12100

Release note: None

98958: sql,backupccl: set system table user ID columns to be NOT NULL r=rafiss,stevendanna a=andyyang890

This PR sets the user ID columns in system tables to be NOT NULL
and when applicable, updates the `RESTORE` logic to account for
the case where a backup may have been created before the user ID
column was added.

Part of #87079

Release note: None

Co-authored-by: David Hartunian <[email protected]>
Co-authored-by: Andy Yang <[email protected]>
  • Loading branch information
3 people committed Mar 21, 2023
3 parents b89fa2c + b0d563a + d3268f9 commit 1cdcf18
Show file tree
Hide file tree
Showing 59 changed files with 638 additions and 76 deletions.
148 changes: 146 additions & 2 deletions pkg/ccl/backupccl/restore_old_versions_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -58,8 +58,11 @@ func TestRestoreOldVersions(t *testing.T) {
defer log.Scope(t).Close(t)
testdataBase := datapathutils.TestDataPath(t, "restore_old_versions")
var (
clusterDirs = testdataBase + "/cluster"
systemRoleMembersDirs = testdataBase + "/system-role-members-restore"
clusterDirs = testdataBase + "/cluster"
systemRoleMembersDirs = testdataBase + "/system-role-members-restore"
systemPrivilegesDirs = testdataBase + "/system-privileges-restore"
systemDatabaseRoleSettingsDirs = testdataBase + "/system-database-role-settings-restore"
systemExternalConnectionsDirs = testdataBase + "/system-external-connections-restore"
)

t.Run("cluster-restore", func(t *testing.T) {
Expand Down Expand Up @@ -87,6 +90,39 @@ func TestRestoreOldVersions(t *testing.T) {
t.Run(dir.Name(), fullClusterRestoreSystemRoleMembersWithoutIDs(exportDir))
}
})

t.Run("full-cluster-restore-system-privileges-without-ids", func(t *testing.T) {
dirs, err := os.ReadDir(systemPrivilegesDirs)
require.NoError(t, err)
for _, dir := range dirs {
require.True(t, dir.IsDir())
exportDir, err := filepath.Abs(filepath.Join(systemPrivilegesDirs, dir.Name()))
require.NoError(t, err)
t.Run(dir.Name(), fullClusterRestoreSystemPrivilegesWithoutIDs(exportDir))
}
})

t.Run("full-cluster-restore-system-database-role-settings-without-ids", func(t *testing.T) {
dirs, err := os.ReadDir(systemDatabaseRoleSettingsDirs)
require.NoError(t, err)
for _, dir := range dirs {
require.True(t, dir.IsDir())
exportDir, err := filepath.Abs(filepath.Join(systemDatabaseRoleSettingsDirs, dir.Name()))
require.NoError(t, err)
t.Run(dir.Name(), fullClusterRestoreSystemDatabaseRoleSettingsWithoutIDs(exportDir))
}
})

t.Run("full-cluster-restore-system-external-connections-without-ids", func(t *testing.T) {
dirs, err := os.ReadDir(systemExternalConnectionsDirs)
require.NoError(t, err)
for _, dir := range dirs {
require.True(t, dir.IsDir())
exportDir, err := filepath.Abs(filepath.Join(systemExternalConnectionsDirs, dir.Name()))
require.NoError(t, err)
t.Run(dir.Name(), fullClusterRestoreSystemExternalConnectionsWithoutIDs(exportDir))
}
})
}

func restoreOldVersionClusterTest(exportDir string) func(t *testing.T) {
Expand Down Expand Up @@ -300,3 +336,111 @@ func fullClusterRestoreSystemRoleMembersWithoutIDs(exportDir string) func(t *tes
})
}
}

func fullClusterRestoreSystemPrivilegesWithoutIDs(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)

// The restore queries are run with `UNSAFE_RESTORE_INCOMPATIBLE_VERSION`
// option to ensure the restore is successful on development branches. This
// is because, while the backups were generated on release branches and have
// versions such as 22.2 in their manifest, the development branch will have
// a BinaryMinSupportedVersion offset by the clusterversion.DevOffset
// described in `pkg/clusterversion/cockroach_versions.go`. This will mean
// that the manifest version is always less than the
// BinaryMinSupportedVersion which will in turn fail the restore unless we
// pass in the specified option to elide the compatability check.
sqlDB.Exec(t, fmt.Sprintf("RESTORE FROM '%s' WITH UNSAFE_RESTORE_INCOMPATIBLE_VERSION", localFoo))

sqlDB.CheckQueryResults(t, "SELECT * FROM system.privileges", [][]string{
{"testuser1", "/global/", "{VIEWACTIVITY}", "{}", "100"},
{"testuser2", "/global/", "{MODIFYCLUSTERSETTING}", "{}", "101"},
})
}
}

func fullClusterRestoreSystemDatabaseRoleSettingsWithoutIDs(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)

// The restore queries are run with `UNSAFE_RESTORE_INCOMPATIBLE_VERSION`
// option to ensure the restore is successful on development branches. This
// is because, while the backups were generated on release branches and have
// versions such as 22.2 in their manifest, the development branch will have
// a BinaryMinSupportedVersion offset by the clusterversion.DevOffset
// described in `pkg/clusterversion/cockroach_versions.go`. This will mean
// that the manifest version is always less than the
// BinaryMinSupportedVersion which will in turn fail the restore unless we
// pass in the specified option to elide the compatability check.
sqlDB.Exec(t, fmt.Sprintf("RESTORE FROM '%s' WITH UNSAFE_RESTORE_INCOMPATIBLE_VERSION", localFoo))

sqlDB.CheckQueryResults(t, "SELECT * FROM system.database_role_settings", [][]string{
{"0", "", "{timezone=America/New_York}", "0"},
{"0", "testuser1", "{application_name=roachdb}", "100"},
{"0", "testuser2", "{disallow_full_table_scans=on}", "101"},
})
}
}

func fullClusterRestoreSystemExternalConnectionsWithoutIDs(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)

// The restore queries are run with `UNSAFE_RESTORE_INCOMPATIBLE_VERSION`
// option to ensure the restore is successful on development branches. This
// is because, while the backups were generated on release branches and have
// versions such as 22.2 in their manifest, the development branch will have
// a BinaryMinSupportedVersion offset by the clusterversion.DevOffset
// described in `pkg/clusterversion/cockroach_versions.go`. This will mean
// that the manifest version is always less than the
// BinaryMinSupportedVersion which will in turn fail the restore unless we
// pass in the specified option to elide the compatability check.
sqlDB.Exec(t, fmt.Sprintf("RESTORE FROM '%s' WITH UNSAFE_RESTORE_INCOMPATIBLE_VERSION", localFoo))

sqlDB.CheckQueryResults(t, "SELECT * FROM system.external_connections", [][]string{
{"connection1", "2023-03-20 01:26:50.174781 +0000 +0000", "2023-03-20 01:26:50.174781 +0000 +0000", "STORAGE",
"\b\u0005\u0012\u0019\n\u0017userfile:///connection1", "testuser1", "100"},
{"connection2", "2023-03-20 01:26:51.223986 +0000 +0000", "2023-03-20 01:26:51.223986 +0000 +0000", "STORAGE",
"\b\u0005\u0012\u0019\n\u0017userfile:///connection2", "testuser2", "101"},
})
}
}
166 changes: 166 additions & 0 deletions pkg/ccl/backupccl/system_schema.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/security/username"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
Expand Down Expand Up @@ -362,6 +363,165 @@ func roleOptionsRestoreFunc(
return nil
}

func systemPrivilegesRestoreFunc(
ctx context.Context,
deps customRestoreFuncDeps,
txn isql.Txn,
systemTableName, tempTableName string,
) error {
if !deps.settings.Version.IsActive(ctx, clusterversion.V23_1SystemPrivilegesTableHasUserIDColumn) {
return defaultSystemTableRestoreFunc(ctx, deps, txn, systemTableName, tempTableName)
}

hasUserIDColumn, err := tableHasColumnName(ctx, txn, tempTableName, "user_id")
if err != nil {
return err
}
if hasUserIDColumn {
return defaultSystemTableRestoreFunc(ctx, deps, txn, systemTableName, tempTableName)
}

deleteQuery := fmt.Sprintf("DELETE FROM system.%s WHERE true", systemTableName)
log.Eventf(ctx, "clearing data from system table %s with query %q", systemTableName, deleteQuery)

_, err = txn.Exec(ctx, systemTableName+"-data-deletion", txn.KV(), deleteQuery)
if err != nil {
return errors.Wrapf(err, "deleting data from system.%s", systemTableName)
}

systemPrivilegesRows, err := txn.QueryBufferedEx(ctx, systemTableName+"-query-all-rows",
txn.KV(), sessiondata.NodeUserSessionDataOverride,
fmt.Sprintf(`SELECT * FROM %s`, tempTableName),
)
if err != nil {
return err
}

restoreQuery := fmt.Sprintf(`
INSERT INTO system.%s (username, path, privileges, grant_options, user_id)
VALUES ($1, $2, $3, $4, (
SELECT CASE $1
WHEN '%s' THEN %d
ELSE (SELECT user_id FROM system.users WHERE username = $1)
END
))`,
systemTableName, username.PublicRole, username.PublicRoleID)
for _, row := range systemPrivilegesRows {
if _, err := txn.ExecEx(ctx, systemTableName+"-data-insert",
txn.KV(), sessiondata.NodeUserSessionDataOverride,
restoreQuery, row[0], row[1], row[2], row[3],
); err != nil {
return errors.Wrapf(err, "inserting data to system.%s", systemTableName)
}
}

return nil
}

func systemDatabaseRoleSettingsRestoreFunc(
ctx context.Context,
deps customRestoreFuncDeps,
txn isql.Txn,
systemTableName, tempTableName string,
) error {
if !deps.settings.Version.IsActive(ctx, clusterversion.V23_1DatabaseRoleSettingsHasRoleIDColumn) {
return defaultSystemTableRestoreFunc(ctx, deps, txn, systemTableName, tempTableName)
}

hasRoleIDColumn, err := tableHasColumnName(ctx, txn, tempTableName, "role_id")
if err != nil {
return err
}
if hasRoleIDColumn {
return defaultSystemTableRestoreFunc(ctx, deps, txn, systemTableName, tempTableName)
}

deleteQuery := fmt.Sprintf("DELETE FROM system.%s WHERE true", systemTableName)
log.Eventf(ctx, "clearing data from system table %s with query %q", systemTableName, deleteQuery)

_, err = txn.Exec(ctx, systemTableName+"-data-deletion", txn.KV(), deleteQuery)
if err != nil {
return errors.Wrapf(err, "deleting data from system.%s", systemTableName)
}

databaseRoleSettingsRows, err := txn.QueryBufferedEx(ctx, systemTableName+"-query-all-rows",
txn.KV(), sessiondata.NodeUserSessionDataOverride,
fmt.Sprintf(`SELECT * FROM %s`, tempTableName),
)
if err != nil {
return err
}

restoreQuery := fmt.Sprintf(`
INSERT INTO system.%s (database_id, role_name, settings, role_id)
VALUES ($1, $2, $3, (
SELECT CASE $2
WHEN '%s' THEN %d
ELSE (SELECT user_id FROM system.users WHERE username = $2)
END
))`,
systemTableName, username.EmptyRole, username.EmptyRoleID)
for _, row := range databaseRoleSettingsRows {
if _, err := txn.ExecEx(ctx, systemTableName+"-data-insert",
txn.KV(), sessiondata.NodeUserSessionDataOverride,
restoreQuery, row[0], row[1], row[2],
); err != nil {
return errors.Wrapf(err, "inserting data to system.%s", systemTableName)
}
}

return nil
}

func systemExternalConnectionsRestoreFunc(
ctx context.Context,
deps customRestoreFuncDeps,
txn isql.Txn,
systemTableName, tempTableName string,
) error {
if !deps.settings.Version.IsActive(ctx, clusterversion.V23_1ExternalConnectionsTableHasOwnerIDColumn) {
return defaultSystemTableRestoreFunc(ctx, deps, txn, systemTableName, tempTableName)
}

hasOwnerIDColumn, err := tableHasColumnName(ctx, txn, tempTableName, "owner_id")
if err != nil {
return err
}
if hasOwnerIDColumn {
return defaultSystemTableRestoreFunc(ctx, deps, txn, systemTableName, tempTableName)
}

deleteQuery := fmt.Sprintf("DELETE FROM system.%s WHERE true", systemTableName)
log.Eventf(ctx, "clearing data from system table %s with query %q", systemTableName, deleteQuery)

_, err = txn.Exec(ctx, systemTableName+"-data-deletion", txn.KV(), deleteQuery)
if err != nil {
return errors.Wrapf(err, "deleting data from system.%s", systemTableName)
}

externalConnectionsRows, err := txn.QueryBufferedEx(ctx, systemTableName+"-query-all-rows",
txn.KV(), sessiondata.NodeUserSessionDataOverride,
fmt.Sprintf(`SELECT * FROM %s`, tempTableName),
)
if err != nil {
return err
}

restoreQuery := fmt.Sprintf(`
INSERT INTO system.%s (connection_name, created, updated, connection_type, connection_details, owner, owner_id)
VALUES ($1, $2, $3, $4, $5, $6, (SELECT user_id FROM system.users WHERE username = $6))`, systemTableName)
for _, row := range externalConnectionsRows {
if _, err := txn.ExecEx(ctx, systemTableName+"-data-insert",
txn.KV(), sessiondata.NodeUserSessionDataOverride,
restoreQuery, row[0], row[1], row[2], row[3], row[4], row[5],
); err != nil {
return errors.Wrapf(err, "inserting data to system.%s", systemTableName)
}
}

return nil
}

func tableHasColumnName(
ctx context.Context, txn isql.Txn, tableName string, columnName string,
) (bool, error) {
Expand Down Expand Up @@ -570,6 +730,8 @@ var systemTableBackupConfiguration = map[string]systemBackupConfiguration{
systemschema.DatabaseRoleSettingsTable.GetName(): {
shouldIncludeInClusterBackup: optInToClusterBackup, // ID in "database_id".
migrationFunc: rekeySystemTable("database_id"),
customRestoreFunc: systemDatabaseRoleSettingsRestoreFunc,
restoreInOrder: 1, // Restore after system.users.
},
systemschema.TenantUsageTable.GetName(): {
shouldIncludeInClusterBackup: optOutOfClusterBackup,
Expand All @@ -596,9 +758,13 @@ var systemTableBackupConfiguration = map[string]systemBackupConfiguration{
},
systemschema.SystemPrivilegeTable.GetName(): {
shouldIncludeInClusterBackup: optInToClusterBackup, // No desc ID columns.
customRestoreFunc: systemPrivilegesRestoreFunc,
restoreInOrder: 1, // Restore after system.users.
},
systemschema.SystemExternalConnectionsTable.GetName(): {
shouldIncludeInClusterBackup: optInToClusterBackup, // No desc ID columns.
customRestoreFunc: systemExternalConnectionsRestoreFunc,
restoreInOrder: 1, // Restore after system.users.
},
systemschema.RoleIDSequence.GetName(): {
shouldIncludeInClusterBackup: optInToClusterBackup,
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,18 @@
-- The below SQL is used to create the data that is then exported with BACKUP
-- for use in the TestRestoreOldVersions test. This should be run on a v22.2
-- cluster and used to test that after a restore on a v23.1 cluster, the user
-- ID column in the system.database_role_settings table is backfilled.

CREATE DATABASE test;

SET database = test;

CREATE USER testuser1;

CREATE USER testuser2;

ALTER USER testuser1 SET application_name = 'roachdb';

ALTER USER testuser2 SET disallow_full_table_scans = on;

ALTER ROLE ALL SET timezone = 'America/New_York';
Loading

0 comments on commit 1cdcf18

Please sign in to comment.