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 kicking off goroutines
to fetch these privileges eagerly at server startup. This is okay, but is
pretty expensive. Each privilege fetch itself internally runs a query which
means we're running 290 statements in 290 transactions to fetch this
information.

Ideally we'd do something better, like pre-warm the cache with one scan against
the system table.

Partially addresses cockroachdb#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, with some parallelism, to mitigate the latency when accessing
pg_catalog right after the server boots up.
  • Loading branch information
ajwerner committed Dec 12, 2022
1 parent 30fce41 commit ef773ac
Showing 3 changed files with 87 additions and 5 deletions.
5 changes: 5 additions & 0 deletions pkg/server/server_sql.go
Original file line number Diff line number Diff line change
@@ -1519,6 +1519,11 @@ func (s *SQLServer) preStart(
)

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

74 changes: 74 additions & 0 deletions pkg/sql/authorization.go
Original file line number Diff line number Diff line change
@@ -13,12 +13,14 @@ package sql
import (
"context"
"fmt"
"sync"

"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/keys"
"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 +35,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"
@@ -41,9 +44,11 @@ import (
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/mon"
"github.com/cockroachdb/cockroach/pkg/util/quotapool"
"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 +945,72 @@ func insufficientPrivilegeError(
"user %s does not have %s privilege on %s %s",
user, kind, typeForError, object.GetName())
}

// WarmSyntheticSchemaPrivilegeCacheForVirtualTables 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 WarmSyntheticSchemaPrivilegeCacheForVirtualTables(
ctx context.Context,
stopper *stop.Stopper,
version clusterversion.Handle,
cache *cacheutil.Cache,
schemas catalog.VirtualSchemas,
ief descs.TxnManager,
db *kv.DB,
) {
start := timeutil.Now()
var vTables []syntheticprivilege.VirtualTablePrivilege
for n := range catconstants.VirtualSchemaNames {
sc, _ := schemas.GetVirtualSchema(n)
sc.VisitTables(func(object catalog.VirtualObject) {
vTables = append(vTables, syntheticprivilege.VirtualTablePrivilege{
SchemaName: n,
TableName: sc.Desc().GetName(),
})
})
}
const opName = "virtual-table-privilege-cache-warm"
const concurrency = 8 // totally arbitrary
lim := quotapool.NewIntPool(opName, concurrency)
opts := stop.TaskOpts{
TaskName: opName,
Sem: lim,
WaitForSem: true,
}
errCh := make(chan error)
var wg sync.WaitGroup
for i := range vTables {
q := &vTables[i]
wg.Add(1)
_ = stopper.RunAsyncTaskEx(ctx, opts, func(ctx context.Context) {
defer wg.Done()
err := ief.DescsTxnWithExecutor(ctx, db, nil, func(
ctx context.Context, txn *kv.Txn, descriptors *descs.Collection,
ie sqlutil.InternalExecutor,
) error {
_, err := synthesizePrivilegeDescriptor(
ctx, version, cache, ie, descriptors, txn, q,
)
return err
})
if err != nil {
select {
case errCh <- err:
case <-ctx.Done():
}
}
})
}
go func() { wg.Wait(); close(errCh) }()
var totalErr error
for err := range errCh {
totalErr = errors.CombineErrors(totalErr, err)
}
if totalErr != nil {
log.Warningf(ctx, "failed to warm privileges for virtual tables: %v", totalErr)
} else {
log.Infof(ctx, "warmed virtual table privilege for %d table in %v", len(vTables), timeutil.Since(start))
}
}
13 changes: 8 additions & 5 deletions pkg/sql/grant_revoke_system.go
Original file line number Diff line number Diff line change
@@ -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,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)
@@ -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()

0 comments on commit ef773ac

Please sign in to comment.