Skip to content

Commit

Permalink
sql: add setting to control CREATE privilege on public schema
Browse files Browse the repository at this point in the history
This setting can be used to opt-in to the behavior that PostgreSQL
provides, as of PG15.

Release note (sql change): Added a new cluster setting named
sql.auth.public_schema_create_privilege.enabled. The setting
controls whether users receive `CREATE` privileges on
the public schema or not. The setting applies at the time that the
public schema is created, which happens whenever a database is
created. The setting is true by default.
  • Loading branch information
e-mbrown authored and rafiss committed May 24, 2023
1 parent c8084f8 commit 51a9861
Show file tree
Hide file tree
Showing 10 changed files with 74 additions and 25 deletions.
10 changes: 6 additions & 4 deletions pkg/ccl/backupccl/restore_job.go
Original file line number Diff line number Diff line change
Expand Up @@ -1074,10 +1074,10 @@ func createImportingDescriptors(
}
descsCol := txn.Descriptors()
// Write the new descriptors which are set in the OFFLINE state.
includePublicSchemaCreatePriv := sql.PublicSchemaCreatePrivilegeEnabled.Get(&p.ExecCfg().Settings.SV)
if err := ingesting.WriteDescriptors(
ctx, p.ExecCfg().Codec, txn.KV(), p.User(), descsCol,
databases, writtenSchemas, tables, writtenTypes, writtenFunctions,
details.DescriptorCoverage, nil /* extra */, restoreTempSystemDB,
ctx, txn.KV(), p.User(), descsCol, databases, writtenSchemas, tables, writtenTypes, writtenFunctions,
details.DescriptorCoverage, nil /* extra */, restoreTempSystemDB, includePublicSchemaCreatePriv,
); err != nil {
return errors.Wrapf(err, "restoring %d TableDescriptors from %d databases", len(tables), len(databases))
}
Expand Down Expand Up @@ -1461,10 +1461,12 @@ func remapPublicSchemas(
return err
}

includeCreatePriv := sql.PublicSchemaCreatePrivilegeEnabled.Get(p.ExecCfg().SV())

