diff --git a/pkg/server/server_sql.go b/pkg/server/server_sql.go index 537cd212e564..dde618e2a776 100644 --- a/pkg/server/server_sql.go +++ b/pkg/server/server_sql.go @@ -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 } diff --git a/pkg/sql/authorization.go b/pkg/sql/authorization.go index b6dc3f9c8926..880537f87613 100644 --- a/pkg/sql/authorization.go +++ b/pkg/sql/authorization.go @@ -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" @@ -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" @@ -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" ) @@ -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) + }) + } +} diff --git a/pkg/sql/grant_revoke_system.go b/pkg/sql/grant_revoke_system.go index 7edac5eca5fb..4d2afdd0ab64 100644 --- a/pkg/sql/grant_revoke_system.go +++ b/pkg/sql/grant_revoke_system.go @@ -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" @@ -271,7 +272,8 @@ 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 @@ -279,7 +281,8 @@ func (p *planner) getPrivilegeDescriptor( 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) @@ -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 } @@ -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() @@ -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 @@ -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 +} diff --git a/pkg/sql/syntheticprivilege/vtable_privilege.go b/pkg/sql/syntheticprivilege/vtable_privilege.go index 67f3fcbcd00a..840064ea0b01 100644 --- a/pkg/sql/syntheticprivilege/vtable_privilege.go +++ b/pkg/sql/syntheticprivilege/vtable_privilege.go @@ -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.