Skip to content

Commit

Permalink
sql,server: fetch privileges for virtual tables on startup
Browse files Browse the repository at this point in the history
This commit attempts to alleviate the pain caused by synthetic virtual table
privileges introduced in 22.2. We need to fetch privileges for virtual tables
to determine whether the user has access. This is done lazily. However, when
a user attempts to read a virtual table like pg_class or run `SHOW TABLES` it
will force the privileges to be determined for each virtual table (of which
there are 290 at the time of writing). This sequential process can be somewhat
slow in a single region cluster and will be *very* slow in an MR cluster.

This patch attempts to somewhat alleviate this pain by scanning the
table eagerly during server startup.

Partially addresses #93182.

Release note (performance improvement): In 22.2 we introduced privileges on
virtual tables (system catalogs like pg_catalog, information_schema, and
crdb_internal). A problem with this new feature is that we now must fetch those
privileges into a cache before we can use those tables or determine their
visibility in other system catalogs. This process used to occur on-demand, when
the privilege was needed. Now we'll fetch these privileges eagerly during
startup to mitigate the latency when accessing pg_catalog right after the
server boots up.
  • Loading branch information
ajwerner authored and rafiss committed Dec 12, 2022
1 parent 30fce41 commit 6a7c337
Show file tree
Hide file tree
Showing 4 changed files with 160 additions and 38 deletions.
5 changes: 5 additions & 0 deletions pkg/server/server_sql.go
Original file line number Diff line number Diff line change
Expand Up @@ -1519,6 +1519,11 @@ func (s *SQLServer) preStart(
)

scheduledlogging.Start(ctx, stopper, s.execCfg.DB, s.execCfg.Settings, s.internalExecutor, s.execCfg.CaptureIndexUsageStatsKnobs)
sql.WarmSyntheticPrivilegeCacheForVirtualTables(
ctx, stopper, s.execCfg.Settings.Version,
s.execCfg.SyntheticPrivilegeCache, s.execCfg.VirtualSchemas,
s.execCfg.InternalExecutorFactory, s.execCfg.DB,
)
return nil
}

Expand Down
101 changes: 101 additions & 0 deletions pkg/sql/authorization.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/security/username"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/sql/cacheutil"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/dbdesc"
Expand All @@ -33,6 +34,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/privilege"
"github.com/cockroachdb/cockroach/pkg/sql/roleoption"
"github.com/cockroachdb/cockroach/pkg/sql/sem/catconstants"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
"github.com/cockroachdb/cockroach/pkg/sql/sessioninit"
Expand All @@ -44,6 +46,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/stop"
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
"github.com/cockroachdb/cockroach/pkg/util/syncutil/singleflight"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/errors"
"github.com/cockroachdb/logtags"
)
Expand Down Expand Up @@ -940,3 +943,101 @@ func insufficientPrivilegeError(
"user %s does not have %s privilege on %s %s",
user, kind, typeForError, object.GetName())
}

// WarmSyntheticPrivilegeCacheForVirtualTables will attempt to access
// each virtual table, and, in doing so, will cache the privileges for those
// virtual tables. This work is done in parallel. It aids in reducing the cold
// start latency when accessing virtual tables which themselves list virtual
// tables.
func WarmSyntheticPrivilegeCacheForVirtualTables(
ctx context.Context,
stopper *stop.Stopper,
version clusterversion.Handle,
cache *cacheutil.Cache,
schemas catalog.VirtualSchemas,
ief descs.TxnManager,
db *kv.DB,
) {
if !version.IsActive(ctx, clusterversion.V22_2SystemPrivilegesTable) {
return
}

start := timeutil.Now()
var totalErr error
if totalErr != nil {
log.Warningf(ctx, "failed to warm privileges for virtual tables: %v", totalErr)
} else {
log.Infof(ctx, "warmed privileges for virtual tables in %v", timeutil.Since(start))
}

var tableVersions []descpb.DescriptorVersion
vtablePathToPrivilegeDescs := make(map[string]*catpb.PrivilegeDescriptor)
query := fmt.Sprintf(
`SELECT path, username, privileges, grant_options FROM system.%s WHERE path LIKE '/%s/%%'`,
syntheticprivilege.VirtualTablePathPrefix,
)
totalErr = ief.DescsTxnWithExecutor(ctx, db, nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, descsCol *descs.Collection, ie sqlutil.InternalExecutor) (retErr error) {
_, systemPrivDesc, err := descsCol.GetImmutableTableByName(
ctx,
txn,
syntheticprivilege.SystemPrivilegesTableName,
tree.ObjectLookupFlagsWithRequired(),
)
if err != nil {
return err
}
if systemPrivDesc.IsUncommittedVersion() {
// This shouldn't ever happen, but if it does somehow, then we can't pre-warm the cache.
return errors.Newf("%s is at an uncommitted version", syntheticprivilege.SystemPrivilegesTableName)
}
tableVersions = []descpb.DescriptorVersion{systemPrivDesc.GetVersion()}

it, err := ie.QueryIteratorEx(
ctx, `get-vtable-privileges`, txn, sessiondata.NodeUserSessionDataOverride, query,
)
if err != nil {
return err
}
defer func() {
retErr = errors.CombineErrors(retErr, it.Close())
}()

for {
ok, err := it.Next(ctx)
if err != nil {
return err
}
if !ok {
break
}
path := tree.MustBeDString(it.Cur()[0])
user := tree.MustBeDString(it.Cur()[1])
privArr := tree.MustBeDArray(it.Cur()[2])
grantOptionArr := tree.MustBeDArray(it.Cur()[3])
privileges, err := synthesizePrivilegeDescriptorFromSystemPrivilegesRow(privilege.VirtualTable, user, privArr, grantOptionArr)
if err != nil {
return err
}
vtablePathToPrivilegeDescs[string(path)] = privileges
}
return nil
})
if totalErr != nil {
return
}

