Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
75660: sql,security: gate session revival behind a cluster setting r=JeffSwenson,otan a=rafiss

fixes #74643

Stop allowing any multitenant cluster from using this functionality.

Release note (security update): The cluster setting
server.user_login.session_revival_token.enabled has been added. It is
false by default. If it is set to true, then a new
token-based authentication mechanism is enabled. A token can be
generated using the crdb_internal.create_session_revival_token builtin
function. The token has a lifetime of 10 minutes and is
cryptographically signed to prevent spoofing and brute-forcing attempts.
When initializing a session later, the token can be presented in a
pgwire StartupMessage with a parameter name of
`crdb:session_revival_token_base64`, with the value encoded in base64.
If this parameter is present, all other authentication checks are
disabled, and if the token is valid and has a valid signature, the user
who originally generated the token authenticates into a new SQL
session. If the token is not valid, then authentication fails.

The token does not have "use-once" semantics, so the same token can be
used any number of times to create multiple new SQL sessions within the
10 minute lifetime of the token. As such, the token should be treated as
highly sensitive cryptographic information.

This feature is meant to be used by multitenant deployments to move a
SQL session from one node to another. It requires the presence of a
valid Ed25519 keypair in tenant-signing.<tenant_id>.crt and
tenant-signing.<tenant_id>.key.

76635: backupccl: ignore all dropped descriptors during backup r=ajwerner,stevendanna a=adityamaru

Previously, descriptors that were resolved at `EndTime`
and were in a `DROP` state were not treated uniformly. While
we ignored table descriptors, we continued to backup database,
schema and type descrpitors.

This resulted in  atleast two bugs:
1) If a database descriptor was in a dropped state, and a new
descriptor with the same name was created then a BACKUP DATABASE
of the new database would fail during resolution.

2) A cluster backup of the above state would succeed, but since it
would include duplicate entries for the same name database, the restore
of such a backup would fail.

This change unifies the behaviour by ignoring all DROP descriptors seen
by the backup at `EndTime`. A follow up PR will teach restore to ignore
all dropped descriptors so as to allow users with "corrupt" backups as
explained in 2) to be able to restore.

Informs: #76517

Release note (bug fix): Backup incorrectly backed up database, schema,
and type descriptors that were in a DROP state at the time the backup was
run. This bug resulted in the user being unable to backup and restore if
their cluster had dropped and public descriptors with colliding names.

76691: ui: Remove "reset time" n Statements and Transactions Pages r=jocrl a=jocrl

Addresses #70997

This commit removes the "reset time" link  on Statements and Transactions Pages.

Release note (ui): The "Now" button had been added in this commit
jocrl@82f2673
to the Statements and Transactions Pages. This commit removes the "reset time"
link which the "Now" button replaces.

Before, Statements Page:
<img width="1356" alt="image" src="https://user-images.githubusercontent.com/91907326/154362540-4268376e-9e4e-4ed2-9f0e-86727ce0ce23.png">

After, Statements Page:
<img width="1264" alt="image" src="https://user-images.githubusercontent.com/91907326/154362398-ac1cd126-a5bc-4268-aad5-823e82327271.png">

Before, Transactions Page:
<img width="1395" alt="image" src="https://user-images.githubusercontent.com/91907326/154362513-c107240f-f8c3-44ad-95d3-ca0f54db8a48.png">

After, Transactions Page:
<img width="1272" alt="image" src="https://user-images.githubusercontent.com/91907326/154362438-33911b3b-cd23-428d-97af-b2fbf69acb94.png">

Co-authored-by: Rafi Shamim <[email protected]>
Co-authored-by: Aditya Maru <[email protected]>
Co-authored-by: Josephine Lee <[email protected]>
  • Loading branch information