db.AddSchemaToDatabase(tree.PublicSchema, descpb.DatabaseDescriptor_SchemaInfo{ID: id})
// Every database must be initialized with the public schema.
// Create the SchemaDescriptor.
publicSchemaPrivileges := catpb.NewPublicSchemaPrivilegeDescriptor()
publicSchemaPrivileges := catpb.NewPublicSchemaPrivilegeDescriptor(includeCreatePriv)
publicSchemaDesc := schemadesc.NewBuilder(&descpb.SchemaDescriptor{
ParentID: db.GetID(),
Name: tree.PublicSchema,
Expand Down
12 changes: 7 additions & 5 deletions pkg/sql/catalog/catpb/privilege.go
Original file line number Diff line number Diff line change
Expand Up @@ -189,15 +189,17 @@ func NewBaseDatabasePrivilegeDescriptor(owner username.SQLUsername) *PrivilegeDe
// descriptor owned by the admin user which has CREATE and USAGE privilege for
// the public role, and ALL privileges for superusers. It is used for the
// public schema.
func NewPublicSchemaPrivilegeDescriptor() *PrivilegeDescriptor {
func NewPublicSchemaPrivilegeDescriptor(includeCreatePriv bool) *PrivilegeDescriptor {
// In postgres, the user "postgres" is the owner of the public schema in a
// newly created db. In CockroachDB, admin is our substitute for the postgres
// user.
p := NewBasePrivilegeDescriptor(username.AdminRoleName())
// By default, everyone has USAGE and CREATE on the public schema.
// Once https://github.com/cockroachdb/cockroach/issues/70266 is resolved,
// the public role will no longer have CREATE privileges.
p.Grant(username.PublicRoleName(), privilege.List{privilege.CREATE, privilege.USAGE}, false)

if includeCreatePriv {
p.Grant(username.PublicRoleName(), privilege.List{privilege.CREATE, privilege.USAGE}, false)
} else {
p.Grant(username.PublicRoleName(), privilege.List{privilege.USAGE}, false)
}
return p
}

Expand Down
6 changes: 5 additions & 1 deletion pkg/sql/catalog/ingesting/privileges.go
Original file line number Diff line number Diff line change
Expand Up @@ -45,6 +45,7 @@ func GetIngestingDescriptorPrivileges(
wroteDBs map[descpb.ID]catalog.DatabaseDescriptor,
wroteSchemas map[descpb.ID]catalog.SchemaDescriptor,
descCoverage tree.DescriptorCoverage,
includePublicSchemaCreatePriv bool,
) (updatedPrivileges *catpb.PrivilegeDescriptor, err error) {
switch desc := desc.(type) {
case catalog.TableDescriptor:
Expand All @@ -58,6 +59,7 @@ func GetIngestingDescriptorPrivileges(
wroteSchemas,
descCoverage,
privilege.Table,
includePublicSchemaCreatePriv,
)
case catalog.SchemaDescriptor:
return getIngestingPrivilegesForTableOrSchema(
Expand All @@ -70,6 +72,7 @@ func GetIngestingDescriptorPrivileges(
wroteSchemas,
descCoverage,
privilege.Schema,
includePublicSchemaCreatePriv,
)
case catalog.TypeDescriptor:
// If the ingestion is not a cluster restore we cannot know that the users
Expand Down Expand Up @@ -103,6 +106,7 @@ func getIngestingPrivilegesForTableOrSchema(
wroteSchemas map[descpb.ID]catalog.SchemaDescriptor,
descCoverage tree.DescriptorCoverage,
privilegeType privilege.ObjectType,
includePublicSchemaCreatePriv bool,
) (updatedPrivileges *catpb.PrivilegeDescriptor, err error) {
if _, ok := wroteDBs[desc.GetParentID()]; ok {
// If we're creating a new database in this ingestion, the tables and
Expand All @@ -111,7 +115,7 @@ func getIngestingPrivilegesForTableOrSchema(
switch privilegeType {
case privilege.Schema:
if desc.GetName() == tree.PublicSchema {
updatedPrivileges = catpb.NewPublicSchemaPrivilegeDescriptor()
updatedPrivileges = catpb.NewPublicSchemaPrivilegeDescriptor(includePublicSchemaCreatePriv)
} else {
updatedPrivileges = catpb.NewBasePrivilegeDescriptor(user)
}
Expand Down
13 changes: 6 additions & 7 deletions pkg/sql/catalog/ingesting/write_descs.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,7 +13,6 @@ package ingesting
import (
"context"

"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/kv/kvpb"
"github.com/cockroachdb/cockroach/pkg/roachpb"
Expand Down Expand Up @@ -46,7 +45,6 @@ import (
// inherited privileges during a cluster restore.
func WriteDescriptors(
ctx context.Context,
codec keys.SQLCodec,
txn *kv.Txn,
user username.SQLUsername,
descsCol *descs.Collection,
Expand All @@ -58,6 +56,7 @@ func WriteDescriptors(
descCoverage tree.DescriptorCoverage,
extra []roachpb.KeyValue,
inheritParentName string,
includePublicSchemaCreatePriv bool,
) (err error) {
ctx, span := tracing.ChildSpan(ctx, "WriteDescriptors")
defer span.Finish()
Expand All @@ -80,7 +79,7 @@ func WriteDescriptors(
for i := range databases {
desc := databases[i]
updatedPrivileges, err := GetIngestingDescriptorPrivileges(ctx, txn, descsCol, desc, user,
wroteDBs, wroteSchemas, descCoverage)
wroteDBs, wroteSchemas, descCoverage, includePublicSchemaCreatePriv)
if err != nil {
return err
}
Expand Down Expand Up @@ -119,7 +118,7 @@ func WriteDescriptors(
for i := range schemas {
sc := schemas[i]
updatedPrivileges, err := GetIngestingDescriptorPrivileges(ctx, txn, descsCol, sc, user,
wroteDBs, wroteSchemas, descCoverage)
wroteDBs, wroteSchemas, descCoverage, includePublicSchemaCreatePriv)
if err != nil {
return err
}
Expand Down Expand Up @@ -147,7 +146,7 @@ func WriteDescriptors(
for i := range tables {
table := tables[i]
updatedPrivileges, err := GetIngestingDescriptorPrivileges(ctx, txn, descsCol, table, user,
wroteDBs, wroteSchemas, descCoverage)
wroteDBs, wroteSchemas, descCoverage, includePublicSchemaCreatePriv)
if err != nil {
return err
}
Expand Down Expand Up @@ -181,7 +180,7 @@ func WriteDescriptors(
for i := range types {
typ := types[i]
updatedPrivileges, err := GetIngestingDescriptorPrivileges(ctx, txn, descsCol, typ, user,
wroteDBs, wroteSchemas, descCoverage)
wroteDBs, wroteSchemas, descCoverage, includePublicSchemaCreatePriv)
if err != nil {
return err
}
Expand All @@ -205,7 +204,7 @@ func WriteDescriptors(

for _, fn := range functions {
updatedPrivileges, err := GetIngestingDescriptorPrivileges(
ctx, txn, descsCol, fn, user, wroteDBs, wroteSchemas, descCoverage,
ctx, txn, descsCol, fn, user, wroteDBs, wroteSchemas, descCoverage, includePublicSchemaCreatePriv,
)
if err != nil {
return err
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/catalog/randgen/randgen.go
Original file line number Diff line number Diff line change
Expand Up @@ -53,7 +53,7 @@ func NewTestSchemaGenerator(
) TestSchemaGenerator {
dbPrivs := catpb.NewBaseDatabasePrivilegeDescriptor(callingUser)
dbDefaultPrivs := catprivilege.MakeDefaultPrivilegeDescriptor(catpb.DefaultPrivilegeDescriptor_DATABASE)
publicSchemaPrivs := catpb.NewPublicSchemaPrivilegeDescriptor()
publicSchemaPrivs := catpb.NewPublicSchemaPrivilegeDescriptor(true /*includeCreatePriv*/)
publicSchemaPrivs.SetOwner(callingUser)
g := &testSchemaGenerator{
rand: rand,
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/catalog/schemadesc/public_schema_desc.go
Original file line number Diff line number Diff line change
Expand Up @@ -55,7 +55,7 @@ var _ syntheticBase = publicBase{}
func (publicBase) kindName() string { return "public" }
func (publicBase) kind() catalog.ResolvedSchemaKind { return catalog.SchemaPublic }
func (publicBase) GetPrivileges() *catpb.PrivilegeDescriptor {
return catpb.NewPublicSchemaPrivilegeDescriptor()
return catpb.NewPublicSchemaPrivilegeDescriptor(true /*includeCreatePriv*/)
}

// publicDesc is a singleton returned by GetPublicSchema.
Expand Down
14 changes: 13 additions & 1 deletion pkg/sql/descriptor.go
Original file line number Diff line number Diff line change
Expand Up @@ -59,6 +59,17 @@ var (
errNoMatch = pgerror.New(pgcode.UndefinedObject, "no object matched")
)

// PublicSchemaCreatePrivilegeEnabled is the cluster setting that determines
// whether the CREATE privilege is given to the `public` role on the `public`
// schema at the time the schema is created.
var PublicSchemaCreatePrivilegeEnabled = settings.RegisterBoolSetting(
settings.TenantWritable,
"sql.auth.public_schema_create_privilege.enabled",
"determines whether to grant all users the CREATE privileges on the public "+
"schema when it is created",
true,
).WithPublic()

// createDatabase takes Database descriptor and creates it if needed,
// incrementing the descriptor counter. Returns true if the descriptor
// is actually created, false if it already existed, or an error if one was
Expand Down Expand Up @@ -145,11 +156,12 @@ func (p *planner) createDatabase(
dbdesc.MaybeWithDatabaseRegionConfig(regionConfig),
dbdesc.WithPublicSchemaID(publicSchemaID),
)
includeCreatePriv := PublicSchemaCreatePrivilegeEnabled.Get(&p.execCfg.Settings.SV)
publicSchema := schemadesc.NewBuilder(&descpb.SchemaDescriptor{
ParentID: id,
Name: tree.PublicSchema,
ID: publicSchemaID,
Privileges: catpb.NewPublicSchemaPrivilegeDescriptor(),
Privileges: catpb.NewPublicSchemaPrivilegeDescriptor(includeCreatePriv),
Version: 1,
}).BuildCreatedMutableSchema()

Expand Down
8 changes: 5 additions & 3 deletions pkg/sql/importer/import_job.go
Original file line number Diff line number Diff line change
Expand Up @@ -600,10 +600,12 @@ func prepareNewTablesForIngestion(
// Write the new TableDescriptors and flip the namespace entries over to
// them. After this call, any queries on a table will be served by the newly
// imported data.
includePublicSchemaCreatePriv := sql.PublicSchemaCreatePrivilegeEnabled.Get(&p.ExecCfg().Settings.SV)
if err := ingesting.WriteDescriptors(
ctx, p.ExecCfg().Codec, txn, p.User(), descsCol,
nil /* databases */, nil /* schemas */, tableDescs, nil /* types */, nil, /* functions */
tree.RequestedDescriptors, seqValKVs, "" /* inheritParentName */); err != nil {
ctx, txn, p.User(), descsCol, nil /* databases */, nil /* schemas */, tableDescs,
nil /* types */, nil /* functions */, tree.RequestedDescriptors, seqValKVs,
"" /* inheritParentName */, includePublicSchemaCreatePriv,
); err != nil {
return nil, errors.Wrapf(err, "creating importTables")
}

Expand Down
30 changes: 30 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/schema
Original file line number Diff line number Diff line change
Expand Up @@ -1324,3 +1324,33 @@ DROP DATABASE testdb1 CASCADE;
DROP DATABASE testdb2 CASCADE;

subtest end

subtest public_schema_create_privilege

statement ok
CREATE DATABASE should_have_create

statement ok
USE should_have_create

query TTTT
SELECT database_name, schema_name, grantee, privilege_type FROM [SHOW GRANTS ON SCHEMA public] WHERE grantee = 'public'
----
should_have_create public public CREATE
should_have_create public public USAGE

statement ok
SET CLUSTER SETTING sql.auth.public_schema_create_privilege.enabled = false

statement ok
CREATE DATABASE should_not_have_create

statement ok
USE should_not_have_create

query TTTT
SELECT database_name, schema_name, grantee, privilege_type FROM [SHOW GRANTS ON SCHEMA public] WHERE grantee = 'public'
----
should_not_have_create public public USAGE

subtest end
2 changes: 0 additions & 2 deletions pkg/sql/logictest/testdata/logic_test/schema_locked
Original file line number Diff line number Diff line change
Expand Up @@ -113,5 +113,3 @@ statement ok
DROP TABLE t;




0 comments on commit 51a9861

Please sign in to comment.