for scName := range catconstants.VirtualSchemaNames {
sc, _ := schemas.GetVirtualSchema(scName)
sc.VisitTables(func(object catalog.VirtualObject) {
vtablePriv := syntheticprivilege.VirtualTablePrivilege{
SchemaName: scName,
TableName: sc.Desc().GetName(),
}
privDesc, ok := vtablePathToPrivilegeDescs[vtablePriv.GetPath()]
if !ok {
privDesc = vtablePriv.GetFallbackPrivileges()
}
cache.MaybeWriteBackToCache(ctx, tableVersions, vtablePriv.GetPath(), *privDesc)
})
}
}
87 changes: 50 additions & 37 deletions pkg/sql/grant_revoke_system.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/kv"
"github.com/cockroachdb/cockroach/pkg/security/username"
"github.com/cockroachdb/cockroach/pkg/sql/cacheutil"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catprivilege"
Expand Down Expand Up @@ -271,15 +272,17 @@ func (p *planner) getPrivilegeDescriptor(
TableName: d.GetName(),
}
return synthesizePrivilegeDescriptor(
ctx, p.ExecCfg(), p.ExecCfg().InternalExecutor, p.Descriptors(), p.Txn(), vDesc,
ctx, p.ExecCfg().Settings.Version, p.ExecCfg().SyntheticPrivilegeCache,
p.ExecCfg().InternalExecutor, p.Descriptors(), p.Txn(), vDesc,
)
}
return d.GetPrivileges(), nil
case catalog.Descriptor:
return d.GetPrivileges(), nil
case syntheticprivilege.Object:
return synthesizePrivilegeDescriptor(
ctx, p.ExecCfg(), p.ExecCfg().InternalExecutor, p.Descriptors(), p.Txn(), d,
ctx, p.ExecCfg().Settings.Version, p.ExecCfg().SyntheticPrivilegeCache,
p.ExecCfg().InternalExecutor, p.Descriptors(), p.Txn(), d,
)
}
return nil, errors.AssertionFailedf("unknown privilege.Object type %T", po)
Expand All @@ -290,13 +293,14 @@ func (p *planner) getPrivilegeDescriptor(
// PrivilegeDescriptor.
func synthesizePrivilegeDescriptor(
ctx context.Context,
execCfg *ExecutorConfig,
version clusterversion.Handle,
cache *cacheutil.Cache,
ie sqlutil.InternalExecutor,
descsCol *descs.Collection,
txn *kv.Txn,
spo syntheticprivilege.Object,
) (*catpb.PrivilegeDescriptor, error) {
if !execCfg.Settings.Version.IsActive(ctx, spo.SystemPrivilegesTableVersionGate()) {
if !version.IsActive(ctx, spo.SystemPrivilegesTableVersionGate()) {
// Fall back to defaults if the version gate is not active yet.
return spo.GetFallbackPrivileges(), nil
}
Expand All @@ -313,7 +317,6 @@ func synthesizePrivilegeDescriptor(
return synthesizePrivilegeDescriptorFromSystemPrivilegesTable(ctx, ie, txn, spo)
}
var tableVersions []descpb.DescriptorVersion
cache := execCfg.SyntheticPrivilegeCache
found, privileges, retErr := func() (bool, catpb.PrivilegeDescriptor, error) {
cache.Lock()
defer cache.Unlock()
Expand Down Expand Up @@ -382,42 +385,11 @@ func synthesizePrivilegeDescriptorFromSystemPrivilegesTable(

user := tree.MustBeDString(it.Cur()[0])
privArr := tree.MustBeDArray(it.Cur()[1])
var privilegeStrings []string
for _, elem := range privArr.Array {
privilegeStrings = append(privilegeStrings, string(tree.MustBeDString(elem)))
}

grantOptionArr := tree.MustBeDArray(it.Cur()[2])
var grantOptionStrings []string
for _, elem := range grantOptionArr.Array {
grantOptionStrings = append(grantOptionStrings, string(tree.MustBeDString(elem)))
}
privs, err := privilege.ListFromStrings(privilegeStrings)
privileges, err = synthesizePrivilegeDescriptorFromSystemPrivilegesRow(spo.GetObjectType(), user, privArr, grantOptionArr)
if err != nil {
return nil, err
}
grantOptions, err := privilege.ListFromStrings(grantOptionStrings)
if err != nil {
return nil, err
}
privsWithGrantOption := privilege.ListFromBitField(
privs.ToBitField()&grantOptions.ToBitField(),
spo.GetObjectType(),
)
privsWithoutGrantOption := privilege.ListFromBitField(
privs.ToBitField()&^privsWithGrantOption.ToBitField(),
spo.GetObjectType(),
)
privileges.Grant(
username.MakeSQLUsernameFromPreNormalizedString(string(user)),
privsWithGrantOption,
true, /* withGrantOption */
)
privileges.Grant(
username.MakeSQLUsernameFromPreNormalizedString(string(user)),
privsWithoutGrantOption,
false, /* withGrantOption */
)
}

// To avoid having to insert a row for public for each virtual
Expand All @@ -443,3 +415,44 @@ func synthesizePrivilegeDescriptorFromSystemPrivilegesTable(
}
return privileges, err
}

func synthesizePrivilegeDescriptorFromSystemPrivilegesRow(
objectType privilege.ObjectType, user tree.DString, privArr, grantOptionArr *tree.DArray,
) (privileges *catpb.PrivilegeDescriptor, retErr error) {
var privilegeStrings []string
for _, elem := range privArr.Array {
privilegeStrings = append(privilegeStrings, string(tree.MustBeDString(elem)))
}

var grantOptionStrings []string
for _, elem := range grantOptionArr.Array {
grantOptionStrings = append(grantOptionStrings, string(tree.MustBeDString(elem)))
}
privs, err := privilege.ListFromStrings(privilegeStrings)
if err != nil {
return nil, err
}
grantOptions, err := privilege.ListFromStrings(grantOptionStrings)
if err != nil {
return nil, err
}
privsWithGrantOption := privilege.ListFromBitField(
privs.ToBitField()&grantOptions.ToBitField(),
objectType,
)
privsWithoutGrantOption := privilege.ListFromBitField(
privs.ToBitField()&^privsWithGrantOption.ToBitField(),
objectType,
)
privileges.Grant(
username.MakeSQLUsernameFromPreNormalizedString(string(user)),
privsWithGrantOption,
true, /* withGrantOption */
)
privileges.Grant(
username.MakeSQLUsernameFromPreNormalizedString(string(user)),
privsWithoutGrantOption,
false, /* withGrantOption */
)
return privileges, nil
}
5 changes: 4 additions & 1 deletion pkg/sql/syntheticprivilege/vtable_privilege.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,9 +32,12 @@ const VirtualTablePrivilegeType = "VirtualTable"

var _ Object = &VirtualTablePrivilege{}

// VirtualTablePathPrefix is the prefix used for virtual table privileges in system.privileges.
var VirtualTablePathPrefix = "vtable"

// GetPath implements the Object interface.
func (p *VirtualTablePrivilege) GetPath() string {
return fmt.Sprintf("/vtable/%s/%s", p.SchemaName, p.TableName)
return fmt.Sprintf("/%s/%s/%s", VirtualTablePathPrefix, p.SchemaName, p.TableName)
}

// SystemPrivilegesTableVersionGate implements the Object interface.
Expand Down

0 comments on commit 6a7c337

Please sign in to comment.