Skip to content

Commit

Permalink
sql: grant/revoke on a large number of objects can create a lot of jobs
Browse files Browse the repository at this point in the history
Previously we would create multiple jobs for granting on multiple
tables and types within one transaction.  This caused a performance
slowdown, these code changes skip the making of multiple jobs and
just execute the grants in a batch.

Fixes: #117643
Release note (performance improvement): Multiple/large amounts of grants on
tables and types within one transaction now run faster.
  • Loading branch information
Dedej-Bergin committed May 9, 2024
1 parent fc8482f commit 2f3e936
Show file tree
Hide file tree
Showing 8 changed files with 220 additions and 23 deletions.
12 changes: 6 additions & 6 deletions pkg/bench/rttanalysis/testdata/benchmark_expectations
Original file line number Diff line number Diff line change
Expand Up @@ -56,9 +56,9 @@ exp,benchmark
5,GenerateObjects/generate_10_tables
16,GenerateObjects/generate_10x10_schemas_and_tables_in_existing_db
5,GenerateObjects/generate_50000_tables
15,Grant/grant_all_on_1_table
19,Grant/grant_all_on_2_tables
23,Grant/grant_all_on_3_tables
10,Grant/grant_all_on_1_table
10,Grant/grant_all_on_2_tables
10,Grant/grant_all_on_3_tables
19,GrantRole/grant_1_role
25,GrantRole/grant_2_roles
6,Jobs/cancel_job
Expand Down Expand Up @@ -104,9 +104,9 @@ exp,benchmark
4,ORMQueries/pg_type
133,ORMQueries/prisma_column_descriptions
3,ORMQueries/prisma_column_descriptions_updated
15,Revoke/revoke_all_on_1_table
19,Revoke/revoke_all_on_2_tables
23,Revoke/revoke_all_on_3_tables
10,Revoke/revoke_all_on_1_table
10,Revoke/revoke_all_on_2_tables
10,Revoke/revoke_all_on_3_tables
17,RevokeRole/revoke_1_role
21,RevokeRole/revoke_2_roles
14,ShowGrants/grant_2_roles
Expand Down
22 changes: 5 additions & 17 deletions pkg/sql/grant_revoke.go
Original file line number Diff line number Diff line change
Expand Up @@ -391,17 +391,7 @@ func (n *changeDescriptorBackedPrivilegesNode) startExec(params runParams) error
DatabaseName: (*tree.Name)(&d.Name).String(),
})
}

