From 635efa07f8405732d653dc91b4d17f9ebd4527d6 Mon Sep 17 00:00:00 2001 From: Paul Bardea Date: Thu, 20 Aug 2020 12:45:42 -0400 Subject: [PATCH 1/3] sql: fix panic when resolving a target on non-existent db 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. --- pkg/sql/logictest/testdata/logic_test/enums | 3 +++ pkg/sql/resolver.go | 3 +++ 2 files changed, 6 insertions(+) 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/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 } From bab8fc23c16fb4e90b2041cdb37aa75413245918 Mon Sep 17 00:00:00 2001 From: Rohan Yadav Date: Wed, 19 Aug 2020 16:24:10 -0400 Subject: [PATCH 2/3] stats: use leased types in the stats cache 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 --- pkg/server/server_sql.go | 2 + pkg/sql/catalog/resolver/resolver.go | 50 ----------------------- pkg/sql/stats/automatic_stats_test.go | 9 ++++ pkg/sql/stats/delete_stats_test.go | 3 ++ pkg/sql/stats/gossip_invalidation_test.go | 10 +++-- pkg/sql/stats/stats_cache.go | 50 +++++++++++++---------- pkg/sql/stats/stats_cache_test.go | 7 ++++ 7 files changed, 56 insertions(+), 75 deletions(-) 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/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 { From ab3463b4cfc466d4c3e4ce2741c42a08e6179a58 Mon Sep 17 00:00:00 2001 From: Rohan Yadav Date: Tue, 18 Aug 2020 15:49:35 -0400 Subject: [PATCH 3/3] sql: introduce a command to convert a database into a schema 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`. --- docs/generated/sql/bnf/stmt_block.bnf | 5 + pkg/sql/create_schema.go | 3 +- pkg/sql/drop_cascade.go | 2 +- pkg/sql/drop_database.go | 2 +- .../testdata/logic_test/reparent_database | 74 +++++ pkg/sql/opaque.go | 3 + pkg/sql/parser/sql.y | 11 +- pkg/sql/plan.go | 2 + pkg/sql/reparent_database.go | 275 ++++++++++++++++++ pkg/sql/sem/tree/rename.go | 14 + pkg/sql/sem/tree/stmt.go | 7 + pkg/sql/sqlbase/database_desc.go | 6 + pkg/sql/walk.go | 1 + 13 files changed, 400 insertions(+), 5 deletions(-) create mode 100644 pkg/sql/logictest/testdata/logic_test/reparent_database create mode 100644 pkg/sql/reparent_database.go 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/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/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/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/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",