Skip to content

Commit

Permalink
Merge pull request #112110 from cockroachdb/blathers/backport-release…
Browse files Browse the repository at this point in the history
…-23.2-111934

release-23.2: sql: cache role existence checks to fix perf regression
  • Loading branch information
rafiss authored Oct 10, 2023
2 parents 77f062e + af2fa1a commit bcbe369
Show file tree
Hide file tree
Showing 10 changed files with 63 additions and 34 deletions.
52 changes: 26 additions & 26 deletions pkg/bench/rttanalysis/testdata/benchmark_expectations
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,7 @@ exp,benchmark
8,AlterTableUnsplit/alter_table_unsplit_at_1_value
10,AlterTableUnsplit/alter_table_unsplit_at_2_values
12,AlterTableUnsplit/alter_table_unsplit_at_3_values
7,Audit/select_from_an_audit_table
6,Audit/select_from_an_audit_table
20,CreateRole/create_role_with_1_option
23,CreateRole/create_role_with_2_options
26,CreateRole/create_role_with_3_options
Expand All @@ -41,7 +41,7 @@ exp,benchmark
18,DropDatabase/drop_database_3_tables
27,DropRole/drop_1_role
36,DropRole/drop_2_roles
42-45,DropRole/drop_3_roles
45,DropRole/drop_3_roles
15,DropSequence/drop_1_sequence
17,DropSequence/drop_2_sequences
19,DropSequence/drop_3_sequences
Expand All @@ -61,44 +61,44 @@ exp,benchmark
21,Grant/grant_all_on_3_tables
16,GrantRole/grant_1_role
21,GrantRole/grant_2_roles
5,ORMQueries/activerecord_type_introspection_query
7,ORMQueries/column_descriptions_json_agg
11,ORMQueries/django_column_introspection_1_table
11,ORMQueries/django_column_introspection_4_tables
11,ORMQueries/django_column_introspection_8_tables
13,ORMQueries/django_comment_introspection_with_comments
11,ORMQueries/django_table_introspection_1_table
11,ORMQueries/django_table_introspection_8_tables
4,ORMQueries/activerecord_type_introspection_query
1,ORMQueries/column_descriptions_json_agg
5,ORMQueries/django_column_introspection_1_table
5,ORMQueries/django_column_introspection_4_tables
5,ORMQueries/django_column_introspection_8_tables
7,ORMQueries/django_comment_introspection_with_comments
7,ORMQueries/django_table_introspection_1_table
7,ORMQueries/django_table_introspection_8_tables
0,ORMQueries/has_column_privilege_using_attnum
0,ORMQueries/has_column_privilege_using_column_name
0,ORMQueries/has_schema_privilege
0,ORMQueries/has_sequence_privilege
0,ORMQueries/has_table_privilege
9,ORMQueries/hasura_column_descriptions
16,ORMQueries/hasura_column_descriptions_8_tables
9,ORMQueries/hasura_column_descriptions_modified
6-7,ORMQueries/information_schema._pg_index_position
7,ORMQueries/introspection_description_join
9,ORMQueries/npgsql_fields
13,ORMQueries/npgsql_types
6,ORMQueries/hasura_column_descriptions
13,ORMQueries/hasura_column_descriptions_8_tables
6,ORMQueries/hasura_column_descriptions_modified
5,ORMQueries/information_schema._pg_index_position
5,ORMQueries/introspection_description_join
6,ORMQueries/npgsql_fields
6,ORMQueries/npgsql_types
5,ORMQueries/pg_attribute
5,ORMQueries/pg_class
9,ORMQueries/pg_is_other_temp_schema
17,ORMQueries/pg_is_other_temp_schema_multiple_times
7,ORMQueries/pg_is_other_temp_schema
7,ORMQueries/pg_is_other_temp_schema_multiple_times
5,ORMQueries/pg_my_temp_schema
5,ORMQueries/pg_my_temp_schema_multiple_times
4,ORMQueries/pg_namespace
5,ORMQueries/pg_type
129,ORMQueries/prisma_column_descriptions
9,ORMQueries/prisma_column_descriptions_updated
132,ORMQueries/prisma_column_descriptions
4,ORMQueries/prisma_column_descriptions_updated
13,Revoke/revoke_all_on_1_table
17,Revoke/revoke_all_on_2_tables
21,Revoke/revoke_all_on_3_tables
15,RevokeRole/revoke_1_role
18,RevokeRole/revoke_2_roles
13,ShowGrants/grant_2_roles
14,ShowGrants/grant_3_roles
15,ShowGrants/grant_4_roles
12,ShowGrants/grant_2_roles
13,ShowGrants/grant_3_roles
14,ShowGrants/grant_4_roles
1,SystemDatabaseQueries/select_system.users_with_empty_database_Name
1,SystemDatabaseQueries/select_system.users_with_schema_Name
1,SystemDatabaseQueries/select_system.users_without_schema_Name
Expand All @@ -111,5 +111,5 @@ exp,benchmark
4,UDFResolution/select_from_udf
2,VirtualTableQueries/select_crdb_internal.invalid_objects_with_1_fk
2,VirtualTableQueries/select_crdb_internal.tables_with_1_fk
14,VirtualTableQueries/virtual_table_cache_with_point_lookups
20,VirtualTableQueries/virtual_table_cache_with_schema_change
10,VirtualTableQueries/virtual_table_cache_with_point_lookups
16,VirtualTableQueries/virtual_table_cache_with_schema_change
Original file line number Diff line number Diff line change
Expand Up @@ -11,9 +11,9 @@ exp,benchmark
14,AlterSurvivalGoals/alter_empty_database_from_zone_to_region
24,AlterSurvivalGoals/alter_populated_database_from_region_to_zone
24,AlterSurvivalGoals/alter_populated_database_from_zone_to_region
27,AlterTableLocality/alter_from_global_to_rbr
25,AlterTableLocality/alter_from_global_to_rbr
16,AlterTableLocality/alter_from_global_to_regional_by_table
13,AlterTableLocality/alter_from_rbr_to_global
13,AlterTableLocality/alter_from_rbr_to_regional_by_table
16,AlterTableLocality/alter_from_regional_by_table_to_global
27,AlterTableLocality/alter_from_regional_by_table_to_rbr
25,AlterTableLocality/alter_from_regional_by_table_to_rbr
2 changes: 1 addition & 1 deletion pkg/sql/authorization.go
Original file line number Diff line number Diff line change
Expand Up @@ -907,7 +907,7 @@ func (p *planner) checkCanAlterToNewOwner(
ctx context.Context, desc catalog.MutableDescriptor, newOwner username.SQLUsername,
) error {
// Make sure the newOwner exists.
roleExists, err := RoleExists(ctx, p.InternalSQLTxn(), newOwner)
roleExists, err := p.RoleExists(ctx, newOwner)
if err != nil {
return err
}
Expand Down
7 changes: 7 additions & 0 deletions pkg/sql/conn_executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -1562,6 +1562,11 @@ type connExecutor struct {
// in a transaction.
hasAdminRoleCache HasAdminRoleCache

// roleExistsCache is a cache of role existence checks. This is used because
// role existence checks are made when checking privileges. Only positive
// values are cached.
roleExistsCache map[username.SQLUsername]struct{}

// createdSequences keeps track of sequences created in the current transaction.
// The map key is the sequence descpb.ID.
createdSequences map[descpb.ID]struct{}
Expand Down Expand Up @@ -1973,6 +1978,7 @@ func (ex *connExecutor) resetExtraTxnState(ctx context.Context, ev txnEvent, pay
ex.extraTxnState.numDDL = 0
ex.extraTxnState.firstStmtExecuted = false
ex.extraTxnState.hasAdminRoleCache = HasAdminRoleCache{}
ex.extraTxnState.roleExistsCache = make(map[username.SQLUsername]struct{})
ex.extraTxnState.createdSequences = nil

if ex.extraTxnState.fromOuterTxn {
Expand Down Expand Up @@ -3619,6 +3625,7 @@ func (ex *connExecutor) resetEvalCtx(evalCtx *extendedEvalContext, txn *kv.Txn,
evalCtx.SkipNormalize = false
evalCtx.SchemaChangerState = ex.extraTxnState.schemaChangerState
evalCtx.DescIDGenerator = ex.getDescIDGenerator()
evalCtx.RoleExistsCache = ex.extraTxnState.roleExistsCache

// See resetPlanner for more context on setting the maximum timestamp for
// AOST read retries.
Expand Down
3 changes: 3 additions & 0 deletions pkg/sql/internal.go
Original file line number Diff line number Diff line change
Expand Up @@ -351,6 +351,7 @@ func (ie *InternalExecutor) newConnExecutorWithTxn(
ex.extraTxnState.jobs = ie.extraTxnState.jobs
ex.extraTxnState.schemaChangerState = ie.extraTxnState.schemaChangerState
ex.extraTxnState.shouldResetSyntheticDescriptors = shouldResetSyntheticDescriptors
ex.extraTxnState.roleExistsCache = ie.extraTxnState.roleExistsCache
ex.initPlanner(ctx, &ex.planner)
}
}
Expand Down Expand Up @@ -1487,6 +1488,7 @@ type extraTxnState struct {
descCollection *descs.Collection
jobs *txnJobsCollection
schemaChangerState *SchemaChangerState
roleExistsCache map[username.SQLUsername]struct{}

// regionsProvider is populated lazily.
regionsProvider *regions.Provider
Expand Down Expand Up @@ -1628,6 +1630,7 @@ func (ief *InternalDB) newInternalExecutorWithTxn(
descCollection: descCol,
jobs: newTxnJobsCollection(),
schemaChangerState: schemaChangerState,
roleExistsCache: make(map[username.SQLUsername]struct{}),
},
}
populateMinimalSessionData(sd)
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/opt_catalog.go
Original file line number Diff line number Diff line change
Expand Up @@ -498,7 +498,7 @@ func (oc *optCatalog) fullyQualifiedNameWithTxn(

// RoleExists is part of the cat.Catalog interface.
func (oc *optCatalog) RoleExists(ctx context.Context, role username.SQLUsername) (bool, error) {
return RoleExists(ctx, oc.planner.InternalSQLTxn(), role)
return oc.planner.RoleExists(ctx, role)
}

// Optimizer is part of the cat.Catalog interface.
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/planner.go
Original file line number Diff line number Diff line change
Expand Up @@ -650,6 +650,7 @@ func (p *planner) InternalSQLTxn() descs.Txn {
descCollection: p.Descriptors(),
jobs: p.extendedEvalCtx.jobs,
schemaChangerState: p.extendedEvalCtx.SchemaChangerState,
roleExistsCache: p.extendedEvalCtx.RoleExistsCache,
}
p.internalSQLTxn.init(p.txn, ie)
}
Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/reassign_owned_by.go
Original file line number Diff line number Diff line change
Expand Up @@ -56,7 +56,7 @@ func (p *planner) ReassignOwnedBy(ctx context.Context, n *tree.ReassignOwnedBy)
// is a member of old roles and new roles and has CREATE privilege.
// Postgres first checks if the role exists before checking privileges.
for _, oldRole := range normalizedOldRoles {
roleExists, err := RoleExists(ctx, p.InternalSQLTxn(), oldRole)
roleExists, err := p.RoleExists(ctx, oldRole)
if err != nil {
return nil, err
}
Expand All @@ -70,7 +70,7 @@ func (p *planner) ReassignOwnedBy(ctx context.Context, n *tree.ReassignOwnedBy)
if err != nil {
return nil, err
}
roleExists, err := RoleExists(ctx, p.InternalSQLTxn(), newRole)
roleExists, err := p.RoleExists(ctx, newRole)
if !roleExists {
return nil, sqlerrors.NewUndefinedUserError(newRole)
}
Expand Down
7 changes: 7 additions & 0 deletions pkg/sql/sem/eval/context.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase"
"github.com/cockroachdb/cockroach/pkg/repstream/streampb"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"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/pgwire/pgcode"
Expand Down Expand Up @@ -286,6 +287,12 @@ type Context struct {
// execution until control returns to the parent routine. It is only valid
// during local execution. It may be unset.
RoutineSender DeferredRoutineSender

// RoleExistsCache is a cache of role existence checks. This is used because
// role existence checks are made when checking privileges, which can happen
// multiple times during the execution of a single query. Only positive
// values are cached. This cache is populated from the extraTxnState.
RoleExistsCache map[username.SQLUsername]struct{}
}

// JobsProfiler is the interface used to fetch job specific execution details
Expand Down
15 changes: 13 additions & 2 deletions pkg/sql/user.go
Original file line number Diff line number Diff line change
Expand Up @@ -473,9 +473,20 @@ func (p *planner) GetAllRoles(ctx context.Context) (map[username.SQLUsername]boo
return users, nil
}

// RoleExists returns true if the role exists.
// RoleExists returns true if the role exists. If a role is found to exist,
// the existence will be cached for the duration of the transaction.
func (p *planner) RoleExists(ctx context.Context, role username.SQLUsername) (bool, error) {
return RoleExists(ctx, p.InternalSQLTxn(), role)
cache := p.EvalContext().RoleExistsCache
if cache != nil {
if _, exists := cache[role]; exists {
return true, nil
}
}
exists, err := RoleExists(ctx, p.InternalSQLTxn(), role)
if cache != nil && exists {
cache[role] = struct{}{}
}
return exists, err
}

// RoleExists returns true if the role exists.
Expand Down

0 comments on commit bcbe369

Please sign in to comment.