case *tabledesc.Mutable:
// TODO (lucy): This should probably have a single consolidated job like
// DROP DATABASE.
if err := p.createOrUpdateSchemaChangeJob(
ctx, d,
fmt.Sprintf("updating privileges for table %d", d.ID),
descpb.InvalidMutationID,
); err != nil {
return err
}
if !d.Dropped() {
if err := p.writeSchemaChangeToBatch(ctx, d, b); err != nil {
return err
Expand All @@ -419,7 +409,7 @@ func (n *changeDescriptorBackedPrivilegesNode) startExec(params runParams) error
})
}
case *typedesc.Mutable:
err := p.writeTypeSchemaChange(ctx, d, fmt.Sprintf("updating privileges for type %d", d.ID))
err := p.writeDescToBatch(ctx, d, b)
if err != nil {
return err
}
Expand All @@ -435,11 +425,8 @@ func (n *changeDescriptorBackedPrivilegesNode) startExec(params runParams) error
})
}
case *schemadesc.Mutable:
if err := p.writeSchemaDescChange(
ctx,
d,
fmt.Sprintf("updating privileges for schema %d", d.ID),
); err != nil {
err := p.writeDescToBatch(ctx, d, b)
if err != nil {
return err
}
for _, grantee := range n.grantees {
Expand All @@ -454,7 +441,8 @@ func (n *changeDescriptorBackedPrivilegesNode) startExec(params runParams) error
})
}
case *funcdesc.Mutable:
if err := p.writeFuncSchemaChange(ctx, d); err != nil {
err := p.writeDescToBatch(ctx, d, b)
if err != nil {
return err
}
for _, grantee := range n.grantees {
Expand Down
80 changes: 80 additions & 0 deletions pkg/sql/grant_revoke_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@ import (
"fmt"
"testing"

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/security/username"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
Expand Down Expand Up @@ -239,3 +240,82 @@ func TestNoOpRevoke(t *testing.T) {
}
}
}

func BenchmarkGrantTables(b *testing.B) {
defer leaktest.AfterTest(b)()
defer log.Scope(b).Close(b)
ctx := context.Background()

for _, numTables := range []int{10, 100, 1000} {
b.Run(fmt.Sprintf("numTables=%d", numTables), func(b *testing.B) {
srv, sqlDB, _ := serverutils.StartServer(b, base.TestServerArgs{})
defer srv.Stopper().Stop(ctx)

sqlRun := sqlutils.MakeSQLRunner(sqlDB)
sqlRun.Exec(b, `CREATE DATABASE t;`)
sqlRun.Exec(b, `USE t;`)

sqlRun.Exec(b, `CREATE USER ROACH;`)

for i := 0; i < numTables; i++ {
sqlRun.Exec(b, fmt.Sprintf(`CREATE TABLE t.a%d (k INT PRIMARY KEY);`, i))
}

b.ResetTimer()
for i := 0; i < b.N; i++ {
sqlRun.Exec(b, `GRANT ALL ON * TO ROACH;`)
sqlRun.Exec(b, `REVOKE ALL ON * FROM ROACH;`)
}
})
}
}

func BenchmarkGrantTypes(b *testing.B) {
defer leaktest.AfterTest(b)()
defer log.Scope(b).Close(b)
ctx := context.Background()

for _, numTypes := range []int{10, 100, 1000} {
b.Run(fmt.Sprintf("numTypes=%d", numTypes), func(b *testing.B) {
srv, sqlDB, _ := serverutils.StartServer(b, base.TestServerArgs{})
defer srv.Stopper().Stop(ctx)

sqlRun := sqlutils.MakeSQLRunner(sqlDB)
sqlRun.Exec(b, `CREATE DATABASE t;`)
sqlRun.Exec(b, `USE t;`)

sqlRun.Exec(b, `CREATE USER ROACH;`)

for i := 0; i < numTypes; i++ {
sqlRun.Exec(b, fmt.Sprintf(`CREATE TYPE a%d AS ENUM ('roach1', 'roach2', 'roach3');`, i))
}

b.ResetTimer()
for i := 0; i < b.N; i++ {
txn := sqlRun.Begin(b)
for i := 0; i < numTypes; i++ {
_, err := txn.Exec(fmt.Sprintf(`GRANT ALL ON TYPE a%d TO ROACH;`, i))
if err != nil {
return
}
}
err := txn.Commit()
if err != nil {
return
}

txn = sqlRun.Begin(b)
for i := 0; i < numTypes; i++ {
_, err = txn.Exec(fmt.Sprintf(`REVOKE ALL ON TYPE a%d FROM ROACH;`, i))
if err != nil {
return
}
}
err = txn.Commit()
if err != nil {
return
}
}
})
}
}
23 changes: 23 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/grant_schema
Original file line number Diff line number Diff line change
Expand Up @@ -197,3 +197,26 @@ test sc102962 root ALL true

statement error pgcode 2BP01 pq: cannot drop role/user r102962: grants still exist on test.sc102962
DROP ROLE r102962

statement ok
CREATE USER ROACH;
CREATE SCHEMA ROACHHOUSE;
CREATE SCHEMA ROACHLAYER;
CREATE SCHEMA ROACHSTATION;
BEGIN;
GRANT ALL ON SCHEMA ROACHHOUSE TO ROACH;
GRANT ALL ON SCHEMA ROACHLAYER TO ROACH;
GRANT ALL ON SCHEMA ROACHSTATION TO ROACH;
COMMIT;

query T
SELECT description FROM [SHOW JOBS] WHERE description LIKE '%updating privileges for schema%';
----

query TTTTTTB colnames,rowsort
SHOW GRANTS FOR roach
----
database_name schema_name object_name object_type grantee privilege_type is_grantable
test roachhouse NULL schema roach ALL false
test roachlayer NULL schema roach ALL false
test roachstation NULL schema roach ALL false
20 changes: 20 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/grant_table
Original file line number Diff line number Diff line change
Expand Up @@ -2973,3 +2973,23 @@ GRANT SELECT ON TABLE t, crdb_internal.tables TO testuser;

statement error pgcode 0A000 cannot mix grants between virtual and non-virtual tables
GRANT SELECT ON TABLE crdb_internal.tables, t TO testuser;

statement ok
CREATE USER ROACH;
CREATE TABLE table1 (count INT);
CREATE TABLE table2 (count INT);
BEGIN;
GRANT ALL ON table1 TO ROACH;
GRANT ALL ON table2 TO ROACH;
COMMIT;

query T rowsort
select description from [show jobs] where description LIKE '%updating privileges for table%';
----

query TTTTTTB colnames,rowsort
SHOW GRANTS FOR roach
----
database_name schema_name object_name object_type grantee privilege_type is_grantable
db public table1 table roach ALL false
db public table2 table roach ALL false
23 changes: 23 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/grant_type
Original file line number Diff line number Diff line change
Expand Up @@ -126,3 +126,26 @@ test public owner_grant_option other_owner ALL
test public owner_grant_option owner_grant_option_child USAGE false
test public owner_grant_option public USAGE false
test public owner_grant_option root ALL true

statement ok
CREATE USER roach;
CREATE TYPE custom_type1 AS ENUM ('roach1', 'roach2', 'roach3');
CREATE TYPE custom_type2 AS ENUM ('roachA', 'roachB', 'roachC');
CREATE TYPE custom_type3 AS ENUM ('roachI', 'roachII', 'roachIII');
BEGIN;
GRANT ALL ON TYPE custom_type1 TO roach;
GRANT ALL ON TYPE custom_type2 TO roach;
GRANT ALL ON TYPE custom_type3 TO roach;
COMMIT

query T
select description from [show jobs] where description LIKE '%type%'
----

query TTTTTTB colnames,rowsort
SHOW GRANTS FOR roach
----
database_name schema_name object_name object_type grantee privilege_type is_grantable
test public custom_type1 type roach ALL false
test public custom_type2 type roach ALL false
test public custom_type3 type roach ALL false
55 changes: 55 additions & 0 deletions pkg/sql/schema_changer_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -7821,3 +7821,58 @@ func TestMemoryMonitorErrorsDuringBackfillAreRetried(t *testing.T) {
require.Equalf(t, shouldFail.Load(), int64(2), "not all failure conditions were hit %d", shouldFail.Load())
})
}

