Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
52997: sql: introduce a command to convert a database into a schema r=rohany a=rohany

Fixes #50885.

This commit introduces a command to convert a database into a user
defined schema under a desired database. This command can be used by
users who are currently emulating a Postgres style set of schemas in
CockroachDB with separate databases.

Release note (sql change): Users can now convert existing databases into
schemas under other databases through the
`ALTER DATABASE ... CONVERT TO SCHEMA UNDER PARENT ...` command. This
command can only be run by `admin` and is only valid for databases that
don't already have any child schemas other than `public`.

53078: stats: use leased types in the stats cache r=rohany a=rohany

This commit switches the stats cache to use leased types rather than
accessing types directly. This was the final user of the direct type
access methods, so that code is removed as well.

Release note: None

53137: sql: fix panic when resolving a target on non-existent db r=pbardea a=pbardea

When trying to resolve a target on non-existent DB, we would have a nil
deference panic due to a missing error check.

Some ways that could have triggered this panic:
- Resolving {types,sequences}
- Renaming a table

Release note (bug fix): Fix a crash that may occur when referencing a
database that does not exist when trying to create a type, sequence or
when renaming a table.

Co-authored-by: Rohan Yadav <[email protected]>
Co-authored-by: Paul Bardea <[email protected]>
  • Loading branch information
3 people committed Aug 20, 2020
4 parents 8bc4417 + ab3463b + bab8fc2 + 635efa0 commit 7023c94
Show file tree
Hide file tree
Showing 22 changed files with 462 additions and 80 deletions.
5 changes: 5 additions & 0 deletions docs/generated/sql/bnf/stmt_block.bnf
Original file line number Diff line number Diff line change
Expand Up @@ -743,6 +743,7 @@ unreserved_keyword ::=
| 'CONSTRAINTS'
| 'CONTROLJOB'
| 'CONVERSION'
| 'CONVERT'
| 'COPY'
| 'COVERING'
| 'CREATEROLE'
Expand Down Expand Up @@ -1117,6 +1118,7 @@ alter_database_stmt ::=
alter_rename_database_stmt
| alter_zone_database_stmt
| alter_database_owner
| alter_database_to_schema_stmt

alter_range_stmt ::=
alter_zone_range_stmt
Expand Down Expand Up @@ -1531,6 +1533,9 @@ alter_zone_database_stmt ::=
alter_database_owner ::=
'ALTER' 'DATABASE' database_name 'OWNER' 'TO' role_spec

alter_database_to_schema_stmt ::=
'ALTER' 'DATABASE' database_name 'CONVERT' 'TO' 'SCHEMA' 'WITH' 'PARENT' database_name

alter_zone_range_stmt ::=
'ALTER' 'RANGE' zone_name set_zone_config

Expand Down
2 changes: 2 additions & 0 deletions pkg/server/server_sql.go
Original file line number Diff line number Diff line change
Expand Up @@ -458,6 +458,8 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*sqlServer, error) {
cfg.db,
cfg.circularInternalExecutor,
codec,
leaseMgr,
cfg.Settings,
),

// Note: don't forget to add the secondary loggers as closers
Expand Down
50 changes: 0 additions & 50 deletions pkg/sql/catalog/resolver/resolver.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catconstants"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
Expand Down Expand Up @@ -274,55 +273,6 @@ func ResolveSchemaNameByID(
return "", errors.Newf("unable to resolve schema id %d for db %d", schemaID, dbID)
}