4 people committed Feb 17, 2022
4 parents f2a722f + 02a0943 + 451274b + 78c68e7 commit a6ef4b0
Show file tree
Hide file tree
Showing 15 changed files with 271 additions and 47 deletions.
11 changes: 9 additions & 2 deletions pkg/ccl/backupccl/backup_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -335,14 +335,21 @@ func TestBackupRestoreDataDriven(t *testing.T) {
_, err := ds.getSQLDB(t, server, user).Exec(d.Input)
ret := ds.noticeBuffer
if err != nil {
ret = append(ds.noticeBuffer, err.Error())
if pqErr := (*pq.Error)(nil); errors.As(err, &pqErr) {
// pausepoint errors have the job ID in them, and datadriven tests
// don't seem to support regex matching. Clean the error up to not
// include the job ID.
if i := strings.Index(err.Error(), "paused before it completed with reason"); i != -1 {
ret = append(ds.noticeBuffer, err.Error()[i:])
} else if pqErr := (*pq.Error)(nil); errors.As(err, &pqErr) {
ret = append(ds.noticeBuffer, err.Error())
if pqErr.Detail != "" {
ret = append(ret, "DETAIL: "+pqErr.Detail)
}
if pqErr.Hint != "" {
ret = append(ret, "HINT: "+pqErr.Hint)
}
} else {
t.Errorf("failed to execute stmt %s due to %s", d.Input, err.Error())
}
}
return strings.Join(ret, "\n")
Expand Down
14 changes: 14 additions & 0 deletions pkg/ccl/backupccl/backupresolver/targets.go
Original file line number Diff line number Diff line change
Expand Up @@ -182,6 +182,9 @@ func NewDescriptorResolver(descs []catalog.Descriptor) (*DescriptorResolver, err
// check the ParentID for tables, and all the valid parents must be
// known before we start to check that.
for _, desc := range descs {
if desc.Dropped() {
continue
}
if _, isDB := desc.(catalog.DatabaseDescriptor); isDB {
if _, ok := r.DbsByName[desc.GetName()]; ok {
return nil, errors.Errorf("duplicate database name: %q used for ID %d and %d",
Expand All @@ -207,6 +210,9 @@ func NewDescriptorResolver(descs []catalog.Descriptor) (*DescriptorResolver, err

// Add all schemas to the resolver.
for _, desc := range descs {
if desc.Dropped() {
continue
}
if sc, ok := desc.(catalog.SchemaDescriptor); ok {
schemaMap := r.ObjsByName[sc.GetParentID()]
if schemaMap == nil {
Expand Down Expand Up @@ -332,6 +338,14 @@ func DescriptorsMatchingTargets(
}
if _, ok := alreadyRequestedDBs[dbID]; !ok {
desc := r.DescByID[dbID]
// Verify that the database is in the correct state.
doesNotExistErr := errors.Errorf(`database %q does not exist`, d)
if err := catalog.FilterDescriptorState(
desc, tree.CommonLookupFlags{},
); err != nil {
// Return a does not exist error if explicitly asking for this database.
return ret, doesNotExistErr
}
ret.Descs = append(ret.Descs, desc)
ret.RequestedDBs = append(ret.RequestedDBs,
desc.(catalog.DatabaseDescriptor))
Expand Down
15 changes: 9 additions & 6 deletions pkg/ccl/backupccl/targets.go
Original file line number Diff line number Diff line change
Expand Up @@ -229,8 +229,9 @@ func getAllDescChanges(
return res, nil
}

// fullClusterTargets returns all of the tableDescriptors to be included in a
// full cluster backup, and all the user databases.
// fullClusterTargets returns all of the descriptors to be included in a full
// cluster backup, along with all the "complete databases" that we are backing
// up.
func fullClusterTargets(
allDescs []catalog.Descriptor,
) ([]catalog.Descriptor, []catalog.DatabaseDescriptor, error) {
Expand All @@ -240,6 +241,11 @@ func fullClusterTargets(
systemTablesToBackup := GetSystemTablesToIncludeInClusterBackup()

for _, desc := range allDescs {
// If a descriptor is in the DROP state at `EndTime` we do not want to
// include it in the backup.
if desc.Dropped() {
continue
}
switch desc := desc.(type) {
case catalog.DatabaseDescriptor:
dbDesc := dbdesc.NewBuilder(desc.DatabaseDesc()).BuildImmutableDatabase()
Expand All @@ -256,10 +262,7 @@ func fullClusterTargets(
fullClusterDescs = append(fullClusterDescs, desc)
}
} else {
// Add all user tables that are not in a DROP state.
if !desc.Dropped() {
fullClusterDescs = append(fullClusterDescs, desc)
}
fullClusterDescs = append(fullClusterDescs, desc)
}
case catalog.SchemaDescriptor:
fullClusterDescs = append(fullClusterDescs, desc)
Expand Down
200 changes: 200 additions & 0 deletions pkg/ccl/backupccl/testdata/backup-restore/backup-dropped-descriptors
Original file line number Diff line number Diff line change
@@ -0,0 +1,200 @@
# backup-dropped-desctiprors tests backup and restore interaction with database, schema
# and type descriptors in the DROP state.
subtest dropped-database-descriptors

new-server name=s1
----

exec-sql
SET CLUSTER SETTING jobs.debug.pausepoints = 'schemachanger.before.exec';
CREATE DATABASE d;
CREATE TABLE d.foo (id INT);
DROP DATABASE d CASCADE;
----
paused before it completed with reason: pause point "schemachanger.before.exec" hit

# At this point, we have a descriptor entry for `d` in a DROP state.
query-sql
WITH tbls AS (
SELECT id, crdb_internal.pb_to_json('cockroach.sql.sqlbase.Descriptor', descriptor) AS orig FROM system.descriptor
)
SELECT orig->'database'->'name', orig->'database'->'state' FROM tbls WHERE id = 107;
----
"d" "DROP"

# A database backup should fail since we are explicitly targeting a dropped
# object.
exec-sql
BACKUP DATABASE d INTO 'nodelocal://0/dropped-database';
----
pq: failed to resolve targets specified in the BACKUP stmt: database "d" does not exist, or invalid RESTORE timestamp: supplied backups do not cover requested time

# A cluster backup should succeed but should ignore the dropped database
# and table descriptors.
exec-sql
BACKUP INTO 'nodelocal://0/cluster/dropped-database';
----

query-sql
SELECT count(*) FROM [SHOW BACKUP LATEST IN 'nodelocal://0/cluster/dropped-database'] WHERE object_name = 'd' OR object_name = 'foo';
----
0

# Now create another descriptor entry with the same name in a PUBLIC state.
exec-sql
CREATE DATABASE d;
CREATE TABLE d.bar (id INT);
----

# A database backup should succeed since we have a public database descriptor that matches the
# target.
exec-sql
BACKUP DATABASE d INTO 'nodelocal://0/dropped-database';
----

# A cluster backup should succeed and include the public database descriptor and
# its table.
exec-sql
BACKUP INTO 'nodelocal://0/cluster/dropped-database';
----

# Restore from the database backup to ensure it is valid.
# Sanity check that we did not backup the table 'foo' that belonged to the
# dropped database 'd'.
exec-sql
RESTORE DATABASE d FROM LATEST IN 'nodelocal://0/dropped-database' WITH new_db_name = 'd1';
USE d1;
----

query-sql
SELECT schema_name,table_name FROM [SHOW TABLES];
----
public bar

# Restore from the cluster backup to ensure it is valid.
# Sanity check that we did not backup the table 'foo' that belonged to the
# dropped database 'd'.
exec-sql
RESTORE DATABASE d FROM LATEST IN 'nodelocal://0/cluster/dropped-database' WITH new_db_name = 'd2';
USE d2;
----

query-sql
SELECT schema_name,table_name FROM [SHOW TABLES];
----
public bar

subtest end

# Test backup/restore interaction with dropped schema and type in a database.
subtest dropped-schema-descriptors

new-server name=s2
----

exec-sql
CREATE DATABASE d2;
CREATE TABLE d2.t2 (id INT);
----

exec-sql
CREATE TYPE d2.typ AS ENUM ('hello');
CREATE SCHEMA d2.s;
CREATE TABLE d2.s.t (id INT);
SET CLUSTER SETTING jobs.debug.pausepoints = 'schemachanger.before.exec';
DROP SCHEMA d2.s CASCADE;
----
paused before it completed with reason: pause point "schemachanger.before.exec" hit

exec-sql
SET CLUSTER SETTING jobs.debug.pausepoints = 'typeschemachanger.before.exec';
DROP TYPE d2.typ;
----
paused before it completed with reason: pause point "typeschemachanger.before.exec" hit

query-sql
WITH tbls AS (
SELECT id, crdb_internal.pb_to_json('cockroach.sql.sqlbase.Descriptor', descriptor) AS orig FROM system.descriptor
)
SELECT orig->'schema'->'name', orig->'schema'->'state' FROM tbls WHERE id = 112;
----
"s" "DROP"


query-sql
WITH tbls AS (
SELECT id, crdb_internal.pb_to_json('cockroach.sql.sqlbase.Descriptor', descriptor) AS orig FROM system.descriptor
)
SELECT orig->'type'->'name', orig->'type'->'state' FROM tbls WHERE id = 110 OR id = 111;
----
"typ" "DROP"
"_typ" "DROP"

# A database backup should succeed but should not include the dropped schema,
# type, and table.
exec-sql
BACKUP DATABASE d2 INTO 'nodelocal://0/dropped-schema-in-database';
----

query-sql
SELECT count(*) FROM [SHOW BACKUP LATEST IN 'nodelocal://0/dropped-schema-in-database'] WHERE
object_name = 's' OR object_name = 'typ';
----
0


# A cluster backup should succeed but should not include the dropped schema,
# type, and table.
exec-sql
BACKUP INTO 'nodelocal://0/cluster/dropped-schema-in-database';
----

query-sql
SELECT count(*) FROM [SHOW BACKUP LATEST IN 'nodelocal://0/cluster/dropped-schema-in-database']
WHERE object_name = 's' OR object_name = 'typ';
----
0

# Restore the backups to check they are valid.
exec-sql
RESTORE DATABASE d2 FROM LATEST IN 'nodelocal://0/dropped-schema-in-database' WITH new_db_name = 'd3';
USE d3;
----

# We don't expect to see the dropped schema 's'.
query-sql
SELECT schema_name FROM [SHOW SCHEMAS];
----
public
crdb_internal
information_schema
pg_catalog
pg_extension


query-sql
SELECT schema_name, table_name FROM [SHOW TABLES];
----
public t2


exec-sql
RESTORE DATABASE d2 FROM LATEST IN 'nodelocal://0/cluster/dropped-schema-in-database' WITH new_db_name ='d4';
USE d4;
----

query-sql
SELECT schema_name FROM [SHOW SCHEMAS];
----
public
crdb_internal
information_schema
pg_catalog
pg_extension

query-sql
SELECT schema_name, table_name FROM [SHOW TABLES];
----
public t2

subtest end
11 changes: 7 additions & 4 deletions pkg/ccl/testccl/sqlccl/session_revival_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -34,13 +34,16 @@ func TestAuthenticateWithSessionRevivalToken(t *testing.T) {
ctx := context.Background()

params, _ := tests.CreateTestServerParams()
s, _, _ := serverutils.StartServer(t, params)
s, mainDB, _ := serverutils.StartServer(t, params)
defer s.Stopper().Stop(ctx)
tenant, mainDB := serverutils.StartTenant(t, s, tests.CreateTestTenantParams(serverutils.TestTenantID()))
defer tenant.Stopper().Stop(ctx)
defer mainDB.Close()
tenant, tenantDB := serverutils.StartTenant(t, s, tests.CreateTestTenantParams(serverutils.TestTenantID()))
defer tenant.Stopper().Stop(ctx)
defer tenantDB.Close()

_, err := mainDB.Exec("CREATE USER testuser WITH PASSWORD 'hunter2'")
_, err := tenantDB.Exec("CREATE USER testuser WITH PASSWORD 'hunter2'")
require.NoError(t, err)
_, err = tenantDB.Exec("SET CLUSTER SETTING server.user_login.session_revival_token.enabled = true")
require.NoError(t, err)

var token string
Expand Down
5 changes: 0 additions & 5 deletions pkg/server/server_sql.go
Original file line number Diff line number Diff line change
Expand Up @@ -306,10 +306,6 @@ type sqlServerArgs struct {
// monitorAndMetrics contains the return value of newRootSQLMemoryMonitor.
monitorAndMetrics monitorAndMetrics

// allowSessionRevival is true if the cluster is allowed to create session
// revival tokens and use them to authenticate a session.
allowSessionRevival bool

// settingsStorage is an optional interface to drive storing of settings
// data on disk to provide a fresh source of settings upon next startup.
settingsStorage settingswatcher.Storage
Expand Down Expand Up @@ -695,7 +691,6 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) {
CompactEngineSpanFunc: compactEngineSpanFunc,
TraceCollector: traceCollector,
TenantUsageServer: cfg.tenantUsageServer,
AllowSessionRevival: cfg.allowSessionRevival,

DistSQLPlanner: sql.NewDistSQLPlanner(
ctx,
Expand Down
1 change: 0 additions & 1 deletion pkg/server/tenant.go
Original file line number Diff line number Diff line change
Expand Up @@ -567,7 +567,6 @@ func makeTenantSQLServerArgs(
rangeFeedFactory: rangeFeedFactory,
regionsServer: tenantConnect,
costController: costController,
allowSessionRevival: true,
grpc: grpcServer,
}, nil
}
Expand Down
4 changes: 0 additions & 4 deletions pkg/sql/exec_util.go
Original file line number Diff line number Diff line change
Expand Up @@ -1273,10 +1273,6 @@ type ExecutorConfig struct {
// SessionData and other ExtraTxnState.
// This is currently only for builtin functions where we need to execute sql.
InternalExecutorFactory sqlutil.SessionBoundInternalExecutorFactory

// AllowSessionRevival is true if the cluster is allowed to create session
// revival tokens and use them to authenticate a session.
AllowSessionRevival bool
}

// UpdateVersionSystemSettingHook provides a callback that allows us
Expand Down
14 changes: 10 additions & 4 deletions pkg/sql/logictest/testdata/logic_test/builtin_function
Original file line number Diff line number Diff line change
Expand Up @@ -3136,6 +3136,11 @@ SELECT hmac('dog', 'key', 'made up alg')

subtest session_revival_token

statement ok
SET CLUSTER SETTING server.user_login.session_revival_token.enabled = true;
CREATE USER parentuser;
GRANT parentuser TO testuser

skipif config 3node-tenant
statement error session revival tokens are not supported on this cluster
select crdb_internal.create_session_revival_token()
Expand All @@ -3144,10 +3149,6 @@ onlyif config 3node-tenant
statement error cannot create token for root user
select crdb_internal.create_session_revival_token()

statement ok
CREATE USER parentuser;
GRANT parentuser TO testuser

user testuser

statement ok
Expand Down Expand Up @@ -3194,3 +3195,8 @@ FROM
c, a
----
Ed25519 testuser true true true true

user root

statement ok
SET CLUSTER SETTING server.user_login.session_revival_token.enabled = false
2 changes: 1 addition & 1 deletion pkg/sql/pgwire/auth_methods.go
Original file line number Diff line number Diff line change
Expand Up @@ -609,7 +609,7 @@ func authSessionRevivalToken(token []byte) AuthMethod {
b.SetRoleMapper(UseProvidedIdentity)
b.SetAuthenticator(func(ctx context.Context, user security.SQLUsername, _ bool, _ PasswordRetrievalFn) error {
c.LogAuthInfof(ctx, "session revival token detected; attempting to use it")
if !execCfg.AllowSessionRevival {
if !sql.AllowSessionRevival.Get(&execCfg.Settings.SV) || execCfg.Codec.ForSystemTenant() {
return errors.New("session revival tokens are not supported on this cluster")
}
cm, err := execCfg.RPCContext.SecurityContext.GetCertificateManager()
Expand Down
Loading

0 comments on commit a6ef4b0

Please sign in to comment.