// TestLeaseTimeoutWithConcurrentTransactions tests two concurrent transactions
// on tables, we verify that the second transaction waits for the first
// transaction to commit or for the lease to expire.
func TestLeaseTimeoutWithConcurrentTransactions(t *testing.T) {
skip.UnderDuress(t, "slow test")
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
ctx := context.Background()

settings := cluster.MakeTestingClusterSettings()
lease.LeaseDuration.Override(ctx, &settings.SV, 15*time.Second)

s, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{Settings: settings})
defer s.Stopper().Stop(ctx)

sqlRunner := sqlutils.MakeSQLRunner(sqlDB)

sqlRunner.Exec(t, `CREATE USER ROACHMIN;`)
sqlRunner.Exec(t, `GRANT ADMIN TO ROACHMIN;`)
sqlRunner.Exec(t, `CREATE TABLE PROMO_CODES (my_int INT);`)
sqlRunner.Exec(t, `CREATE TABLE RIDES (my_int INT);`)

txn1 := sqlRunner.Begin(t)
txn1.Exec(`SELECT * FROM PROMO_CODES;`)

txn2 := sqlRunner.Begin(t)
txn2.Exec(`GRANT ALL ON TABLE PROMO_CODES TO ROACHMIN;`)
txn2.Exec(`GRANT ALL ON TABLE RIDES TO ROACHMIN;`)

var duration time.Duration
blocker := make(chan struct{})
group := ctxgroup.WithContext(ctx)

group.GoCtx(func(ctx context.Context) error {
startTime := timeutil.Now()
err := txn2.Commit()
duration = timeutil.Since(startTime)
close(blocker)
return err
})

<-blocker
_, err := txn1.Exec("INSERT INTO promo_codes values (1)")
require.NoError(t, err)

// txn1.commit() completes with an error due to lease timeout on txn2.commit().
err = txn1.Commit()
require.ErrorContains(t, err, "RETRY_COMMIT_DEADLINE_EXCEEDED")

err = group.Wait()
require.NoError(t, err)

require.Greaterf(t, duration, 15*time.Second, "transaction2 did not hold for at least 15 seconds")
}
8 changes: 8 additions & 0 deletions pkg/sql/type_change.go
Original file line number Diff line number Diff line change
Expand Up @@ -165,6 +165,14 @@ func (p *planner) writeTypeDesc(ctx context.Context, typeDesc *typedesc.Mutable)
return p.txn.Run(ctx, b)
}

func (p *planner) writeDescToBatch(
ctx context.Context, typeDesc catalog.MutableDescriptor, b *kv.Batch,
) error {
return p.Descriptors().WriteDescToBatch(
ctx, p.extendedEvalCtx.Tracing.KVTracingEnabled(), typeDesc, b,
)
}

// typeSchemaChanger is the struct that actually runs the type schema change.
type typeSchemaChanger struct {
typeID descpb.ID
Expand Down

0 comments on commit 2f3e936

Please sign in to comment.