// ResolveTypeDescByID resolves a TypeDescriptor and fully qualified name
// from an ID.
// TODO (rohany): Once we start to cache type descriptors, this needs to
// look into the set of leased copies.
// TODO (rohany): Once we lease types, this should be pushed down into the
// leased object collection.
func ResolveTypeDescByID(
ctx context.Context,
txn *kv.Txn,
codec keys.SQLCodec,
id descpb.ID,
lookupFlags tree.ObjectLookupFlags,
) (tree.TypeName, sqlbase.TypeDescriptor, error) {
desc, err := catalogkv.GetDescriptorByID(ctx, txn, codec, id, catalogkv.Immutable,
catalogkv.TypeDescriptorKind, lookupFlags.Required)
if err != nil {
if pgerror.GetPGCode(err) == pgcode.WrongObjectType {
err = errors.HandleAsAssertionFailure(err)
}
return tree.TypeName{}, nil, err
}
// Get the parent database and schema names to create a fully qualified
// name for the type.
// TODO (SQLSchema): As we add leasing for all descriptors, these calls
// should look into those leased copies, rather than do raw reads.
typDesc := desc.(*sqlbase.ImmutableTypeDescriptor)
db, err := catalogkv.MustGetDatabaseDescByID(ctx, txn, codec, typDesc.ParentID)
if err != nil {
return tree.TypeName{}, nil, err
}
schemaName, err := ResolveSchemaNameByID(ctx, txn, codec, typDesc.ParentID, typDesc.ParentSchemaID)
if err != nil {
return tree.TypeName{}, nil, err
}
name := tree.MakeNewQualifiedTypeName(db.GetName(), schemaName, typDesc.GetName())
var ret sqlbase.TypeDescriptor
if lookupFlags.RequireMutable {
// TODO(ajwerner): Figure this out later when we construct this inside of
// the name resolution. This really shouldn't be happening here. Instead we
// should be taking a SchemaResolver and resolving through it which should
// be able to hit a descs.Collection and determine whether this is a new
// type or not.
desc = sqlbase.NewMutableExistingTypeDescriptor(*typDesc.TypeDesc())
} else {
ret = typDesc
}
return name, ret, nil
}

