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 3, 2024
1 parent 1ea95bb commit f227b7f
Show file tree
Hide file tree
Showing 5 changed files with 99 additions and 16 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
11 changes: 1 addition & 10 deletions pkg/sql/grant_revoke.go
Original file line number Diff line number Diff line change
Expand Up @@ -393,15 +393,6 @@ func (n *changeDescriptorBackedPrivilegesNode) startExec(params runParams) error
}

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 +410,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 Down
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
61 changes: 61 additions & 0 deletions pkg/sql/schema_changer_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -7821,3 +7821,64 @@ func TestMemoryMonitorErrorsDuringBackfillAreRetried(t *testing.T) {
require.Equalf(t, shouldFail.Load(), int64(2), "not all failure conditions were hit %d", shouldFail.Load())
})
}

// TestGrantAndSelectsWithinTransaction tests two concurrent
// transactions on tables, where one transaction is a grant.
// We verify that grants on tables no longer creates a job(s).
// We also verify that the second transaction does wait for
// the first transaction to commit.
func TestGrantAndSelectsWithinTransaction(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
ctx := context.Background()

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

connection1 := sqlutils.MakeSQLRunner(sqlDB)
connection2 := sqlutils.MakeSQLRunner(sqlDB)

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

transaction1 := connection1.Begin(t)
transaction1.Exec(`SELECT * FROM PROMO_CODES;`)

messages := make(chan struct{})

group := ctxgroup.WithContext(ctx)
group.Go(func() error {
// wait for channel before starting the sleep
<-messages
// we sleep on transaction1 which should hold transaction2
time.Sleep(2 * time.Second)
err := transaction1.Commit()
return err
})

resBefore := connection2.QueryStr(t, `select count(*) from [show jobs];`)

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

// close channel to start sleep timer in go routine
close(messages)
startTime := timeutil.Now()

err := transaction2.Commit()
duration := timeutil.Since(startTime)

err = group.Wait()
// we verify that transaction2 did hold, it should hold for up to
// 2 seconds, so we check that the duration is at least 1.5 seconds
require.True(t, duration > 1500*time.Millisecond)
require.NoError(t, err)

resAfter := connection2.QueryStr(t, `select count(*) from [show jobs];`)
// no new jobs should have been created between resBefore to resAfter
require.True(t, resBefore[0][0] == resAfter[0][0])

}
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 f227b7f

Please sign in to comment.