diff --git a/docs/generated/sql/bnf/stmt_block.bnf b/docs/generated/sql/bnf/stmt_block.bnf index 593ed9d676d1..b9eb483a46fc 100644 --- a/docs/generated/sql/bnf/stmt_block.bnf +++ b/docs/generated/sql/bnf/stmt_block.bnf @@ -743,6 +743,7 @@ unreserved_keyword ::= | 'CONSTRAINTS' | 'CONTROLJOB' | 'CONVERSION' + | 'CONVERT' | 'COPY' | 'COVERING' | 'CREATEROLE' @@ -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 @@ -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 diff --git a/pkg/server/server_sql.go b/pkg/server/server_sql.go index 222e233d7ffb..592ede0bbbe5 100644 --- a/pkg/server/server_sql.go +++ b/pkg/server/server_sql.go @@ -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 diff --git a/pkg/sql/catalog/resolver/resolver.go b/pkg/sql/catalog/resolver/resolver.go index 2efb036bf990..52b89beb2ec0 100644 --- a/pkg/sql/catalog/resolver/resolver.go +++ b/pkg/sql/catalog/resolver/resolver.go @@ -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" @@ -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( diff --git a/pkg/sql/create_schema.go b/pkg/sql/create_schema.go index 143120b14457..682ea8b24aa7 100644 --- a/pkg/sql/create_schema.go +++ b/pkg/sql/create_schema.go @@ -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, diff --git a/pkg/sql/drop_cascade.go b/pkg/sql/drop_cascade.go index 38c50e70529c..06b0b66b297d 100644 --- a/pkg/sql/drop_cascade.go +++ b/pkg/sql/drop_cascade.go @@ -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, diff --git a/pkg/sql/drop_database.go b/pkg/sql/drop_database.go index 8a5c91cef9fa..7cedfe00de64 100644 --- a/pkg/sql/drop_database.go +++ b/pkg/sql/drop_database.go @@ -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, diff --git a/pkg/sql/logictest/testdata/logic_test/enums b/pkg/sql/logictest/testdata/logic_test/enums index 650204d7f1b4..e8aeeb015a9f 100644 --- a/pkg/sql/logictest/testdata/logic_test/enums +++ b/pkg/sql/logictest/testdata/logic_test/enums @@ -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') diff --git a/pkg/sql/logictest/testdata/logic_test/reparent_database b/pkg/sql/logictest/testdata/logic_test/reparent_database new file mode 100644 index 000000000000..de148ede1919 --- /dev/null +++ b/pkg/sql/logictest/testdata/logic_test/reparent_database @@ -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 diff --git a/pkg/sql/opaque.go b/pkg/sql/opaque.go index 2f3bfe8ec042..6a5228db684a 100644 --- a/pkg/sql/opaque.go +++ b/pkg/sql/opaque.go @@ -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: @@ -216,6 +218,7 @@ func init() { &tree.RenameDatabase{}, &tree.RenameIndex{}, &tree.RenameTable{}, + &tree.ReparentDatabase{}, &tree.Revoke{}, &tree.RevokeRole{}, &tree.Scatter{}, diff --git a/pkg/sql/parser/sql.y b/pkg/sql/parser/sql.y index 77cdd9ae9a0d..db0b9096f10e 100644 --- a/pkg/sql/parser/sql.y +++ b/pkg/sql/parser/sql.y @@ -576,7 +576,7 @@ func (u *sqlSymUnion) executorType() tree.ScheduledJobExecutorType { %token CHARACTER CHARACTERISTICS CHECK CLOSE %token CLUSTER COALESCE COLLATE COLLATION COLUMN COLUMNS COMMENT COMMENTS COMMIT %token COMMITTED COMPACT COMPLETE CONCAT CONCURRENTLY CONFIGURATION CONFIGURATIONS CONFIGURE -%token CONFLICT CONSTRAINT CONSTRAINTS CONTAINS CONTROLJOB CONVERSION COPY COVERING CREATE CREATEROLE +%token CONFLICT CONSTRAINT CONSTRAINTS CONTAINS CONTROLJOB CONVERSION CONVERT COPY COVERING CREATE CREATEROLE %token CROSS CUBE CURRENT CURRENT_CATALOG CURRENT_DATE CURRENT_SCHEMA %token CURRENT_ROLE CURRENT_TIME CURRENT_TIMESTAMP %token CURRENT_USER CYCLE @@ -714,6 +714,7 @@ func (u *sqlSymUnion) executorType() tree.ScheduledJobExecutorType { // ALTER DATABASE %type alter_rename_database_stmt +%type alter_database_to_schema_stmt %type alter_zone_database_stmt %type alter_database_owner @@ -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 @@ -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 { @@ -11173,6 +11181,7 @@ unreserved_keyword: | CONSTRAINTS | CONTROLJOB | CONVERSION +| CONVERT | COPY | COVERING | CREATEROLE diff --git a/pkg/sql/plan.go b/pkg/sql/plan.go index 9568d8552837..8fc4255344fd 100644 --- a/pkg/sql/plan.go +++ b/pkg/sql/plan.go @@ -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{} @@ -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 diff --git a/pkg/sql/reparent_database.go b/pkg/sql/reparent_database.go new file mode 100644 index 000000000000..d5671544578b --- /dev/null +++ b/pkg/sql/reparent_database.go @@ -0,0 +1,275 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package sql + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/resolver" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" + "github.com/cockroachdb/cockroach/pkg/util/protoutil" + "github.com/cockroachdb/errors" +) + +type reparentDatabaseNode struct { + n *tree.ReparentDatabase + db *sqlbase.MutableDatabaseDescriptor + newParent *sqlbase.MutableDatabaseDescriptor +} + +func (p *planner) ReparentDatabase( + ctx context.Context, n *tree.ReparentDatabase, +) (planNode, error) { + // We'll only allow the admin to perform this reparenting action. + if err := p.RequireAdminRole(ctx, "ALTER DATABASE ... CONVERT TO SCHEMA"); err != nil { + return nil, err + } + + // Ensure that the cluster version is high enough to create the schema. + if !p.ExecCfg().Settings.Version.IsActive(ctx, clusterversion.VersionUserDefinedSchemas) { + return nil, pgerror.Newf(pgcode.ObjectNotInPrerequisiteState, + `creating schemas requires all nodes to be upgraded to %s`, + clusterversion.VersionByKey(clusterversion.VersionUserDefinedSchemas)) + } + + // Check that creation of schemas is enabled. + if !p.EvalContext().SessionData.UserDefinedSchemasEnabled { + return nil, pgerror.Newf(pgcode.FeatureNotSupported, + "session variable experimental_enable_user_defined_schemas is set to false, cannot create a schema") + } + + db, err := p.ResolveMutableDatabaseDescriptor(ctx, string(n.Name), true /* required */) + if err != nil { + return nil, err + } + + parent, err := p.ResolveMutableDatabaseDescriptor(ctx, string(n.Parent), true /* required */) + if err != nil { + return nil, err + } + + // Ensure that this database wouldn't collide with a name under the new database. + exists, err := p.schemaExists(ctx, parent.ID, db.Name) + if err != nil { + return nil, err + } + if exists { + return nil, pgerror.Newf(pgcode.DuplicateSchema, "schema %q already exists", db.Name) + } + + // Ensure the database has a valid schema name. + if err := sqlbase.IsSchemaNameValid(db.Name); err != nil { + return nil, err + } + + // We can't reparent a database that has any child schemas other than public. + if len(db.Schemas) > 0 { + return nil, pgerror.Newf(pgcode.ObjectNotInPrerequisiteState, "cannot convert database with schemas into schema") + } + + return &reparentDatabaseNode{ + n: n, + db: db, + newParent: parent, + }, nil +} + +func (n *reparentDatabaseNode) startExec(params runParams) error { + ctx, p, codec := params.ctx, params.p, params.ExecCfg().Codec + + // Make a new schema corresponding to the target db. + id, err := catalogkv.GenerateUniqueDescID(ctx, p.ExecCfg().DB, codec) + if err != nil { + return err + } + schema := sqlbase.NewMutableCreatedSchemaDescriptor(descpb.SchemaDescriptor{ + ParentID: n.newParent.ID, + Name: n.db.Name, + ID: id, + Privileges: protoutil.Clone(n.db.Privileges).(*descpb.PrivilegeDescriptor), + }) + // Add the new schema to the parent database's name map. + if n.newParent.Schemas == nil { + n.newParent.Schemas = make(map[string]descpb.DatabaseDescriptor_SchemaInfo) + } + n.newParent.Schemas[n.db.Name] = descpb.DatabaseDescriptor_SchemaInfo{ + ID: schema.ID, + Dropped: false, + } + + if err := p.createDescriptorWithID( + ctx, + sqlbase.NewSchemaKey(n.newParent.ID, schema.Name).Key(p.ExecCfg().Codec), + id, + schema, + params.ExecCfg().Settings, + tree.AsStringWithFQNames(n.n, params.Ann()), + ); err != nil { + return err + } + + b := p.txn.NewBatch() + + // Get all objects under the target database. + objNames, err := resolver.GetObjectNames(ctx, p.txn, p, codec, n.db, tree.PublicSchema, true /* explicitPrefix */) + if err != nil { + return err + } + + // For each object, adjust the ParentID and ParentSchemaID fields to point + // to the new parent DB and schema. + for _, objName := range objNames { + // First try looking up objName as a table. + found, desc, err := p.LookupObject( + ctx, + tree.ObjectLookupFlags{ + // Note we set required to be false here in order to not error out + // if we don't find the object. + CommonLookupFlags: tree.CommonLookupFlags{Required: false}, + RequireMutable: true, + IncludeOffline: true, + DesiredObjectKind: tree.TableObject, + }, + objName.Catalog(), + objName.Schema(), + objName.Object(), + ) + if err != nil { + return err + } + if found { + // Remap the ID's on the table. + tbl, ok := desc.(*sqlbase.MutableTableDescriptor) + if !ok { + return errors.AssertionFailedf("%q was not a MutableTableDescriptor", objName.Object()) + } + + // If this table has any dependents, then we can't proceed (similar to the + // restrictions around renaming tables). See #10083. + if len(tbl.GetDependedOnBy()) > 0 { + var names []string + const errStr = "cannot convert database %q into schema because %q has dependent objects" + tblName, err := p.getQualifiedTableName(ctx, tbl) + if err != nil { + return errors.Wrapf(err, errStr, n.db.Name, tbl.Name) + } + for _, ref := range tbl.GetDependedOnBy() { + dep, err := p.Descriptors().GetMutableTableVersionByID(ctx, ref.ID, p.txn) + if err != nil { + return errors.Wrapf(err, errStr, n.db.Name, tblName.String()) + } + fqName, err := p.getQualifiedTableName(ctx, dep) + if err != nil { + return errors.Wrapf(err, errStr, n.db.Name, dep.Name) + } + names = append(names, fqName.String()) + } + return sqlbase.NewDependentObjectErrorf( + "could not convert database %q into schema because %q has dependent objects %v", + n.db.Name, + tblName.String(), + names, + ) + } + + tbl.AddDrainingName(descpb.NameInfo{ + ParentID: tbl.ParentID, + ParentSchemaID: tbl.GetParentSchemaID(), + Name: tbl.Name, + }) + tbl.ParentID = n.newParent.ID + tbl.UnexposedParentSchemaID = schema.ID + objKey := catalogkv.MakeObjectNameKey(ctx, p.ExecCfg().Settings, tbl.ParentID, tbl.GetParentSchemaID(), tbl.Name).Key(codec) + b.CPut(objKey, tbl.ID, nil /* expected */) + if err := p.writeSchemaChange(ctx, tbl, descpb.InvalidMutationID, tree.AsStringWithFQNames(n.n, params.Ann())); err != nil { + return err + } + } else { + // If we couldn't resolve objName as a table, try a type. + found, desc, err := p.LookupObject( + ctx, + tree.ObjectLookupFlags{ + CommonLookupFlags: tree.CommonLookupFlags{Required: true}, + RequireMutable: true, + IncludeOffline: true, + DesiredObjectKind: tree.TypeObject, + }, + objName.Catalog(), + objName.Schema(), + objName.Object(), + ) + if err != nil { + return err + } + // If we couldn't find the object at all, then continue. + if !found { + continue + } + // Remap the ID's on the type. + typ, ok := desc.(*sqlbase.MutableTypeDescriptor) + if !ok { + return errors.AssertionFailedf("%q was not a MutableTypeDescriptor", objName.Object()) + } + typ.AddDrainingName(descpb.NameInfo{ + ParentID: typ.ParentID, + ParentSchemaID: typ.ParentSchemaID, + Name: typ.Name, + }) + typ.ParentID = n.newParent.ID + typ.ParentSchemaID = schema.ID + objKey := catalogkv.MakeObjectNameKey(ctx, p.ExecCfg().Settings, typ.ParentID, typ.ParentSchemaID, typ.Name).Key(codec) + b.CPut(objKey, typ.ID, nil /* expected */) + if err := p.writeTypeSchemaChange(ctx, typ, tree.AsStringWithFQNames(n.n, params.Ann())); err != nil { + return err + } + } + } + + // Delete the public schema namespace entry for this database. Per our check + // during initialization, this is the only schema present under n.db. + b.Del(catalogkv.MakeObjectNameKey(ctx, p.ExecCfg().Settings, n.db.ID, keys.RootNamespaceID, tree.PublicSchema).Key(codec)) + + // This command can only be run when database leasing is supported, so we don't + // have to handle the case where it isn't. + n.db.AddDrainingName(descpb.NameInfo{ + ParentID: keys.RootNamespaceID, + ParentSchemaID: keys.RootNamespaceID, + Name: n.db.Name, + }) + n.db.State = descpb.DatabaseDescriptor_DROP + if err := p.writeDatabaseChangeToBatch(ctx, n.db, b); err != nil { + return err + } + + if err := p.txn.Run(ctx, b); err != nil { + return err + } + + return p.createDropDatabaseJob( + ctx, + n.db.ID, + nil, /* tableDropDetails */ + nil, /* typesToDrop */ + tree.AsStringWithFQNames(n.n, params.Ann()), + ) +} + +func (n *reparentDatabaseNode) Next(params runParams) (bool, error) { return false, nil } +func (n *reparentDatabaseNode) Values() tree.Datums { return tree.Datums{} } +func (n *reparentDatabaseNode) Close(ctx context.Context) {} +func (n *reparentDatabaseNode) ReadingOwnWrites() {} diff --git a/pkg/sql/resolver.go b/pkg/sql/resolver.go index 47a2a4d1a4cc..ef08b9a8f27b 100644 --- a/pkg/sql/resolver.go +++ b/pkg/sql/resolver.go @@ -120,6 +120,9 @@ func (p *planner) ResolveTargetObject( p.runWithOptions(resolveFlags{skipCache: true}, func() { prefix, namePrefix, err = resolver.ResolveTargetObject(ctx, p, un) }) + if err != nil { + return nil, namePrefix, err + } return prefix.Database, namePrefix, err } diff --git a/pkg/sql/sem/tree/rename.go b/pkg/sql/sem/tree/rename.go index ff857e1ad3f1..1f6766325b57 100644 --- a/pkg/sql/sem/tree/rename.go +++ b/pkg/sql/sem/tree/rename.go @@ -33,6 +33,20 @@ func (node *RenameDatabase) Format(ctx *FmtCtx) { ctx.FormatNode(&node.NewName) } +// ReparentDatabase represents a database reparenting as a schema operation. +type ReparentDatabase struct { + Name Name + Parent Name +} + +// Format implements the NodeFormatter interface. +func (node *ReparentDatabase) Format(ctx *FmtCtx) { + ctx.WriteString("ALTER DATABASE ") + node.Name.Format(ctx) + ctx.WriteString(" CONVERT TO SCHEMA WITH PARENT ") + node.Parent.Format(ctx) +} + // RenameTable represents a RENAME TABLE or RENAME VIEW or RENAME SEQUENCE // statement. Whether the user has asked to rename a view or a sequence // is indicated by the IsView and IsSequence fields. diff --git a/pkg/sql/sem/tree/stmt.go b/pkg/sql/sem/tree/stmt.go index 23f179a76e5c..db330bc764c7 100644 --- a/pkg/sql/sem/tree/stmt.go +++ b/pkg/sql/sem/tree/stmt.go @@ -592,6 +592,12 @@ func (*RenameDatabase) StatementType() StatementType { return DDL } // StatementTag returns a short string identifying the type of statement. func (*RenameDatabase) StatementTag() string { return "RENAME DATABASE" } +// StatementType implements the Statement interface. +func (*ReparentDatabase) StatementType() StatementType { return DDL } + +// StatementTag returns a short string identifying the type of statement. +func (*ReparentDatabase) StatementTag() string { return "TODO (rohany): Implement" } + // StatementType implements the Statement interface. func (*RenameIndex) StatementType() StatementType { return DDL } @@ -1051,6 +1057,7 @@ func (n *Relocate) String() string { return AsString(n) } func (n *RefreshMaterializedView) String() string { return AsString(n) } func (n *RenameColumn) String() string { return AsString(n) } func (n *RenameDatabase) String() string { return AsString(n) } +func (n *ReparentDatabase) String() string { return AsString(n) } func (n *RenameIndex) String() string { return AsString(n) } func (n *RenameTable) String() string { return AsString(n) } func (n *Restore) String() string { return AsString(n) } diff --git a/pkg/sql/sqlbase/database_desc.go b/pkg/sql/sqlbase/database_desc.go index 2069c92fdb48..2b603ff815ec 100644 --- a/pkg/sql/sqlbase/database_desc.go +++ b/pkg/sql/sqlbase/database_desc.go @@ -233,3 +233,9 @@ func (desc *MutableDatabaseDescriptor) Immutable() Descriptor { func (desc *MutableDatabaseDescriptor) IsNew() bool { return desc.ClusterVersion == nil } + +// AddDrainingName adds a draining name to the DatabaseDescriptor's slice of +// draining names. +func (desc *MutableDatabaseDescriptor) AddDrainingName(name descpb.NameInfo) { + desc.DrainingNames = append(desc.DrainingNames, name) +} diff --git a/pkg/sql/stats/automatic_stats_test.go b/pkg/sql/stats/automatic_stats_test.go index f12ba7da4105..6b21c3371ffc 100644 --- a/pkg/sql/stats/automatic_stats_test.go +++ b/pkg/sql/stats/automatic_stats_test.go @@ -24,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/lease" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -65,6 +66,8 @@ func TestMaybeRefreshStats(t *testing.T) { kvDB, executor, keys.SystemSQLCodec, + s.LeaseManager().(*lease.Manager), + s.ClusterSettings(), ) refresher := MakeRefresher(st, executor, cache, time.Microsecond /* asOfTime */) @@ -142,6 +145,8 @@ func TestAverageRefreshTime(t *testing.T) { kvDB, executor, keys.SystemSQLCodec, + s.LeaseManager().(*lease.Manager), + s.ClusterSettings(), ) refresher := MakeRefresher(st, executor, cache, time.Microsecond /* asOfTime */) @@ -378,6 +383,8 @@ func TestAutoStatsReadOnlyTables(t *testing.T) { kvDB, executor, keys.SystemSQLCodec, + s.LeaseManager().(*lease.Manager), + s.ClusterSettings(), ) refresher := MakeRefresher(st, executor, cache, time.Microsecond /* asOfTime */) @@ -416,6 +423,8 @@ func TestNoRetryOnFailure(t *testing.T) { kvDB, executor, keys.SystemSQLCodec, + s.LeaseManager().(*lease.Manager), + s.ClusterSettings(), ) r := MakeRefresher(st, executor, cache, time.Microsecond /* asOfTime */) diff --git a/pkg/sql/stats/delete_stats_test.go b/pkg/sql/stats/delete_stats_test.go index 40cab062e0a5..fb4eb438c312 100644 --- a/pkg/sql/stats/delete_stats_test.go +++ b/pkg/sql/stats/delete_stats_test.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/lease" "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" @@ -44,6 +45,8 @@ func TestDeleteOldStatsForColumns(t *testing.T) { db, ex, keys.SystemSQLCodec, + s.LeaseManager().(*lease.Manager), + s.ClusterSettings(), ) // The test data must be ordered by CreatedAt DESC so the calculated set of diff --git a/pkg/sql/stats/gossip_invalidation_test.go b/pkg/sql/stats/gossip_invalidation_test.go index d57aca51ecaf..13ab8a8054be 100644 --- a/pkg/sql/stats/gossip_invalidation_test.go +++ b/pkg/sql/stats/gossip_invalidation_test.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/lease" "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/sql/stats" "github.com/cockroachdb/cockroach/pkg/testutils" @@ -38,12 +39,15 @@ func TestGossipInvalidation(t *testing.T) { tc := testcluster.StartTestCluster(t, 3, base.TestClusterArgs{}) defer tc.Stopper().Stop(ctx) + s := tc.Server(0) sc := stats.NewTableStatisticsCache( 10, /* cacheSize */ - gossip.MakeOptionalGossip(tc.Server(0).GossipI().(*gossip.Gossip)), - tc.Server(0).DB(), - tc.Server(0).InternalExecutor().(sqlutil.InternalExecutor), + gossip.MakeOptionalGossip(s.GossipI().(*gossip.Gossip)), + s.DB(), + s.InternalExecutor().(sqlutil.InternalExecutor), keys.SystemSQLCodec, + s.LeaseManager().(*lease.Manager), + s.ClusterSettings(), ) sr0 := sqlutils.MakeSQLRunner(tc.ServerConn(0)) diff --git a/pkg/sql/stats/stats_cache.go b/pkg/sql/stats/stats_cache.go index ec7bff46dad1..ab50e7674aaa 100644 --- a/pkg/sql/stats/stats_cache.go +++ b/pkg/sql/stats/stats_cache.go @@ -18,8 +18,10 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/resolver" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/lease" "github.com/cockroachdb/cockroach/pkg/sql/opt/cat" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" @@ -61,6 +63,9 @@ type TableStatisticsCache struct { ClientDB *kv.DB SQLExecutor sqlutil.InternalExecutor Codec keys.SQLCodec + + LeaseMgr *lease.Manager + Settings *cluster.Settings } // The cache stores *cacheEntry objects. The fields are protected by the @@ -99,11 +104,15 @@ func NewTableStatisticsCache( db *kv.DB, sqlExecutor sqlutil.InternalExecutor, codec keys.SQLCodec, + leaseManager *lease.Manager, + settings *cluster.Settings, ) *TableStatisticsCache { tableStatsCache := &TableStatisticsCache{ ClientDB: db, SQLExecutor: sqlExecutor, Codec: codec, + LeaseMgr: leaseManager, + Settings: settings, } tableStatsCache.mu.cache = cache.NewUnorderedCache(cache.Config{ Policy: cache.CacheLRU, @@ -334,8 +343,8 @@ const ( // parseStats converts the given datums to a TableStatistic object. It might // need to run a query to get user defined type metadata. -func parseStats( - ctx context.Context, db *kv.DB, codec keys.SQLCodec, datums tree.Datums, +func (sc *TableStatisticsCache) parseStats( + ctx context.Context, datums tree.Datums, ) (*TableStatistic, error) { if datums == nil || datums.Len() == 0 { return nil, nil @@ -401,28 +410,25 @@ func parseStats( // Decode the histogram data so that it's usable by the opt catalog. res.Histogram = make([]cat.HistogramBucket, len(res.HistogramData.Buckets)) - typ := res.HistogramData.ColumnType // Hydrate the type in case any user defined types are present. // There are cases where typ is nil, so don't do anything if so. - if typ != nil && typ.UserDefined() { - // TODO (rohany): This should instead query a leased copy of the type. - // TODO (rohany): If we are caching data about types here, then this - // cache needs to be invalidated as well when type metadata changes. - // TODO (rohany): It might be better to store the type metadata used when - // collecting the stats in the HistogramData object itself, and avoid - // this query and caching/leasing problem. + if typ := res.HistogramData.ColumnType; typ != nil && typ.UserDefined() { // The metadata accessed here is never older than the metadata used when // collecting the stats. Changes to types are backwards compatible across // versions, so using a newer version of the type metadata here is safe. - err := db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - typeLookup := func(ctx context.Context, id descpb.ID) (tree.TypeName, sqlbase.TypeDescriptor, error) { - return resolver.ResolveTypeDescByID(ctx, txn, codec, id, tree.ObjectLookupFlags{}) - } - name, typeDesc, err := typeLookup(ctx, sqlbase.GetTypeDescID(typ)) - if err != nil { - return err - } - return typeDesc.HydrateTypeInfoWithName(ctx, typ, &name, sqlbase.TypeLookupFunc(typeLookup)) + // Given that we never delete members from enum types, a descriptor we + // get from the lease manager will be able to be used to decode these stats, + // even if it wasn't the descriptor that was used to collect the stats. + // If have types that are not backwards compatible in this way, then we + // will need to start writing a timestamp on the stats objects and request + // TypeDescriptor's with the timestamp that the stats were recorded with. + err := sc.ClientDB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + collection := descs.NewCollection(ctx, sc.Settings, sc.LeaseMgr) + defer collection.ReleaseAll(ctx) + resolver := descs.NewDistSQLTypeResolver(collection, txn) + var err error + res.HistogramData.ColumnType, err = resolver.ResolveTypeByOID(ctx, typ.Oid()) + return err }) if err != nil { return nil, err @@ -431,7 +437,7 @@ func parseStats( var a sqlbase.DatumAlloc for i := range res.Histogram { bucket := &res.HistogramData.Buckets[i] - datum, _, err := sqlbase.DecodeTableKey(&a, typ, bucket.UpperBound, encoding.Ascending) + datum, _, err := sqlbase.DecodeTableKey(&a, res.HistogramData.ColumnType, bucket.UpperBound, encoding.Ascending) if err != nil { return nil, err } @@ -476,7 +482,7 @@ ORDER BY "createdAt" DESC var statsList []*TableStatistic for _, row := range rows { - stats, err := parseStats(ctx, sc.ClientDB, sc.Codec, row) + stats, err := sc.parseStats(ctx, row) if err != nil { return nil, err } diff --git a/pkg/sql/stats/stats_cache_test.go b/pkg/sql/stats/stats_cache_test.go index 3092fea2cb28..06dfa276a5e5 100644 --- a/pkg/sql/stats/stats_cache_test.go +++ b/pkg/sql/stats/stats_cache_test.go @@ -25,6 +25,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/lease" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -233,6 +234,8 @@ func TestCacheBasic(t *testing.T) { db, ex, keys.SystemSQLCodec, + s.LeaseManager().(*lease.Manager), + s.ClusterSettings(), ) for _, tableID := range tableIDs { if err := checkStatsForTable(ctx, sc, expectedStats[tableID], tableID); err != nil { @@ -325,6 +328,8 @@ CREATE STATISTICS s FROM tt; kvDB, s.InternalExecutor().(sqlutil.InternalExecutor), keys.SystemSQLCodec, + s.LeaseManager().(*lease.Manager), + s.ClusterSettings(), ) tbl := catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "tt") // Get stats for our table. We are ensuring here that the access to the stats @@ -364,6 +369,8 @@ func TestCacheWait(t *testing.T) { db, ex, keys.SystemSQLCodec, + s.LeaseManager().(*lease.Manager), + s.ClusterSettings(), ) for _, tableID := range tableIDs { if err := checkStatsForTable(ctx, sc, expectedStats[tableID], tableID); err != nil { diff --git a/pkg/sql/walk.go b/pkg/sql/walk.go index 9e990a4c5a34..37047c012ee2 100644 --- a/pkg/sql/walk.go +++ b/pkg/sql/walk.go @@ -403,6 +403,7 @@ var planNodeNames = map[reflect.Type]string{ reflect.TypeOf(&renameDatabaseNode{}): "rename database", reflect.TypeOf(&renameIndexNode{}): "rename index", reflect.TypeOf(&renameTableNode{}): "rename table", + reflect.TypeOf(&reparentDatabaseNode{}): "TODO (rohany): fill out", reflect.TypeOf(&renderNode{}): "render", reflect.TypeOf(&RevokeRoleNode{}): "revoke role", reflect.TypeOf(&rowCountNode{}): "count",