// GetForDatabase looks up and returns all available
// schema ids to names for a given database.
func GetForDatabase(
Expand Down
3 changes: 1 addition & 2 deletions pkg/sql/create_schema.go
Original file line number Diff line number Diff line change
Expand Up @@ -129,8 +129,7 @@ func (*createSchemaNode) Next(runParams) (bool, error) { return false, nil }
func (*createSchemaNode) Values() tree.Datums { return tree.Datums{} }
func (n *createSchemaNode) Close(ctx context.Context) {}

// CreateSchema creates a schema. Currently only works in IF NOT EXISTS mode,
// for schemas that do in fact already exist.
// CreateSchema creates a schema.
func (p *planner) CreateSchema(ctx context.Context, n *tree.CreateSchema) (planNode, error) {
return &createSchemaNode{
n: n,
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/drop_cascade.go
Original file line number Diff line number Diff line change
Expand Up @@ -70,7 +70,7 @@ func (d *dropCascadeState) resolveCollectedObjects(
ctx,
tree.ObjectLookupFlags{
// Note we set required to be false here in order to not error out
// if we don't find the object,
// if we don't find the object.
CommonLookupFlags: tree.CommonLookupFlags{Required: false},
RequireMutable: true,
IncludeOffline: true,
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/drop_database.go
Original file line number Diff line number Diff line change
Expand Up @@ -164,7 +164,7 @@ func (n *dropDatabaseNode) startExec(params runParams) error {
p.Descriptors().AddUncommittedDatabaseDeprecated(n.dbDesc.GetName(), n.dbDesc.GetID(), descs.DBDropped)

} else {
n.dbDesc.DrainingNames = append(n.dbDesc.DrainingNames, descpb.NameInfo{
n.dbDesc.AddDrainingName(descpb.NameInfo{
ParentID: keys.RootNamespaceID,
ParentSchemaID: keys.RootNamespaceID,
Name: n.dbDesc.Name,
Expand Down
3 changes: 3 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/enums
Original file line number Diff line number Diff line change
Expand Up @@ -1135,3 +1135,6 @@ public greeting2 hello
public int Z|S of int
public notbad dup|DUP
uds typ schema

statement error pq: cannot create "fakedb.typ" because the target database or schema does not exist
CREATE TYPE fakedb.typ AS ENUM ('schema')
74 changes: 74 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/reparent_database
Original file line number Diff line number Diff line change
@@ -0,0 +1,74 @@
statement ok
SET experimental_enable_user_defined_schemas = true;
SET experimental_enable_enums = true;

statement ok
CREATE DATABASE pgdatabase;
CREATE TABLE pgdatabase.t1 (x INT PRIMARY KEY);
CREATE DATABASE pgschema;
CREATE TABLE pgschema.t1 (x INT);
CREATE TABLE pgschema.t2 (x INT);
CREATE TABLE pgschema.t3 (x INT PRIMARY KEY);
ALTER TABLE pgschema.t3 ADD FOREIGN KEY (x) REFERENCES pgdatabase.t1 (x); -- references shouldn't be affected by reparenting.
CREATE TYPE pgschema.typ AS ENUM ('schema');

let $db_id
SELECT id FROM system.namespace WHERE name = 'pgschema'

statement ok
ALTER DATABASE pgschema CONVERT TO SCHEMA WITH PARENT pgdatabase

query I
SELECT * FROM pgdatabase.pgschema.t1

query I
SELECT * FROM pgdatabase.pgschema.t2

query T
SELECT 'schema'::pgdatabase.pgschema.typ
----
schema

statement error pq: insert on table "t3" violates foreign key constraint "fk_x_ref_t1"
INSERT INTO pgdatabase.pgschema.t3 VALUES (1)

# Assert there aren't any namespace entries left with the old parentID.
query T
SELECT name FROM system.namespace WHERE "parentID" = $db_id

# We can't reparent a database that has any child schemas.
statement ok
CREATE DATABASE parent;
USE parent;
CREATE SCHEMA child;
USE test;

statement error pq: cannot convert database with schemas into schema
ALTER DATABASE parent CONVERT TO SCHEMA WITH PARENT pgdatabase

# We can't reparent a database if it causes a name conflict.
statement ok
CREATE DATABASE pgschema

statement error pq: schema "pgschema" already exists
ALTER DATABASE pgschema CONVERT TO SCHEMA WITH PARENT pgdatabase

statement ok
DROP DATABASE pgschema

# We can't convert a database with an invalid schema name into a schema.
statement ok
CREATE DATABASE pg_temp

statement error pq: unacceptable schema name "pg_temp"
ALTER DATABASE pg_temp CONVERT TO SCHEMA WITH PARENT pgdatabase

# We can't reparent a database that has any tables in use by views,
# because we aren't able to rewrite those references.
statement ok
CREATE DATABASE with_views;
CREATE TABLE with_views.t (x INT);
CREATE VIEW with_views.v AS SELECT x FROM with_views.t

statement error pq: could not convert database "with_views" into schema because "with_views.public.t" has dependent objects \[with_views.public.v\]
ALTER DATABASE with_views CONVERT TO SCHEMA WITH PARENT pgdatabase
3 changes: 3 additions & 0 deletions pkg/sql/opaque.go
Original file line number Diff line number Diff line change
Expand Up @@ -119,6 +119,8 @@ func buildOpaque(
plan, err = p.RenameColumn(ctx, n)
case *tree.RenameDatabase:
plan, err = p.RenameDatabase(ctx, n)
case *tree.ReparentDatabase:
plan, err = p.ReparentDatabase(ctx, n)
case *tree.RenameIndex:
plan, err = p.RenameIndex(ctx, n)
case *tree.RenameTable:
Expand Down Expand Up @@ -216,6 +218,7 @@ func init() {
&tree.RenameDatabase{},
&tree.RenameIndex{},
&tree.RenameTable{},
&tree.ReparentDatabase{},
&tree.Revoke{},
&tree.RevokeRole{},
&tree.Scatter{},
Expand Down
11 changes: 10 additions & 1 deletion pkg/sql/parser/sql.y
Original file line number Diff line number Diff line change
Expand Up @@ -576,7 +576,7 @@ func (u *sqlSymUnion) executorType() tree.ScheduledJobExecutorType {
%token <str> CHARACTER CHARACTERISTICS CHECK CLOSE
%token <str> CLUSTER COALESCE COLLATE COLLATION COLUMN COLUMNS COMMENT COMMENTS COMMIT
%token <str> COMMITTED COMPACT COMPLETE CONCAT CONCURRENTLY CONFIGURATION CONFIGURATIONS CONFIGURE
%token <str> CONFLICT CONSTRAINT CONSTRAINTS CONTAINS CONTROLJOB CONVERSION COPY COVERING CREATE CREATEROLE
%token <str> CONFLICT CONSTRAINT CONSTRAINTS CONTAINS CONTROLJOB CONVERSION CONVERT COPY COVERING CREATE CREATEROLE
%token <str> CROSS CUBE CURRENT CURRENT_CATALOG CURRENT_DATE CURRENT_SCHEMA
%token <str> CURRENT_ROLE CURRENT_TIME CURRENT_TIMESTAMP
%token <str> CURRENT_USER CYCLE
Expand Down Expand Up @@ -714,6 +714,7 @@ func (u *sqlSymUnion) executorType() tree.ScheduledJobExecutorType {

// ALTER DATABASE
%type <tree.Statement> alter_rename_database_stmt
%type <tree.Statement> alter_database_to_schema_stmt
%type <tree.Statement> alter_zone_database_stmt
%type <tree.Statement> alter_database_owner

Expand Down Expand Up @@ -1387,6 +1388,7 @@ alter_database_stmt:
alter_rename_database_stmt
| alter_zone_database_stmt
| alter_database_owner
| alter_database_to_schema_stmt
// ALTER DATABASE has its error help token here because the ALTER DATABASE
// prefix is spread over multiple non-terminals.
| ALTER DATABASE error // SHOW HELP: ALTER DATABASE
Expand Down Expand Up @@ -6597,6 +6599,12 @@ opt_asc_desc:
$$.val = tree.DefaultDirection
}

alter_database_to_schema_stmt:
ALTER DATABASE database_name CONVERT TO SCHEMA WITH PARENT database_name
{
$$.val = &tree.ReparentDatabase{Name: tree.Name($3), Parent: tree.Name($9)}
}

alter_rename_database_stmt:
ALTER DATABASE database_name RENAME TO database_name
{
Expand Down Expand Up @@ -11173,6 +11181,7 @@ unreserved_keyword:
| CONSTRAINTS
| CONTROLJOB
| CONVERSION
| CONVERT
| COPY
| COVERING
| CREATEROLE
Expand Down
2 changes: 2 additions & 0 deletions pkg/sql/plan.go
Original file line number Diff line number Diff line change
Expand Up @@ -192,6 +192,7 @@ var _ planNode = &renameColumnNode{}
var _ planNode = &renameDatabaseNode{}
var _ planNode = &renameIndexNode{}
var _ planNode = &renameTableNode{}
var _ planNode = &reparentDatabaseNode{}
var _ planNode = &renderNode{}
var _ planNode = &RevokeRoleNode{}
var _ planNode = &rowCountNode{}
Expand Down Expand Up @@ -237,6 +238,7 @@ var _ planNodeReadingOwnWrites = &changePrivilegesNode{}
var _ planNodeReadingOwnWrites = &dropSchemaNode{}
var _ planNodeReadingOwnWrites = &dropTypeNode{}
var _ planNodeReadingOwnWrites = &refreshMaterializedViewNode{}
var _ planNodeReadingOwnWrites = &reparentDatabaseNode{}
var _ planNodeReadingOwnWrites = &setZoneConfigNode{}

// planNodeRequireSpool serves as marker for nodes whose parent must
Expand Down
Loading

0 comments on commit 7023c94

Please sign in to comment.