From 90f5b694a62cbda84badc9e87849cdf927ed4930 Mon Sep 17 00:00:00 2001 From: Arul Ajmani Date: Fri, 8 Nov 2019 22:11:46 -0500 Subject: [PATCH] sql, *: allow table scoping under pg_temp_ physical schema MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Previously, CRDB only supported the `public` physical schema. All table entries in `system.namespace` assumed an implied `public` schema, so name resolution for tables only required a databaseID and table name to uniquely identify a table. As highlighted in the temp tables RFC, temp tables will be scoped under a session specific schema of the form `pg_temp_`. This motivated adding support for additional physical schemas. This PR involves the following changes to `system.namespace`: - A new `system.namespace` table is constructed for cluster versions >= 20.1, which has an additional primary key column `publicSchemaID`. - The older `system.namespace` table is marked deprecated. All `system.namespace` read accesses default to the new `system.namespace`. If a match isn't found, the deprecated `system.namespace` is checked. - All `system.namespace` write accesses for key deletions remove entries from both versions of the table. This ensures the fallback code doesn't read an old key that was deleted. - All `system.namespace` write accesses that involve creating new entries are added to the `system.namespace` table according to the cluster version. - Selecting from `system.namespace` in mixed version clusters actually selects from `system.namespace_deprecated`, ensuring that the change is invisible to users. - The new `system.namespace` table is moved out of the SystemConfig range. This means `system.namespace` is no longer gossiped for cluster versions >= 20.1 . - Every new database creation adds the `public` schema to `system.namespace` by default. As a result of the above changes to `system.namespace`, there is a new interface that all accesses should go through. - Lookups Previously: Keys were constructed and directly used to perform KV lookups. Now: Use LookupObjectID, or another specialized lookup method provided. This ensures correct fallback semantics for mixed-version 19.2/20.1 clusters. - Removals Previously: Keys were constructed and directly used to perform KV deletes. Now: Use RemoveObjectNamespaceEntry or another specialized method provided. This ensures correct behavior for mixed-version 19.2/20.1 clusters. - Additions Previously: Keys were constructed and directly used to perform CPuts with the appropriate value. Now: Use MakeObjectNameKey or another specialized method provided to construct the key. This ensures that the key created is for the appropriate cluster version. These methods should only be used to create keys for adding entries -- removals/lookups should go through the appropriate interfaces. The `search_path` is responsible for establishing the order in which schemas are searched during name resolution. This PR involves the following changes to the `search_path.go`: - The search semantics are updated to match those described in the temp tables RFC. - The search path is now aware of the session specific temporary schema, which can be used during name resolution. - The distSQL api proto had to be updated to pass the temporary schema to other nodes in addition to the list of paths. Benchmarks: TPC-C with 3 nodes/16CPUs: - max warehouses: 1565 Microbenchmarks for system.namespace access: | name | master time/op | new approach time/op | delta | | ----------------------------------- | -------------- | -------------------- | ----- | | NameResolution/Cockroach-8 | 163µs ± 0% | 252µs ± 0% | ~ | | NameResolution/MultinodeCockroach-8 | 419µs ± 0% | 797µs ± 0% | ~ | | name | master time/op | new approach time/op | delta | | -------------------------------------------------- | -------------- | -------------------- | ----- | | NameResolutionTempTablesExist/Cockroach-8 | 175µs ± 0% | 337µs ± 0% | ~ | | NameResolutionTempTablesExist/MultinodeCockroach-8 | 1.06ms ± 0% | 1.07ms ± 0% | ~ | Follow-up work: - The `TableCollection` cache needs to be updated to have knowledge about schemaIDs. Once this is done, there is a TODO in the code that allows the `CachedPhysicalAccessor` to work correctly. - Migration for clusters upgrading to 20.1. The new `system.namespace` table needs to be populated from the deprecated table and a `public` schema needs to be added for every database during migration. Release note (sql change): CREATE TABLE pg_temp.abc(a int) now creates a temporary table. See temp tables RFC (guide-level explanation) for more details about the search path semantics. --- docs/generated/settings/settings.html | 2 +- pkg/bench/bench_test.go | 45 ++ pkg/ccl/backupccl/backup.go | 4 +- pkg/ccl/backupccl/restore.go | 45 +- pkg/ccl/backupccl/targets_test.go | 2 +- pkg/ccl/importccl/import_stmt.go | 8 +- pkg/ccl/partitionccl/partition_test.go | 6 +- pkg/cli/cli_test.go | 3 + pkg/cli/sql_util_test.go | 20 +- pkg/config/system_test.go | 17 +- pkg/keys/constants.go | 26 +- pkg/keys/printer.go | 3 + pkg/server/admin_test.go | 6 +- pkg/server/node.go | 2 +- pkg/server/node_test.go | 2 +- pkg/settings/cluster/cockroach_versions.go | 10 + pkg/settings/cluster/versionkey_string.go | 5 +- pkg/sql/authorization.go | 2 +- pkg/sql/backfill.go | 35 +- pkg/sql/conn_executor.go | 1 + pkg/sql/create_sequence.go | 6 +- pkg/sql/create_table.go | 31 +- pkg/sql/create_test.go | 9 +- pkg/sql/create_view.go | 4 +- pkg/sql/database.go | 31 +- pkg/sql/descriptor.go | 44 +- pkg/sql/distsql/server.go | 2 +- pkg/sql/drop_database.go | 12 +- pkg/sql/drop_table.go | 1 - pkg/sql/drop_test.go | 10 +- pkg/sql/event_log.go | 1 - pkg/sql/exec_util.go | 8 +- pkg/sql/execinfrapb/api.go | 19 +- pkg/sql/execinfrapb/api.pb.go | 232 +++++----- pkg/sql/execinfrapb/api.proto | 3 +- pkg/sql/internal.go | 5 + pkg/sql/lease.go | 8 +- pkg/sql/logical_schema_accessors.go | 17 +- .../testdata/logic_test/crdb_internal | 2 +- .../logictest/testdata/logic_test/grant_table | 40 +- .../testdata/logic_test/information_schema | 108 +++-- .../logictest/testdata/logic_test/pg_catalog | 4 + pkg/sql/logictest/testdata/logic_test/ranges | 9 +- .../logictest/testdata/logic_test/show_source | 6 +- pkg/sql/logictest/testdata/logic_test/system | 406 ++++++++---------- .../testdata/logic_test/system_namespace | 44 ++ .../logic_test/system_namespace_deprecated | 41 ++ .../logictest/testdata/logic_test/vectorize | 4 +- .../opt/exec/execbuilder/testdata/autocommit | 112 ++--- .../exec/execbuilder/testdata/delete_range | 9 +- .../opt/exec/execbuilder/testdata/show_trace | 24 +- pkg/sql/opt/exec/execbuilder/testdata/upsert | 6 +- pkg/sql/opt/optbuilder/create_table.go | 19 +- pkg/sql/opt/optbuilder/util.go | 13 + pkg/sql/opt_catalog.go | 5 +- pkg/sql/pgwire/pgwire_test.go | 2 +- pkg/sql/physical_schema_accessors.go | 146 +++++-- pkg/sql/planner.go | 11 + pkg/sql/rename_database.go | 4 +- pkg/sql/rename_table.go | 3 +- pkg/sql/resolver.go | 11 +- pkg/sql/schema_accessors.go | 7 +- pkg/sql/schema_changer.go | 6 +- pkg/sql/scrub.go | 2 +- pkg/sql/sem/builtins/builtins.go | 10 +- pkg/sql/sem/tree/function_name_test.go | 2 +- pkg/sql/sem/tree/name_resolution.go | 38 +- pkg/sql/sem/tree/name_resolution_test.go | 74 +++- pkg/sql/sessiondata/search_path.go | 150 ++++++- pkg/sql/sessiondata/search_path_test.go | 174 +++++++- pkg/sql/set_zone_config.go | 4 +- pkg/sql/sqlbase/constants.go | 2 +- pkg/sql/sqlbase/keys.go | 26 +- pkg/sql/sqlbase/keys_test.go | 8 +- pkg/sql/sqlbase/metadata.go | 50 ++- pkg/sql/sqlbase/namespace.go | 206 +++++++++ pkg/sql/sqlbase/structured.go | 89 +++- pkg/sql/sqlbase/structured.pb.go | 74 ++-- pkg/sql/sqlbase/structured.proto | 2 +- pkg/sql/sqlbase/system.go | 56 ++- pkg/sql/sqlbase/testutils.go | 3 +- pkg/sql/table.go | 16 +- pkg/sql/temporary_schema.go | 47 ++ pkg/sql/tests/system_table_test.go | 8 +- pkg/sql/truncate.go | 5 +- pkg/sql/vars.go | 6 +- pkg/sql/virtual_schema.go | 4 +- pkg/sql/zone_config.go | 9 +- pkg/sqlmigrations/migrations.go | 8 +- pkg/sqlmigrations/migrations_test.go | 2 +- pkg/storage/client_split_test.go | 3 +- pkg/storage/client_test.go | 6 +- pkg/storage/store_test.go | 6 +- .../localtestcluster/local_test_cluster.go | 3 +- 94 files changed, 2022 insertions(+), 800 deletions(-) create mode 100644 pkg/sql/logictest/testdata/logic_test/system_namespace create mode 100644 pkg/sql/logictest/testdata/logic_test/system_namespace_deprecated create mode 100644 pkg/sql/sqlbase/namespace.go create mode 100644 pkg/sql/temporary_schema.go diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index fa71a8d55eb7..f94663be7abf 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -60,6 +60,6 @@ trace.debug.enablebooleanfalseif set, traces for recent requests can be seen in the /debug page trace.lightstep.tokenstringif set, traces go to Lightstep using this token trace.zipkin.collectorstringif set, traces go to the given Zipkin instance (example: '127.0.0.1:9411'); ignored if trace.lightstep.token is set -versioncustom validation19.2-4set the active cluster version in the format '.' +versioncustom validation19.2-5set the active cluster version in the format '.' diff --git a/pkg/bench/bench_test.go b/pkg/bench/bench_test.go index 7256029c5ba4..6ef7cc7ad617 100644 --- a/pkg/bench/bench_test.go +++ b/pkg/bench/bench_test.go @@ -1090,3 +1090,48 @@ func BenchmarkSortJoinAggregation(b *testing.B) { } }) } + +func BenchmarkNameResolution(b *testing.B) { + if testing.Short() { + b.Skip("short flag") + } + defer log.Scope(b).Close(b) + ForEachDB(b, func(b *testing.B, db *sqlutils.SQLRunner) { + db.Exec(b, `CREATE TABLE namespace (k INT PRIMARY KEY, v INT)`) + db.Exec(b, `INSERT INTO namespace VALUES(1, 2)`) + + b.ResetTimer() + + for i := 0; i < b.N; i++ { + db.Exec(b, "SELECT * FROM namespace") + } + b.StopTimer() + }) +} + +// When temporary tables are present in the system, the PG search path semantics +// dictate that we try searching for tables under the temporary schema before +// the public physical schema. This test is used to microbenchmark the effects +// of this scenario. +func BenchmarkNameResolutionTempTablesExist(b *testing.B) { + if testing.Short() { + b.Skip("short flag") + } + defer log.Scope(b).Close(b) + ForEachDB(b, func(b *testing.B, db *sqlutils.SQLRunner) { + db.Exec(b, `CREATE TABLE namespace (k INT PRIMARY KEY, v INT)`) + db.Exec(b, `INSERT INTO namespace VALUES(1, 2)`) + + b.ResetTimer() + + for i := 0; i < b.N; i++ { + b.StopTimer() + // Setup + db.Exec(b, `SET experimental_enable_temp_tables = true`) + db.Exec(b, `CREATE TEMP TABLE IF NOT EXISTS temp_table(k INT PRIMARY KEY, v INT)`) + b.StartTimer() + db.Exec(b, "SELECT * FROM namespace") + } + b.StopTimer() + }) +} diff --git a/pkg/ccl/backupccl/backup.go b/pkg/ccl/backupccl/backup.go index 8fc5ee7fff13..68a9b1c6d3c0 100644 --- a/pkg/ccl/backupccl/backup.go +++ b/pkg/ccl/backupccl/backup.go @@ -1366,7 +1366,9 @@ func (b *backupResumer) Resume( } // OnFailOrCancel is part of the jobs.Resumer interface. -func (b *backupResumer) OnFailOrCancel(context.Context, *client.Txn) error { return nil } +func (b *backupResumer) OnFailOrCancel(context.Context, *client.Txn) error { + return nil +} // OnSuccess is part of the jobs.Resumer interface. func (b *backupResumer) OnSuccess(context.Context, *client.Txn) error { return nil } diff --git a/pkg/ccl/backupccl/restore.go b/pkg/ccl/backupccl/restore.go index 2b68f7e725a8..a83670fb2f92 100644 --- a/pkg/ccl/backupccl/restore.go +++ b/pkg/ccl/backupccl/restore.go @@ -362,12 +362,11 @@ func allocateTableRewrites( if err := p.ExecCfg().DB.Txn(ctx, func(ctx context.Context, txn *client.Txn) error { // Check that any DBs being restored do _not_ exist. for name := range restoreDBNames { - dKey := sqlbase.NewDatabaseKey(name) - existingDatabaseID, err := txn.Get(ctx, dKey.Key()) + found, _, err := sqlbase.LookupDatabaseID(ctx, txn, name) if err != nil { return err } - if existingDatabaseID.Value != nil { + if found { return errors.Errorf("database %q already exists", name) } } @@ -390,21 +389,15 @@ func allocateTableRewrites( } else { var parentID sqlbase.ID { - dKey := sqlbase.NewDatabaseKey(targetDB) - existingDatabaseID, err := txn.Get(ctx, dKey.Key()) + found, newParentID, err := sqlbase.LookupDatabaseID(ctx, txn, targetDB) if err != nil { return err } - if existingDatabaseID.Value == nil { + if !found { return errors.Errorf("a database named %q needs to exist to restore table %q", targetDB, table.Name) } - - newParentID, err := existingDatabaseID.Value.GetInt() - if err != nil { - return err - } - parentID = sqlbase.ID(newParentID) + parentID = newParentID } // Check that the table name is _not_ in use. @@ -481,12 +474,11 @@ func allocateTableRewrites( func CheckTableExists( ctx context.Context, txn *client.Txn, parentID sqlbase.ID, name string, ) error { - tKey := sqlbase.NewTableKey(parentID, name) - res, err := txn.Get(ctx, tKey.Key()) + found, _, err := sqlbase.LookupPublicTableID(ctx, txn, parentID, name) if err != nil { return err } - if res.Exists() { + if found { return sqlbase.NewRelationAlreadyExistsError(name) } return nil @@ -1036,7 +1028,11 @@ func WriteTableDescs( if err := sql.WriteNewDescToBatch(ctx, false /* kvTrace */, settings, b, desc.ID, desc); err != nil { return err } - b.CPut(sqlbase.NewDatabaseKey(desc.Name).Key(), desc.ID, nil) + // Depending on which cluster version we are restoring to, we decide which + // namespace table to write the descriptor into. This may cause wrong + // behavior if the cluster version is bumped DURING a restore. + dKey := sqlbase.MakeDatabaseNameKey(ctx, settings, desc.Name) + b.CPut(dKey.Key(), desc.ID, nil) } for i := range tables { if wrote, ok := wroteDBs[tables[i].ParentID]; ok { @@ -1058,7 +1054,11 @@ func WriteTableDescs( if err := sql.WriteNewDescToBatch(ctx, false /* kvTrace */, settings, b, tables[i].ID, tables[i]); err != nil { return err } - b.CPut(sqlbase.NewTableKey(tables[i].ParentID, tables[i].Name).Key(), tables[i].ID, nil) + // Depending on which cluster version we are restoring to, we decide which + // namespace table to write the descriptor into. This may cause wrong + // behavior if the cluster version is bumped DURING a restore. + tkey := sqlbase.MakePublicTableNameKey(ctx, settings, tables[i].ParentID, tables[i].Name) + b.CPut(tkey.Key(), tables[i].ID, nil) } for _, kv := range extra { b.InitPut(kv.Key, &kv.Value, false) @@ -1749,13 +1749,10 @@ func (r *restoreResumer) OnFailOrCancel(ctx context.Context, txn *client.Txn) er tableDesc := *tbl tableDesc.Version++ tableDesc.State = sqlbase.TableDescriptor_DROP - var existingIDVal roachpb.Value - existingIDVal.SetInt(int64(tableDesc.ID)) - b.CPut( - sqlbase.NewTableKey(tableDesc.ParentID, tableDesc.Name).Key(), - nil, - &existingIDVal, - ) + err := sqlbase.RemovePublicTableNamespaceEntry(ctx, txn, tbl.ParentID, tbl.Name) + if err != nil { + return nil + } existingDescVal, err := sqlbase.ConditionalGetTableDescFromTxn(ctx, txn, tbl) if err != nil { return errors.Wrap(err, "dropping tables") diff --git a/pkg/ccl/backupccl/targets_test.go b/pkg/ccl/backupccl/targets_test.go index 57f14a17d6f6..aa1f7756be02 100644 --- a/pkg/ccl/backupccl/targets_test.go +++ b/pkg/ccl/backupccl/targets_test.go @@ -108,7 +108,7 @@ func TestDescriptorsMatchingTargets(t *testing.T) { {"", `TABLE system.offline`, []string{"system", "foo"}, nil, `table "system.public.offline" does not exist`}, {"system", `TABLE *`, []string{"system", "foo", "bar"}, nil, ``}, } - searchPath := sessiondata.MakeSearchPath([]string{"public", "pg_catalog"}) + searchPath := sessiondata.MakeSearchPath([]string{"public", "pg_catalog"}, sessiondata.DefaultTemporarySchemaName) for i, test := range tests { t.Run(fmt.Sprintf("%d/%s/%s", i, test.sessionDatabase, test.pattern), func(t *testing.T) { sql := fmt.Sprintf(`GRANT ALL ON %s TO ignored`, test.pattern) diff --git a/pkg/ccl/importccl/import_stmt.go b/pkg/ccl/importccl/import_stmt.go index d84d7cdddb54..14b1dd021720 100644 --- a/pkg/ccl/importccl/import_stmt.go +++ b/pkg/ccl/importccl/import_stmt.go @@ -922,10 +922,10 @@ func (r *importResumer) OnFailOrCancel(ctx context.Context, txn *client.Txn) err // possible. This is safe since the table data was never visible to users, // and so we don't need to preserve MVCC semantics. tableDesc.DropTime = 1 - var existingIDVal roachpb.Value - existingIDVal.SetInt(int64(tableDesc.ID)) - tKey := sqlbase.NewTableKey(tableDesc.ParentID, tableDesc.Name) - b.CPut(tKey.Key(), nil, &existingIDVal) + err := sqlbase.RemovePublicTableNamespaceEntry(ctx, txn, tableDesc.ParentID, tableDesc.Name) + if err != nil { + return err + } } else { // IMPORT did not create this table, so we should not drop it. tableDesc.State = sqlbase.TableDescriptor_PUBLIC diff --git a/pkg/ccl/partitionccl/partition_test.go b/pkg/ccl/partitionccl/partition_test.go index 3931d86c8d45..fa13d5205ea8 100644 --- a/pkg/ccl/partitionccl/partition_test.go +++ b/pkg/ccl/partitionccl/partition_test.go @@ -1095,10 +1095,14 @@ func verifyScansOnNode( } traceLines = append(traceLines, traceLine.String) if strings.Contains(traceLine.String, "read completed") { - if strings.Contains(traceLine.String, "SystemCon") { + if strings.Contains(traceLine.String, "SystemCon") || strings.Contains(traceLine.String, "NamespaceTab") { // Ignore trace lines for the system config range (abbreviated as // "SystemCon" in pretty printing of the range descriptor). A read might // be performed to the system config range to update the table lease. + // + // Also ignore trace lines for the system.namespace table, which is a + // system table that resides outside the system config range. (abbreviated + // as "NamespaceTab" in pretty printing of the range descriptor). continue } if !strings.Contains(traceLine.String, node) { diff --git a/pkg/cli/cli_test.go b/pkg/cli/cli_test.go index fb50abb6a536..9e9b1d171972 100644 --- a/pkg/cli/cli_test.go +++ b/pkg/cli/cli_test.go @@ -2229,6 +2229,8 @@ writing ` + os.DevNull + ` debug/nodes/1/ranges/22.json debug/nodes/1/ranges/23.json debug/nodes/1/ranges/24.json + debug/nodes/1/ranges/25.json + debug/nodes/1/ranges/26.json debug/schema/defaultdb@details.json debug/schema/postgres@details.json debug/schema/system@details.json @@ -2239,6 +2241,7 @@ writing ` + os.DevNull + ` debug/schema/system/lease.json debug/schema/system/locations.json debug/schema/system/namespace.json + debug/schema/system/namespace_deprecated.json debug/schema/system/rangelog.json debug/schema/system/replication_constraint_stats.json debug/schema/system/replication_critical_localities.json diff --git a/pkg/cli/sql_util_test.go b/pkg/cli/sql_util_test.go index 36eda83b3a32..6e1ff6a589a7 100644 --- a/pkg/cli/sql_util_test.go +++ b/pkg/cli/sql_util_test.go @@ -150,6 +150,7 @@ SET expectedRows := [][]string{ {`parentID`, `INT8`, `false`, `NULL`, ``, `{primary}`, `false`}, + {`parentSchemaID`, `INT8`, `false`, `NULL`, ``, `{primary}`, `false`}, {`name`, `STRING`, `false`, `NULL`, ``, `{primary}`, `false`}, {`id`, `INT8`, `true`, `NULL`, ``, `{}`, `false`}, } @@ -163,12 +164,13 @@ SET } expected = ` - column_name | data_type | is_nullable | column_default | generation_expression | indices | is_hidden -+-------------+-----------+-------------+----------------+-----------------------+-----------+-----------+ - parentID | INT8 | false | NULL | | {primary} | false - name | STRING | false | NULL | | {primary} | false - id | INT8 | true | NULL | | {} | false -(3 rows) + column_name | data_type | is_nullable | column_default | generation_expression | indices | is_hidden ++----------------+-----------+-------------+----------------+-----------------------+-----------+-----------+ + parentID | INT8 | false | NULL | | {primary} | false + parentSchemaID | INT8 | false | NULL | | {primary} | false + name | STRING | false | NULL | | {primary} | false + id | INT8 | true | NULL | | {} | false +(4 rows) ` if a, e := b.String(), expected[1:]; a != e { @@ -183,9 +185,9 @@ SET } expected = ` - parentID | name | id -+----------+------------+----+ - 1 | descriptor | 3 + parentID | parentSchemaID | name | id ++----------+----------------+------------+----+ + 1 | 29 | descriptor | 3 (1 row) ` if a, e := b.String(), expected[1:]; a != e { diff --git a/pkg/config/system_test.go b/pkg/config/system_test.go index 0cbde1a404c2..f7b25c90971d 100644 --- a/pkg/config/system_test.go +++ b/pkg/config/system_test.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/encoding" @@ -163,7 +164,7 @@ func TestGetLargestID(t *testing.T) { ms := sqlbase.MakeMetadataSchema(config.DefaultZoneConfigRef(), config.DefaultSystemZoneConfigRef()) descIDs := ms.DescriptorIDs() maxDescID := descIDs[len(descIDs)-1] - kvs, _ /* splits */ := ms.GetInitialValues() + kvs, _ /* splits */ := ms.GetInitialValues(cluster.TestingClusterVersion) return testCase{kvs, uint32(maxDescID), 0, ""} }(), @@ -256,7 +257,7 @@ func TestComputeSplitKeySystemRanges(t *testing.T) { } cfg := config.NewSystemConfig(config.DefaultZoneConfigRef()) - kvs, _ /* splits */ := sqlbase.MakeMetadataSchema(cfg.DefaultZoneConfig, config.DefaultSystemZoneConfigRef()).GetInitialValues() + kvs, _ /* splits */ := sqlbase.MakeMetadataSchema(cfg.DefaultZoneConfig, config.DefaultSystemZoneConfigRef()).GetInitialValues(cluster.TestingClusterVersion) cfg.SystemConfigEntries = config.SystemConfigEntries{ Values: kvs, } @@ -288,12 +289,14 @@ func TestComputeSplitKeyTableIDs(t *testing.T) { schema := sqlbase.MakeMetadataSchema(config.DefaultZoneConfigRef(), config.DefaultSystemZoneConfigRef()) // Real system tables only. - baseSql, _ /* splits */ := schema.GetInitialValues() + baseSql, _ /* splits */ := schema.GetInitialValues(cluster.TestingClusterVersion) // Real system tables plus some user stuff. - kvs, _ /* splits */ := schema.GetInitialValues() + kvs, _ /* splits */ := schema.GetInitialValues(cluster.TestingClusterVersion) userSQL := append(kvs, descriptor(start), descriptor(start+1), descriptor(start+5)) // Real system tables and partitioned user tables. - subzoneSQL := append(userSQL, + var subzoneSQL = make([]roachpb.KeyValue, len(userSQL)) + copy(subzoneSQL, userSQL) + subzoneSQL = append(subzoneSQL, zoneConfig(start+1, subzone("a", ""), subzone("c", "e")), zoneConfig(start+5, subzone("b", ""), subzone("c", "d"), subzone("d", ""))) @@ -404,13 +407,13 @@ func TestGetZoneConfigForKey(t *testing.T) { {roachpb.RKey(keys.SystemConfigSplitKey), keys.SystemDatabaseID}, // Gossiped system tables should refer to the SystemDatabaseID. - {tkey(keys.NamespaceTableID), keys.SystemDatabaseID}, {tkey(keys.ZonesTableID), keys.SystemDatabaseID}, // Non-gossiped system tables should refer to themselves. {tkey(keys.LeaseTableID), keys.LeaseTableID}, {tkey(keys.JobsTableID), keys.JobsTableID}, {tkey(keys.LocationsTableID), keys.LocationsTableID}, + {tkey(keys.NamespaceTableID), keys.NamespaceTableID}, // Pseudo-tables should refer to the SystemDatabaseID. {tkey(keys.MetaRangesID), keys.SystemDatabaseID}, @@ -430,7 +433,7 @@ func TestGetZoneConfigForKey(t *testing.T) { }() cfg := config.NewSystemConfig(config.DefaultZoneConfigRef()) - kvs, _ /* splits */ := sqlbase.MakeMetadataSchema(cfg.DefaultZoneConfig, config.DefaultSystemZoneConfigRef()).GetInitialValues() + kvs, _ /* splits */ := sqlbase.MakeMetadataSchema(cfg.DefaultZoneConfig, config.DefaultSystemZoneConfigRef()).GetInitialValues(cluster.TestingClusterVersion) cfg.SystemConfigEntries = config.SystemConfigEntries{ Values: kvs, } diff --git a/pkg/keys/constants.go b/pkg/keys/constants.go index 5712100a20c5..b62236f02bce 100644 --- a/pkg/keys/constants.go +++ b/pkg/keys/constants.go @@ -265,6 +265,12 @@ var ( MaxKey = roachpb.KeyMax // MinKey is a minimum key value which sorts before all other keys. MinKey = roachpb.KeyMin + + // NamespaceTableMin is the start key of system.namespace, which is a system + // table that does not reside in the same range as other system tables. + NamespaceTableMin = roachpb.Key(MakeTablePrefix(NamespaceTableID)) + // NamespaceTableMax is the end key of system.namespace. + NamespaceTableMax = roachpb.Key(MakeTablePrefix(NamespaceTableID + 1)) ) // Various IDs used by the structured data layer. @@ -295,12 +301,14 @@ const ( // SystemDatabaseID and following are the database/table IDs for objects // in the system span. // NOTE: IDs must be <= MaxSystemConfigDescID. - SystemDatabaseID = 1 - NamespaceTableID = 2 - DescriptorTableID = 3 - UsersTableID = 4 - ZonesTableID = 5 - SettingsTableID = 6 + SystemDatabaseID = 1 + // DeprecatedNamespaceTableID was the tableID for the system.namespace table + // for pre-20.1 clusters. + DeprecatedNamespaceTableID = 2 + DescriptorTableID = 3 + UsersTableID = 4 + ZonesTableID = 5 + SettingsTableID = 6 // IDs for the important columns and indexes in the zones table live here to // avoid introducing a dependency on sql/sqlbase throughout the codebase. @@ -330,6 +338,10 @@ const ( ReplicationCriticalLocalitiesTableID = 26 ReplicationStatsTableID = 27 ReportsMetaTableID = 28 + PublicSchemaID = 29 + // New NamespaceTableID for cluster version >= 20.1 + // Ensures that NamespaceTable does not get gossiped again + NamespaceTableID = 30 // CommentType is type for system.comments DatabaseCommentType = 0 @@ -342,4 +354,4 @@ const ( // there's no table descriptor). They're grouped here because the cluster // bootstrap process needs to create splits for them; splits for the tables // happen separately. -var PseudoTableIDs = []uint32{MetaRangesID, SystemRangesID, TimeseriesRangesID, LivenessRangesID} +var PseudoTableIDs = []uint32{MetaRangesID, SystemRangesID, TimeseriesRangesID, LivenessRangesID, PublicSchemaID} diff --git a/pkg/keys/printer.go b/pkg/keys/printer.go index 3e1d9e36f2b5..c5c1170e84fe 100644 --- a/pkg/keys/printer.go +++ b/pkg/keys/printer.go @@ -126,6 +126,9 @@ var ( PSFunc: parseUnsupported, }, }}, + {Name: "/NamespaceTable", start: NamespaceTableMin, end: NamespaceTableMax, Entries: []DictEntry{ + {Name: "", prefix: nil, ppFunc: decodeKeyPrint, PSFunc: parseUnsupported}, + }}, {Name: "/Table", start: TableDataMin, end: TableDataMax, Entries: []DictEntry{ {Name: "", prefix: nil, ppFunc: decodeKeyPrint, PSFunc: tableKeyParse}, }}, diff --git a/pkg/server/admin_test.go b/pkg/server/admin_test.go index 32b829f4e293..33cd9bc6dcf3 100644 --- a/pkg/server/admin_test.go +++ b/pkg/server/admin_test.go @@ -386,7 +386,7 @@ func TestAdminAPINonTableStats(t *testing.T) { // with no user data, all the ranges on the Databases page consist of: // 1) the total ranges listed for the system database // 2) the total ranges listed for the Non-Table data -func TestRangeCount_MissingOneRange(t *testing.T) { +func TestRangeCount_MissingTwoRanges(t *testing.T) { defer leaktest.AfterTest(t)() testCluster := serverutils.StartTestCluster(t, 3, base.TestClusterArgs{}) defer testCluster.Stopper().Stop(context.Background()) @@ -444,7 +444,9 @@ func TestRangeCount_MissingOneRange(t *testing.T) { } // TODO(celia): We're missing 1 range -- where is it? - expectedMissingRangeCount := int64(1) + // TODO(arul): We're missing 2 ranges after moving system.namespace out from + // the gossip range -- where are they? + expectedMissingRangeCount := int64(2) assert.Equal(t, totalRangeCount, nonTableRangeCount+systemTableRangeCount+expectedMissingRangeCount) diff --git a/pkg/server/node.go b/pkg/server/node.go index 09ac1b42bf33..f62cec4e43f8 100644 --- a/pkg/server/node.go +++ b/pkg/server/node.go @@ -238,7 +238,7 @@ func bootstrapCluster( // first store. if i == 0 { schema := GetBootstrapSchema(defaultZoneConfig, defaultSystemZoneConfig) - initialValues, tableSplits := schema.GetInitialValues() + initialValues, tableSplits := schema.GetInitialValues(bootstrapVersion) splits := append(config.StaticSplits(), tableSplits...) sort.Slice(splits, func(i, j int) bool { return splits[i].Less(splits[j]) diff --git a/pkg/server/node_test.go b/pkg/server/node_test.go index 20f6e5a52b9b..2515832b415d 100644 --- a/pkg/server/node_test.go +++ b/pkg/server/node_test.go @@ -237,7 +237,7 @@ func TestBootstrapCluster(t *testing.T) { } // Add the initial keys for sql. - kvs, tableSplits := GetBootstrapSchema(config.DefaultZoneConfigRef(), config.DefaultSystemZoneConfigRef()).GetInitialValues() + kvs, tableSplits := GetBootstrapSchema(config.DefaultZoneConfigRef(), config.DefaultSystemZoneConfigRef()).GetInitialValues(cluster.TestingClusterVersion) for _, kv := range kvs { expectedKeys = append(expectedKeys, kv.Key) } diff --git a/pkg/settings/cluster/cockroach_versions.go b/pkg/settings/cluster/cockroach_versions.go index 927cbabe8e60..18e80fd9dfdd 100644 --- a/pkg/settings/cluster/cockroach_versions.go +++ b/pkg/settings/cluster/cockroach_versions.go @@ -49,6 +49,7 @@ const ( VersionContainsEstimatesCounter VersionChangeReplicasDemotion VersionSecondaryIndexColumnFamilies + VersionNamespaceTableWithSchemas // Add new versions here (step one of two). @@ -328,6 +329,15 @@ var versionsSingleton = keyedVersions([]keyedVersion{ Key: VersionSecondaryIndexColumnFamilies, Version: roachpb.Version{Major: 19, Minor: 2, Unstable: 4}, }, + { + // VersionNamespaceTableWithSchemas is https://github.com/cockroachdb/cockroach/pull/41977 + // + // It represents the migration to a new system.namespace table that has an + // added parentSchemaID column. In addition to the new column, the table is + // no longer in the system config range -- implying it is no longer gossiped. + Key: VersionNamespaceTableWithSchemas, + Version: roachpb.Version{Major: 19, Minor: 2, Unstable: 5}, + }, // Add new versions here (step two of two). diff --git a/pkg/settings/cluster/versionkey_string.go b/pkg/settings/cluster/versionkey_string.go index 58dab3273c5b..003154c8b191 100644 --- a/pkg/settings/cluster/versionkey_string.go +++ b/pkg/settings/cluster/versionkey_string.go @@ -25,11 +25,12 @@ func _() { _ = x[VersionContainsEstimatesCounter-14] _ = x[VersionChangeReplicasDemotion-15] _ = x[VersionSecondaryIndexColumnFamilies-16] + _ = x[VersionNamespaceTableWithSchemas-17] } -const _VersionKey_name = "Version19_1VersionStart19_2VersionQueryTxnTimestampVersionStickyBitVersionParallelCommitsVersionGenerationComparableVersionLearnerReplicasVersionTopLevelForeignKeysVersionAtomicChangeReplicasTriggerVersionAtomicChangeReplicasVersionTableDescModificationTimeFromMVCCVersionPartitionedBackupVersion19_2VersionStart20_1VersionContainsEstimatesCounterVersionChangeReplicasDemotionVersionSecondaryIndexColumnFamilies" +const _VersionKey_name = "Version19_1VersionStart19_2VersionQueryTxnTimestampVersionStickyBitVersionParallelCommitsVersionGenerationComparableVersionLearnerReplicasVersionTopLevelForeignKeysVersionAtomicChangeReplicasTriggerVersionAtomicChangeReplicasVersionTableDescModificationTimeFromMVCCVersionPartitionedBackupVersion19_2VersionStart20_1VersionContainsEstimatesCounterVersionChangeReplicasDemotionVersionSecondaryIndexColumnFamiliesVersionNamespaceTableWithSchemas" -var _VersionKey_index = [...]uint16{0, 11, 27, 51, 67, 89, 116, 138, 164, 198, 225, 265, 289, 300, 316, 347, 376, 411} +var _VersionKey_index = [...]uint16{0, 11, 27, 51, 67, 89, 116, 138, 164, 198, 225, 265, 289, 300, 316, 347, 376, 411, 443} func (i VersionKey) String() string { if i < 0 || i >= VersionKey(len(_VersionKey_index)-1) { diff --git a/pkg/sql/authorization.go b/pkg/sql/authorization.go index d303e3580d23..7ff7add6bc3a 100644 --- a/pkg/sql/authorization.go +++ b/pkg/sql/authorization.go @@ -196,7 +196,7 @@ func (p *planner) MemberOfWithAdminOption( ctx context.Context, member string, ) (map[string]bool, error) { // Lookup table version. - objDesc, err := p.PhysicalSchemaAccessor().GetObjectDesc(ctx, p.txn, &roleMembersTableName, + objDesc, err := p.PhysicalSchemaAccessor().GetObjectDesc(ctx, p.txn, p.ExecCfg().Settings, &roleMembersTableName, p.ObjectLookupFlags(true /*required*/, false /*requireMutable*/)) if err != nil { return nil, err diff --git a/pkg/sql/backfill.go b/pkg/sql/backfill.go index 82fdeb4697ab..55ed96facd5d 100644 --- a/pkg/sql/backfill.go +++ b/pkg/sql/backfill.go @@ -600,7 +600,10 @@ func (sc *SchemaChanger) truncateIndexes( defer fn() } - tc := &TableCollection{leaseMgr: sc.leaseMgr} + tc := &TableCollection{ + leaseMgr: sc.leaseMgr, + settings: sc.settings, + } defer tc.releaseTables(ctx) tableDesc, err := sc.getTableVersion(ctx, txn, tc, version) if err != nil { @@ -811,7 +814,10 @@ func (sc *SchemaChanger) distBackfill( } } - tc := &TableCollection{leaseMgr: sc.leaseMgr} + tc := &TableCollection{ + leaseMgr: sc.leaseMgr, + settings: sc.settings, + } // Use a leased table descriptor for the backfill. defer tc.releaseTables(ctx) tableDesc, err := sc.getTableVersion(ctx, txn, tc, version) @@ -1183,7 +1189,10 @@ func (sc *SchemaChanger) validateForwardIndexes( if err != nil { return err } - tc := &TableCollection{leaseMgr: sc.leaseMgr} + tc := &TableCollection{ + leaseMgr: sc.leaseMgr, + settings: sc.settings, + } // pretend that the schema has been modified. if err := tc.addUncommittedTable(*desc); err != nil { return err @@ -1301,15 +1310,13 @@ func runSchemaChangesInTxn( if len(tableDesc.DrainingNames) > 0 { // Reclaim all the old names. Leave the data and descriptor // cleanup for later. - b := planner.Txn().NewBatch() for _, drain := range tableDesc.DrainingNames { - tbKey := sqlbase.NewTableKey(drain.ParentID, drain.Name).Key() - b.Del(tbKey) + err := sqlbase.RemovePublicTableNamespaceEntry(ctx, planner.Txn(), drain.ParentID, drain.Name) + if err != nil { + return err + } } tableDesc.DrainingNames = nil - if err := planner.Txn().Run(ctx, b); err != nil { - return err - } } if tableDesc.Dropped() { @@ -1490,7 +1497,10 @@ func validateCheckInTxn( ) error { ie := evalCtx.InternalExecutor.(*SessionBoundInternalExecutor) if tableDesc.Version > tableDesc.ClusterVersion.Version { - newTc := &TableCollection{leaseMgr: leaseMgr} + newTc := &TableCollection{ + leaseMgr: leaseMgr, + settings: evalCtx.Settings, + } // pretend that the schema has been modified. if err := newTc.addUncommittedTable(*tableDesc); err != nil { return err @@ -1527,7 +1537,10 @@ func validateFkInTxn( ) error { ie := evalCtx.InternalExecutor.(*SessionBoundInternalExecutor) if tableDesc.Version > tableDesc.ClusterVersion.Version { - newTc := &TableCollection{leaseMgr: leaseMgr} + newTc := &TableCollection{ + leaseMgr: leaseMgr, + settings: evalCtx.Settings, + } // pretend that the schema has been modified. if err := newTc.addUncommittedTable(*tableDesc); err != nil { return err diff --git a/pkg/sql/conn_executor.go b/pkg/sql/conn_executor.go index b80426acb253..e628b78c6d39 100644 --- a/pkg/sql/conn_executor.go +++ b/pkg/sql/conn_executor.go @@ -590,6 +590,7 @@ func (s *Server) newConnExecutor( leaseMgr: s.cfg.LeaseManager, databaseCache: s.dbCache.getDatabaseCache(), dbCacheSubscriber: s.dbCache, + settings: s.cfg.Settings, } ex.extraTxnState.txnRewindPos = -1 ex.mu.ActiveQueries = make(map[ClusterWideID]*queryMeta) diff --git a/pkg/sql/create_sequence.go b/pkg/sql/create_sequence.go index 67d03ea0f3ff..1a69667d7179 100644 --- a/pkg/sql/create_sequence.go +++ b/pkg/sql/create_sequence.go @@ -50,7 +50,8 @@ func (n *createSequenceNode) startExec(params runParams) error { return unimplemented.NewWithIssuef(5807, "temporary sequences are unsupported") } - tKey := sqlbase.NewTableKey(n.dbDesc.ID, n.n.Name.Table()) + tKey := sqlbase.MakePublicTableNameKey(params.ctx, params.ExecCfg().Settings, n.dbDesc.ID, n.n.Name.Table()) + if exists, err := descExists(params.ctx, params.p.txn, tKey.Key()); err == nil && exists { if n.n.IfNotExists { // If the sequence exists but the user specified IF NOT EXISTS, return without doing anything. @@ -90,7 +91,8 @@ func doCreateSequence( // makeSequenceTableDesc already validates the table. No call to // desc.ValidateTable() needed here. - key := sqlbase.NewTableKey(dbDesc.ID, name.Table()).Key() + key := sqlbase.MakePublicTableNameKey(params.ctx, params.ExecCfg().Settings, + dbDesc.ID, name.Table()).Key() if err = params.p.createDescriptorWithID(params.ctx, key, id, &desc, params.EvalContext().Settings); err != nil { return err } diff --git a/pkg/sql/create_table.go b/pkg/sql/create_table.go index 2abf3ec9a25e..3665e78947e2 100644 --- a/pkg/sql/create_table.go +++ b/pkg/sql/create_table.go @@ -62,15 +62,34 @@ type createTableRun struct { } func (n *createTableNode) startExec(params runParams) error { - temporary := false - if n.n.Temporary { + isTemporary := n.n.Temporary + + tKey := sqlbase.MakePublicTableNameKey(params.ctx, + params.ExecCfg().Settings, n.dbDesc.ID, n.n.Table.Table()) + + // If a user specifies the pg_temp schema, even without the TEMPORARY keyword, + // a temporary table should be created. + if isTemporary { if !params.SessionData().TempTablesEnabled { return unimplemented.NewWithIssuef(5807, "temporary tables are unsupported") } - temporary = true + + tempSchemaName := params.p.TemporarySchemaName() + sKey := sqlbase.NewSchemaKey(n.dbDesc.ID, tempSchemaName) + schemaID, err := getDescriptorID(params.ctx, params.p.txn, sKey) + if err != nil { + return err + } else if schemaID == sqlbase.InvalidID { + // The temporary schema has not been created yet. + // TODO(arul): Add a job that does deletion for this session(temp schema) + if schemaID, err = createTempSchema(params, sKey); err != nil { + return err + } + } + + tKey = sqlbase.NewTableKey(n.dbDesc.ID, schemaID, n.n.Table.Table()) } - tKey := sqlbase.NewTableKey(n.dbDesc.ID, n.n.Table.Table()) key := tKey.Key() if exists, err := descExists(params.ctx, params.p.txn, key); err == nil && exists { if n.n.IfNotExists { @@ -120,7 +139,7 @@ func (n *createTableNode) startExec(params runParams) error { } desc, err = makeTableDescIfAs(params, - n.n, n.dbDesc.ID, id, creationTime, asCols, privs, params.p.EvalContext(), temporary) + n.n, n.dbDesc.ID, id, creationTime, asCols, privs, params.p.EvalContext(), isTemporary) if err != nil { return err } @@ -132,7 +151,7 @@ func (n *createTableNode) startExec(params runParams) error { } } else { affected = make(map[sqlbase.ID]*sqlbase.MutableTableDescriptor) - desc, err = makeTableDesc(params, n.n, n.dbDesc.ID, id, creationTime, privs, affected, temporary) + desc, err = makeTableDesc(params, n.n, n.dbDesc.ID, id, creationTime, privs, affected, isTemporary) if err != nil { return err } diff --git a/pkg/sql/create_test.go b/pkg/sql/create_test.go index 614d1a4edde1..70383d0b39f9 100644 --- a/pkg/sql/create_test.go +++ b/pkg/sql/create_test.go @@ -40,7 +40,6 @@ func TestDatabaseDescriptor(t *testing.T) { s, sqlDB, kvDB := serverutils.StartServer(t, params) defer s.Stopper().Stop(context.TODO()) ctx := context.TODO() - expectedCounter := int64(keys.MinNonPredefinedUserDescID) // Test values before creating the database. @@ -97,7 +96,13 @@ func TestDatabaseDescriptor(t *testing.T) { if err != nil { t.Fatal(err) } - if e, a := len(descriptorIDs), len(kvs); a != e { + // TODO(arul): Revert this back to to len(descriptorIDs) once the migration + // to the new system.namespace is done. + // Every database is initialized with a public schema, which does not have + // a descriptor associated with it. There are 3 databases: defaultdb, + // system, and postgres. + e := len(descriptorIDs) + 3 + if a := len(kvs); a != e { t.Fatalf("expected %d keys to have been written, found %d keys", e, a) } } diff --git a/pkg/sql/create_view.go b/pkg/sql/create_view.go index 5c494dfbf459..889840886342 100644 --- a/pkg/sql/create_view.go +++ b/pkg/sql/create_view.go @@ -46,7 +46,9 @@ func (n *createViewNode) startExec(params runParams) error { viewName := string(n.viewName) log.VEventf(params.ctx, 2, "dependencies for view %s:\n%s", viewName, n.planDeps.String()) - tKey := sqlbase.NewTableKey(n.dbDesc.ID, viewName) + tKey := sqlbase.MakePublicTableNameKey(params.ctx, + params.ExecCfg().Settings, n.dbDesc.ID, viewName) + key := tKey.Key() if exists, err := descExists(params.ctx, params.p.txn, key); err == nil && exists { // TODO(a-robinson): Support CREATE OR REPLACE commands. diff --git a/pkg/sql/database.go b/pkg/sql/database.go index 3a56f3655908..212abdb56c2d 100644 --- a/pkg/sql/database.go +++ b/pkg/sql/database.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/internal/client" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -72,17 +73,6 @@ func makeDatabaseDesc(p *tree.CreateDatabase) sqlbase.DatabaseDescriptor { } } -// getKeysForDatabaseDescriptor retrieves the KV keys corresponding to -// the zone, name and descriptor of a database. -func getKeysForDatabaseDescriptor( - dbDesc *sqlbase.DatabaseDescriptor, -) (zoneKey roachpb.Key, nameKey roachpb.Key, descKey roachpb.Key) { - zoneKey = config.MakeZoneKey(uint32(dbDesc.ID)) - nameKey = sqlbase.NewDatabaseKey(dbDesc.GetName()).Key() - descKey = sqlbase.MakeDescMetadataKey(dbDesc.ID) - return -} - // getDatabaseID resolves a database name into a database ID. // Returns InvalidID on failure. func getDatabaseID( @@ -91,11 +81,11 @@ func getDatabaseID( if name == sqlbase.SystemDB.Name { return sqlbase.SystemDB.ID, nil } - dbID, err := getDescriptorID(ctx, txn, sqlbase.NewDatabaseKey(name)) + found, dbID, err := sqlbase.LookupDatabaseID(ctx, txn, name) if err != nil { return sqlbase.InvalidID, err } - if dbID == sqlbase.InvalidID && required { + if !found && required { return dbID, sqlbase.NewUndefinedDatabaseError(name) } return dbID, nil @@ -258,10 +248,16 @@ func (dc *databaseCache) getCachedDatabaseID(name string) (sqlbase.ID, error) { return sqlbase.SystemDB.ID, nil } - nameKey := sqlbase.NewDatabaseKey(name) + var nameKey sqlbase.DescriptorKey = sqlbase.NewDatabaseKey(name) nameVal := dc.systemConfig.GetValue(nameKey.Key()) if nameVal == nil { - return sqlbase.InvalidID, nil + // Try the deprecated system.namespace before returning InvalidID. + // TODO(whomever): This can be removed in 20.2. + nameKey = sqlbase.NewDeprecatedDatabaseKey(name) + nameVal = dc.systemConfig.GetValue(nameKey.Key()) + if nameVal == nil { + return sqlbase.InvalidID, nil + } } id, err := nameVal.GetInt() @@ -280,6 +276,11 @@ func (p *planner) renameDatabase( oldKey := sqlbase.NewDatabaseKey(oldName).Key() newKey := sqlbase.NewDatabaseKey(newName).Key() + // TODO(whomever): This can be removed in 20.2. + if !cluster.Version.IsActive(ctx, p.ExecCfg().Settings, cluster.VersionNamespaceTableWithSchemas) { + oldKey = sqlbase.NewDeprecatedDatabaseKey(oldName).Key() + newKey = sqlbase.NewDeprecatedDatabaseKey(newName).Key() + } descID := oldDesc.GetID() descKey := sqlbase.MakeDescMetadataKey(descID) descDesc := sqlbase.WrapDescriptor(oldDesc) diff --git a/pkg/sql/descriptor.go b/pkg/sql/descriptor.go index bb2c6eb4ac1a..e715ceafb857 100644 --- a/pkg/sql/descriptor.go +++ b/pkg/sql/descriptor.go @@ -60,7 +60,16 @@ func GenerateUniqueDescID(ctx context.Context, db *client.DB) (sqlbase.ID, error func (p *planner) createDatabase( ctx context.Context, desc *sqlbase.DatabaseDescriptor, ifNotExists bool, ) (bool, error) { - plainKey := sqlbase.NewDatabaseKey(desc.Name) + shouldCreatePublicSchema := true + var plainKey sqlbase.DescriptorKey = sqlbase.NewDatabaseKey(desc.Name) + // TODO(whomever): This conditional can be removed in 20.2. Every database + // is created with a public schema for cluster version >= 20.1, so we can remove + // the `shouldCreatePublicSchema` logic as well. The key construction should + // also use the MakeDatabaseNameKey interface. + if !cluster.Version.IsActive(ctx, p.ExecCfg().Settings, cluster.VersionNamespaceTableWithSchemas) { + shouldCreatePublicSchema = false + plainKey = sqlbase.NewDeprecatedDatabaseKey(desc.Name) + } idKey := plainKey.Key() if exists, err := descExists(ctx, p.txn, idKey); err == nil && exists { @@ -78,7 +87,20 @@ func (p *planner) createDatabase( return false, err } - return true, p.createDescriptorWithID(ctx, idKey, id, desc, nil) + if err := p.createDescriptorWithID(ctx, idKey, id, desc, nil); err != nil { + return true, err + } + + // TODO(whomever): This check should be removed and a public schema should + // be created in every database in >= 20.2. + if shouldCreatePublicSchema { + // Every database must be initialized with the public schema. + if err := p.createSchemaWithID(ctx, sqlbase.NewPublicSchemaKey(id).Key(), keys.PublicSchemaID); err != nil { + return true, err + } + } + + return true, nil } func descExists(ctx context.Context, txn *client.Txn, idKey roachpb.Key) (bool, error) { @@ -239,10 +261,26 @@ func GetAllDatabaseDescriptorIDs(ctx context.Context, txn *client.Txn) ([]sqlbas if err != nil { return nil, err } + // See the comment in physical_schema_accessors.go, + // func (a UncachedPhysicalAccessor) GetObjectNames. Same concept + // applies here. + // TODO(whomever): This complexity can be removed in 20.2. + nameKey = sqlbase.NewDeprecatedDatabaseKey("" /* name */).Key() + dkvs, err := txn.Scan(ctx, nameKey, nameKey.PrefixEnd(), 0 /* maxRows */) + if err != nil { + return nil, err + } + kvs = append(kvs, dkvs...) descIDs := make([]sqlbase.ID, 0, len(kvs)) + alreadySeen := make(map[sqlbase.ID]bool) for _, kv := range kvs { - descIDs = append(descIDs, sqlbase.ID(kv.ValueInt())) + ID := sqlbase.ID(kv.ValueInt()) + if alreadySeen[ID] { + continue + } + alreadySeen[ID] = true + descIDs = append(descIDs, ID) } return descIDs, nil } diff --git a/pkg/sql/distsql/server.go b/pkg/sql/distsql/server.go index 293f53bfc1e9..a45d65bb604c 100644 --- a/pkg/sql/distsql/server.go +++ b/pkg/sql/distsql/server.go @@ -268,7 +268,7 @@ func (ds *ServerImpl) setupFlow( ApplicationName: req.EvalContext.ApplicationName, Database: req.EvalContext.Database, User: req.EvalContext.User, - SearchPath: sessiondata.MakeSearchPath(req.EvalContext.SearchPath), + SearchPath: sessiondata.MakeSearchPath(req.EvalContext.SearchPath, req.EvalContext.TemporarySchemaName), SequenceState: sessiondata.NewSequenceState(), DataConversion: sessiondata.DataConversionConfig{ Location: location, diff --git a/pkg/sql/drop_database.go b/pkg/sql/drop_database.go index 8be0190d8839..8bf5ce9d484c 100644 --- a/pkg/sql/drop_database.go +++ b/pkg/sql/drop_database.go @@ -106,7 +106,6 @@ func (p *planner) DropDatabase(ctx context.Context, n *tree.DropDatabase) (planN if err != nil { return nil, err } - return &dropDatabaseNode{n: n, dbDesc: dbDesc, td: td}, nil } @@ -159,15 +158,20 @@ func (n *dropDatabaseNode) startExec(params runParams) error { tbNameStrings = append(tbNameStrings, toDel.tn.FQString()) } - _ /* zoneKey */, nameKey, descKey := getKeysForDatabaseDescriptor(n.dbDesc) + descKey := sqlbase.MakeDescMetadataKey(n.dbDesc.ID) b := &client.Batch{} if p.ExtendedEvalContext().Tracing.KVTracingEnabled() { log.VEventf(ctx, 2, "Del %s", descKey) - log.VEventf(ctx, 2, "Del %s", nameKey) } b.Del(descKey) - b.Del(nameKey) + + err = sqlbase.RemoveDatabaseNamespaceEntry( + ctx, p.txn, n.dbDesc.Name, p.ExtendedEvalContext().Tracing.KVTracingEnabled(), + ) + if err != nil { + return err + } // No job was created because no tables were dropped, so zone config can be // immediately removed. diff --git a/pkg/sql/drop_table.go b/pkg/sql/drop_table.go index 213cde5da13d..c5fd9f577f0d 100644 --- a/pkg/sql/drop_table.go +++ b/pkg/sql/drop_table.go @@ -407,7 +407,6 @@ func (p *planner) initiateDropTable( "failed to mark job %d as as successful", errors.Safe(jobID)) } } - // Initiate an immediate schema change. When dropping a table // in a session, the data and the descriptor are not deleted. // Instead, that is taken care of asynchronously by the schema diff --git a/pkg/sql/drop_test.go b/pkg/sql/drop_test.go index f86c2d6da264..9c9ef33580bc 100644 --- a/pkg/sql/drop_test.go +++ b/pkg/sql/drop_test.go @@ -148,7 +148,7 @@ INSERT INTO t.kv VALUES ('c', 'e'), ('a', 'c'), ('b', 'd'); } dbDesc := desc.GetDatabase() - tbNameKey := sqlbase.NewTableKey(dbDesc.ID, "kv").Key() + tbNameKey := sqlbase.NewPublicTableKey(dbDesc.ID, "kv").Key() gr, err := kvDB.Get(ctx, tbNameKey) if err != nil { t.Fatal(err) @@ -322,7 +322,7 @@ INSERT INTO t.kv2 VALUES ('c', 'd'), ('a', 'b'), ('e', 'a'); } dbDesc := desc.GetDatabase() - tKey := sqlbase.NewTableKey(dbDesc.ID, "kv") + tKey := sqlbase.NewPublicTableKey(dbDesc.ID, "kv") gr, err := kvDB.Get(ctx, tKey.Key()) if err != nil { t.Fatal(err) @@ -337,7 +337,7 @@ INSERT INTO t.kv2 VALUES ('c', 'd'), ('a', 'b'), ('e', 'a'); } tbDesc := desc.Table(ts) - t2Key := sqlbase.NewTableKey(dbDesc.ID, "kv2") + t2Key := sqlbase.NewPublicTableKey(dbDesc.ID, "kv2") gr2, err := kvDB.Get(ctx, t2Key.Key()) if err != nil { t.Fatal(err) @@ -703,7 +703,7 @@ func TestDropTable(t *testing.T) { } tableDesc := sqlbase.GetTableDescriptor(kvDB, "t", "kv") - nameKey := sqlbase.NewTableKey(keys.MinNonPredefinedUserDescID, "kv").Key() + nameKey := sqlbase.NewPublicTableKey(keys.MinNonPredefinedUserDescID, "kv").Key() gr, err := kvDB.Get(ctx, nameKey) if err != nil { @@ -800,7 +800,7 @@ func TestDropTableDeleteData(t *testing.T) { descs = append(descs, sqlbase.GetTableDescriptor(kvDB, "t", tableName)) - nameKey := sqlbase.NewTableKey(keys.MinNonPredefinedUserDescID, tableName).Key() + nameKey := sqlbase.NewPublicTableKey(keys.MinNonPredefinedUserDescID, tableName).Key() gr, err := kvDB.Get(ctx, nameKey) if err != nil { t.Fatal(err) diff --git a/pkg/sql/event_log.go b/pkg/sql/event_log.go index a691bd1fa79f..57111a427234 100644 --- a/pkg/sql/event_log.go +++ b/pkg/sql/event_log.go @@ -158,7 +158,6 @@ VALUES( } args[3] = string(infoBytes) } - rows, err := ev.Exec(ctx, "log-event", txn, insertEventTableStmt, args...) if err != nil { return err diff --git a/pkg/sql/exec_util.go b/pkg/sql/exec_util.go index 1a30e33d90f2..1dbbc71de3b9 100644 --- a/pkg/sql/exec_util.go +++ b/pkg/sql/exec_util.go @@ -1786,6 +1786,10 @@ func (m *sessionDataMutator) SetDatabase(dbName string) { m.data.Database = dbName } +func (m *sessionDataMutator) SetTemporarySchemaName(scName string) { + m.data.SearchPath = m.data.SearchPath.WithTemporarySchemaName(scName) +} + func (m *sessionDataMutator) SetDefaultIntSize(size int) { m.data.DefaultIntSize = size } @@ -1834,8 +1838,8 @@ func (m *sessionDataMutator) SetSafeUpdates(val bool) { m.data.SafeUpdates = val } -func (m *sessionDataMutator) SetSearchPath(val sessiondata.SearchPath) { - m.data.SearchPath = val +func (m *sessionDataMutator) UpdateSearchPath(paths []string) { + m.data.SearchPath = m.data.SearchPath.UpdatePaths(paths) } func (m *sessionDataMutator) SetLocation(loc *time.Location) { diff --git a/pkg/sql/execinfrapb/api.go b/pkg/sql/execinfrapb/api.go index ed1473ef25d4..c4fea9e89e85 100644 --- a/pkg/sql/execinfrapb/api.go +++ b/pkg/sql/execinfrapb/api.go @@ -45,15 +45,16 @@ func MakeEvalContext(evalCtx *tree.EvalContext) EvalContext { panic("unknown format") } res := EvalContext{ - StmtTimestampNanos: evalCtx.StmtTimestamp.UnixNano(), - TxnTimestampNanos: evalCtx.TxnTimestamp.UnixNano(), - Location: evalCtx.GetLocation().String(), - Database: evalCtx.SessionData.Database, - User: evalCtx.SessionData.User, - ApplicationName: evalCtx.SessionData.ApplicationName, - BytesEncodeFormat: be, - ExtraFloatDigits: int32(evalCtx.SessionData.DataConversion.ExtraFloatDigits), - Vectorize: int32(evalCtx.SessionData.VectorizeMode), + StmtTimestampNanos: evalCtx.StmtTimestamp.UnixNano(), + TxnTimestampNanos: evalCtx.TxnTimestamp.UnixNano(), + Location: evalCtx.GetLocation().String(), + Database: evalCtx.SessionData.Database, + TemporarySchemaName: evalCtx.SessionData.SearchPath.GetTemporarySchemaName(), + User: evalCtx.SessionData.User, + ApplicationName: evalCtx.SessionData.ApplicationName, + BytesEncodeFormat: be, + ExtraFloatDigits: int32(evalCtx.SessionData.DataConversion.ExtraFloatDigits), + Vectorize: int32(evalCtx.SessionData.VectorizeMode), } // Populate the search path. Make sure not to include the implicit pg_catalog, diff --git a/pkg/sql/execinfrapb/api.pb.go b/pkg/sql/execinfrapb/api.pb.go index e12e10bbc7fe..4afb1ff72b0a 100644 --- a/pkg/sql/execinfrapb/api.pb.go +++ b/pkg/sql/execinfrapb/api.pb.go @@ -75,7 +75,7 @@ func (x *BytesEncodeFormat) UnmarshalJSON(data []byte) error { return nil } func (BytesEncodeFormat) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_cc1a98810c407fa3, []int{0} + return fileDescriptor_api_0f64f87ae4c87c57, []int{0} } type SetupFlowRequest struct { @@ -96,7 +96,7 @@ func (m *SetupFlowRequest) Reset() { *m = SetupFlowRequest{} } func (m *SetupFlowRequest) String() string { return proto.CompactTextString(m) } func (*SetupFlowRequest) ProtoMessage() {} func (*SetupFlowRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cc1a98810c407fa3, []int{0} + return fileDescriptor_api_0f64f87ae4c87c57, []int{0} } func (m *SetupFlowRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -134,7 +134,7 @@ func (m *FlowSpec) Reset() { *m = FlowSpec{} } func (m *FlowSpec) String() string { return proto.CompactTextString(m) } func (*FlowSpec) ProtoMessage() {} func (*FlowSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cc1a98810c407fa3, []int{1} + return fileDescriptor_api_0f64f87ae4c87c57, []int{1} } func (m *FlowSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -165,22 +165,23 @@ type EvalContext struct { TxnTimestampNanos int64 `protobuf:"varint,2,opt,name=txnTimestampNanos" json:"txnTimestampNanos"` // The name of the location according to whose current timezone we're going to // parse timestamps. Used to init EvalContext.Location. - Location string `protobuf:"bytes,4,opt,name=location" json:"location"` - Database string `protobuf:"bytes,5,opt,name=database" json:"database"` - SearchPath []string `protobuf:"bytes,6,rep,name=searchPath" json:"searchPath,omitempty"` - User string `protobuf:"bytes,7,opt,name=user" json:"user"` - SeqState SequenceState `protobuf:"bytes,8,opt,name=seq_state,json=seqState" json:"seq_state"` - ApplicationName string `protobuf:"bytes,9,opt,name=application_name,json=applicationName" json:"application_name"` - BytesEncodeFormat BytesEncodeFormat `protobuf:"varint,10,opt,name=bytes_encode_format,json=bytesEncodeFormat,enum=cockroach.sql.distsqlrun.BytesEncodeFormat" json:"bytes_encode_format"` - ExtraFloatDigits int32 `protobuf:"varint,11,opt,name=extra_float_digits,json=extraFloatDigits" json:"extra_float_digits"` - Vectorize int32 `protobuf:"varint,12,opt,name=vectorize" json:"vectorize"` + Location string `protobuf:"bytes,4,opt,name=location" json:"location"` + Database string `protobuf:"bytes,5,opt,name=database" json:"database"` + SearchPath []string `protobuf:"bytes,6,rep,name=search_path,json=searchPath" json:"search_path,omitempty"` + TemporarySchemaName string `protobuf:"bytes,13,opt,name=temporary_schema_name,json=temporarySchemaName" json:"temporary_schema_name"` + User string `protobuf:"bytes,7,opt,name=user" json:"user"` + SeqState SequenceState `protobuf:"bytes,8,opt,name=seq_state,json=seqState" json:"seq_state"` + ApplicationName string `protobuf:"bytes,9,opt,name=application_name,json=applicationName" json:"application_name"` + BytesEncodeFormat BytesEncodeFormat `protobuf:"varint,10,opt,name=bytes_encode_format,json=bytesEncodeFormat,enum=cockroach.sql.distsqlrun.BytesEncodeFormat" json:"bytes_encode_format"` + ExtraFloatDigits int32 `protobuf:"varint,11,opt,name=extra_float_digits,json=extraFloatDigits" json:"extra_float_digits"` + Vectorize int32 `protobuf:"varint,12,opt,name=vectorize" json:"vectorize"` } func (m *EvalContext) Reset() { *m = EvalContext{} } func (m *EvalContext) String() string { return proto.CompactTextString(m) } func (*EvalContext) ProtoMessage() {} func (*EvalContext) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cc1a98810c407fa3, []int{2} + return fileDescriptor_api_0f64f87ae4c87c57, []int{2} } func (m *EvalContext) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -217,7 +218,7 @@ func (m *SequenceState) Reset() { *m = SequenceState{} } func (m *SequenceState) String() string { return proto.CompactTextString(m) } func (*SequenceState) ProtoMessage() {} func (*SequenceState) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cc1a98810c407fa3, []int{3} + return fileDescriptor_api_0f64f87ae4c87c57, []int{3} } func (m *SequenceState) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -252,7 +253,7 @@ func (m *SequenceState_Seq) Reset() { *m = SequenceState_Seq{} } func (m *SequenceState_Seq) String() string { return proto.CompactTextString(m) } func (*SequenceState_Seq) ProtoMessage() {} func (*SequenceState_Seq) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cc1a98810c407fa3, []int{3, 0} + return fileDescriptor_api_0f64f87ae4c87c57, []int{3, 0} } func (m *SequenceState_Seq) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -285,7 +286,7 @@ func (m *SimpleResponse) Reset() { *m = SimpleResponse{} } func (m *SimpleResponse) String() string { return proto.CompactTextString(m) } func (*SimpleResponse) ProtoMessage() {} func (*SimpleResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cc1a98810c407fa3, []int{4} + return fileDescriptor_api_0f64f87ae4c87c57, []int{4} } func (m *SimpleResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -328,7 +329,7 @@ func (m *ConsumerSignal) Reset() { *m = ConsumerSignal{} } func (m *ConsumerSignal) String() string { return proto.CompactTextString(m) } func (*ConsumerSignal) ProtoMessage() {} func (*ConsumerSignal) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cc1a98810c407fa3, []int{5} + return fileDescriptor_api_0f64f87ae4c87c57, []int{5} } func (m *ConsumerSignal) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -360,7 +361,7 @@ func (m *DrainRequest) Reset() { *m = DrainRequest{} } func (m *DrainRequest) String() string { return proto.CompactTextString(m) } func (*DrainRequest) ProtoMessage() {} func (*DrainRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cc1a98810c407fa3, []int{6} + return fileDescriptor_api_0f64f87ae4c87c57, []int{6} } func (m *DrainRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -411,7 +412,7 @@ func (m *ConsumerHandshake) Reset() { *m = ConsumerHandshake{} } func (m *ConsumerHandshake) String() string { return proto.CompactTextString(m) } func (*ConsumerHandshake) ProtoMessage() {} func (*ConsumerHandshake) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cc1a98810c407fa3, []int{7} + return fileDescriptor_api_0f64f87ae4c87c57, []int{7} } func (m *ConsumerHandshake) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -847,6 +848,10 @@ func (m *EvalContext) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x60 i++ i = encodeVarintApi(dAtA, i, uint64(m.Vectorize)) + dAtA[i] = 0x6a + i++ + i = encodeVarintApi(dAtA, i, uint64(len(m.TemporarySchemaName))) + i += copy(dAtA[i:], m.TemporarySchemaName) return i, nil } @@ -1118,6 +1123,8 @@ func (m *EvalContext) Size() (n int) { n += 1 + sovApi(uint64(m.BytesEncodeFormat)) n += 1 + sovApi(uint64(m.ExtraFloatDigits)) n += 1 + sovApi(uint64(m.Vectorize)) + l = len(m.TemporarySchemaName) + n += 1 + l + sovApi(uint64(l)) return n } @@ -1859,6 +1866,35 @@ func (m *EvalContext) Unmarshal(dAtA []byte) error { break } } + case 13: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TemporarySchemaName", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthApi + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.TemporarySchemaName = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipApi(dAtA[iNdEx:]) @@ -2597,82 +2633,84 @@ var ( ErrIntOverflowApi = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("sql/execinfrapb/api.proto", fileDescriptor_api_cc1a98810c407fa3) } - -var fileDescriptor_api_cc1a98810c407fa3 = []byte{ - // 1180 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x56, 0xcf, 0x6e, 0xdb, 0xc6, - 0x13, 0x16, 0x25, 0xda, 0x92, 0x46, 0xb6, 0x7f, 0xf2, 0xfe, 0x82, 0x82, 0xd5, 0x41, 0x12, 0xd4, - 0x7f, 0x6a, 0x82, 0x4a, 0xa9, 0x93, 0xf6, 0xd0, 0x16, 0x28, 0x22, 0x4b, 0x69, 0xec, 0xd4, 0x81, - 0x4b, 0xa6, 0x41, 0xd0, 0x43, 0x89, 0x15, 0x39, 0x96, 0x88, 0x90, 0x5c, 0x8a, 0xbb, 0x72, 0x9c, - 0x3e, 0x41, 0x8f, 0x79, 0x84, 0x1e, 0xfa, 0x06, 0x7d, 0x82, 0x02, 0x3d, 0xf8, 0x98, 0x43, 0x0f, - 0x41, 0x0f, 0x6e, 0x6b, 0xbf, 0x45, 0x4f, 0xc5, 0x2e, 0x49, 0x9b, 0x96, 0x6b, 0xd7, 0xbd, 0x71, - 0xe7, 0xfb, 0xbe, 0xd9, 0xdd, 0x99, 0xd9, 0x19, 0xc2, 0x9b, 0x7c, 0xe6, 0xf7, 0xf1, 0x00, 0x1d, - 0x2f, 0xdc, 0x8b, 0x69, 0x34, 0xee, 0xd3, 0xc8, 0xeb, 0x45, 0x31, 0x13, 0x8c, 0x18, 0x0e, 0x73, - 0x9e, 0xc5, 0x8c, 0x3a, 0xd3, 0x1e, 0x9f, 0xf9, 0x3d, 0xd7, 0xe3, 0x82, 0xcf, 0xfc, 0x78, 0x1e, - 0x36, 0x6e, 0x4c, 0xd8, 0x84, 0x29, 0x52, 0x5f, 0x7e, 0x25, 0xfc, 0x46, 0x6b, 0xc2, 0xd8, 0xc4, - 0xc7, 0xbe, 0x5a, 0x8d, 0xe7, 0x7b, 0x7d, 0xe1, 0x05, 0xc8, 0x05, 0x0d, 0xa2, 0x94, 0x40, 0x94, - 0xb3, 0x68, 0xdc, 0x77, 0xa9, 0xa0, 0xa9, 0xad, 0xb1, 0xb8, 0x7f, 0x0e, 0x6b, 0x2f, 0x62, 0x51, - 0xcc, 0x1c, 0xe4, 0x9c, 0xc5, 0x3c, 0x61, 0x74, 0x7e, 0x29, 0x42, 0xdd, 0x42, 0x31, 0x8f, 0xee, - 0xfb, 0xec, 0xb9, 0x89, 0xb3, 0x39, 0x72, 0x41, 0x3e, 0x03, 0x7d, 0xcf, 0x67, 0xcf, 0x8d, 0x52, - 0x5b, 0xeb, 0xd6, 0x36, 0x3a, 0xbd, 0xcb, 0xae, 0xd1, 0x93, 0x22, 0x2b, 0x42, 0x67, 0xa0, 0x1f, - 0x1e, 0xb5, 0x0a, 0xa6, 0x52, 0x91, 0xdb, 0x50, 0xde, 0xc7, 0x98, 0x7b, 0x2c, 0x34, 0x96, 0xda, - 0x5a, 0x77, 0x75, 0xf0, 0x86, 0x04, 0xff, 0x3a, 0x6a, 0xad, 0x0d, 0x3d, 0x2e, 0xac, 0xaf, 0xbe, - 0x7c, 0x92, 0xa0, 0x66, 0x46, 0x23, 0x3b, 0x50, 0xc3, 0x7d, 0xea, 0x6f, 0xb2, 0x50, 0xe0, 0x81, - 0x30, 0x96, 0xd5, 0xb6, 0xef, 0x5c, 0xbe, 0xed, 0xe8, 0x8c, 0x9c, 0xee, 0x9c, 0xd7, 0x93, 0x11, - 0xac, 0x89, 0x83, 0xd0, 0x76, 0x18, 0x8b, 0x5d, 0x3b, 0x40, 0x41, 0x8d, 0xb2, 0xf2, 0xd8, 0xca, - 0x79, 0x4c, 0x03, 0xd9, 0x7b, 0x7c, 0x10, 0x6e, 0x4a, 0xde, 0x0e, 0x0a, 0x6a, 0xae, 0x88, 0xdc, - 0x8a, 0x34, 0xa1, 0xfc, 0x38, 0xa6, 0x0e, 0x3e, 0x7c, 0x62, 0x54, 0xda, 0x5a, 0xb7, 0x92, 0x6e, - 0x95, 0x19, 0xb7, 0xf5, 0x8a, 0x56, 0x2f, 0x6e, 0xeb, 0x95, 0x62, 0xbd, 0xd4, 0x39, 0xd1, 0xa0, - 0x92, 0x05, 0x83, 0x7c, 0x08, 0x65, 0x19, 0x08, 0xdb, 0x73, 0x0d, 0xad, 0xad, 0x75, 0x57, 0x06, - 0x86, 0x14, 0xfe, 0x76, 0xd4, 0x5a, 0x96, 0x94, 0xad, 0xe1, 0xf1, 0xe9, 0x97, 0xb9, 0x2c, 0x89, - 0x5b, 0x2e, 0xd9, 0x01, 0x38, 0x4b, 0x8d, 0x51, 0x6c, 0x97, 0xba, 0xb5, 0x8d, 0xf7, 0x2e, 0x0f, - 0xc0, 0x6e, 0xc6, 0xcd, 0x05, 0x3f, 0xe7, 0x80, 0x7c, 0x0d, 0xe5, 0x09, 0x15, 0xf8, 0x9c, 0xbe, - 0x50, 0x39, 0x5c, 0x1a, 0x7c, 0x9a, 0xa6, 0xe0, 0xce, 0xc4, 0x13, 0xd3, 0xf9, 0xb8, 0xe7, 0xb0, - 0xa0, 0x7f, 0xea, 0xdd, 0x1d, 0x9f, 0x7d, 0xf7, 0xa3, 0x67, 0x93, 0x7e, 0x16, 0x9c, 0x47, 0xcc, - 0xc5, 0xad, 0xa1, 0x99, 0xf9, 0xea, 0xfc, 0xa4, 0x43, 0x2d, 0x17, 0x7b, 0x72, 0x17, 0x08, 0x17, - 0x81, 0x78, 0x9c, 0x55, 0xe9, 0x23, 0x1a, 0x32, 0xae, 0xee, 0x5c, 0x4a, 0x0f, 0xf5, 0x0f, 0x38, - 0xd9, 0x80, 0x75, 0x71, 0x10, 0x2e, 0x88, 0x8a, 0x39, 0xd1, 0x45, 0x98, 0xb4, 0xa1, 0xe2, 0x33, - 0x87, 0x0a, 0x59, 0x54, 0x7a, 0x5b, 0xeb, 0x56, 0x53, 0xea, 0xa9, 0x55, 0x32, 0x64, 0xe1, 0x8f, - 0x29, 0x47, 0x55, 0x76, 0xa7, 0x8c, 0xcc, 0x4a, 0x9a, 0x00, 0x1c, 0x69, 0xec, 0x4c, 0x77, 0xa9, - 0x98, 0x1a, 0xcb, 0xed, 0x52, 0xb7, 0x6a, 0xe6, 0x2c, 0xc4, 0x00, 0x7d, 0xce, 0x31, 0x56, 0xc5, - 0x92, 0xa9, 0x95, 0x85, 0x6c, 0x43, 0x95, 0xe3, 0xcc, 0xe6, 0x82, 0x0a, 0x54, 0xb5, 0x70, 0x65, - 0x72, 0x2c, 0xf9, 0x8a, 0x42, 0x07, 0x2d, 0x49, 0xcf, 0x4e, 0xc1, 0x71, 0xa6, 0xd6, 0xa4, 0x0f, - 0x75, 0x1a, 0x45, 0xbe, 0x97, 0x1c, 0xdb, 0x0e, 0x69, 0x80, 0x46, 0x35, 0xb7, 0xe3, 0xff, 0x72, - 0xe8, 0x23, 0x1a, 0x20, 0xa1, 0xf0, 0xff, 0xf1, 0x0b, 0x81, 0xdc, 0xc6, 0xd0, 0x61, 0x2e, 0xda, - 0x7b, 0x2c, 0x0e, 0xa8, 0x30, 0xa0, 0xad, 0x75, 0xd7, 0x36, 0x6e, 0x5d, 0x7e, 0x8c, 0x81, 0x14, - 0x8d, 0x94, 0xe6, 0xbe, 0x92, 0x64, 0xd1, 0x1d, 0x2f, 0x02, 0x64, 0x03, 0x08, 0x1e, 0x88, 0x98, - 0xda, 0x7b, 0x3e, 0xa3, 0xc2, 0x76, 0xbd, 0x89, 0x27, 0xb8, 0x51, 0x53, 0x95, 0x93, 0x88, 0xea, - 0x0a, 0xbf, 0x2f, 0xe1, 0xa1, 0x42, 0x49, 0x07, 0xaa, 0xfb, 0xe8, 0x08, 0x16, 0x7b, 0xdf, 0xa1, - 0xb1, 0x92, 0xa3, 0x9e, 0x99, 0xb7, 0xf5, 0x4a, 0xa9, 0xae, 0x77, 0x7e, 0xd5, 0x60, 0xf5, 0x5c, - 0x4c, 0xc8, 0xe7, 0xa0, 0x73, 0x9c, 0xc9, 0x4a, 0x91, 0x75, 0x7e, 0xeb, 0x9a, 0xa1, 0x94, 0x2b, - 0x53, 0x09, 0xc9, 0x6d, 0xb8, 0xe1, 0x53, 0x2e, 0x6c, 0x99, 0x15, 0x2f, 0x74, 0x62, 0x0c, 0x30, - 0x14, 0xe8, 0xaa, 0x2a, 0x5a, 0x35, 0x89, 0xc4, 0x2c, 0x9c, 0x6d, 0x9d, 0x21, 0x8d, 0x5d, 0x28, - 0x59, 0x38, 0x23, 0x6f, 0xc3, 0xb2, 0xd2, 0x24, 0x2f, 0x73, 0x75, 0xb0, 0x2a, 0x8f, 0x7c, 0x7c, - 0xd4, 0x5a, 0x92, 0xf4, 0xa1, 0xb9, 0xc4, 0x71, 0xb6, 0xe5, 0x92, 0xb7, 0x00, 0x7c, 0x2a, 0x90, - 0x0b, 0x7b, 0x9f, 0xfa, 0xe7, 0x4a, 0xb3, 0x9a, 0xd8, 0x9f, 0x50, 0xbf, 0xf3, 0x05, 0xac, 0x59, - 0x5e, 0x10, 0xf9, 0x68, 0x22, 0x8f, 0x58, 0xc8, 0x91, 0x7c, 0x04, 0x4b, 0x18, 0xc7, 0x2c, 0x56, - 0xbe, 0xcf, 0xb7, 0x9b, 0xc5, 0x06, 0x26, 0x69, 0x66, 0xc2, 0xee, 0x7c, 0x5f, 0x84, 0xb5, 0x4d, - 0x16, 0xf2, 0x79, 0x80, 0xb1, 0xe5, 0x4d, 0x42, 0xea, 0x93, 0x87, 0xb0, 0xea, 0xc6, 0xd4, 0x0b, - 0xed, 0x38, 0xe9, 0xc8, 0xa9, 0xc7, 0x77, 0x2f, 0xf7, 0x38, 0x94, 0xf4, 0xb4, 0x7f, 0x9b, 0x2b, - 0x6e, 0x6e, 0x45, 0x9e, 0x02, 0xe1, 0xb2, 0xc3, 0xdb, 0xaa, 0x29, 0x65, 0x1e, 0x8b, 0xca, 0xe3, - 0xcd, 0xab, 0x62, 0x7f, 0x7e, 0x2a, 0x98, 0x75, 0xbe, 0x38, 0x27, 0xb6, 0xa0, 0x3a, 0xa5, 0xa1, - 0xcb, 0xa7, 0xf4, 0x19, 0xa6, 0xc3, 0xe2, 0x8a, 0x64, 0x66, 0x77, 0x7c, 0x90, 0x49, 0xcc, 0x33, - 0xf5, 0x27, 0xfa, 0xe1, 0x0f, 0x2d, 0xad, 0xb3, 0x06, 0x2b, 0xf9, 0x8b, 0x74, 0x7e, 0x2c, 0xc2, - 0xfa, 0x05, 0x19, 0xb9, 0x03, 0xc4, 0x49, 0x8d, 0x36, 0x77, 0xa6, 0xe8, 0xce, 0x7d, 0x4c, 0x12, - 0x9a, 0xf5, 0xe8, 0xf5, 0x0c, 0xb7, 0x32, 0x98, 0x7c, 0x0b, 0x8d, 0x0b, 0x22, 0xdb, 0x45, 0xea, - 0xfa, 0x5e, 0x88, 0x69, 0x34, 0x1a, 0xbd, 0x64, 0x00, 0xf7, 0xb2, 0x01, 0xdc, 0x3b, 0x6d, 0x43, - 0x03, 0xfd, 0xe5, 0xef, 0x2d, 0xcd, 0x34, 0x16, 0x1d, 0x0f, 0x53, 0x0f, 0xf9, 0xa9, 0x57, 0xba, - 0xde, 0xd4, 0x7b, 0x00, 0x37, 0x02, 0x2f, 0xb4, 0xa9, 0xe3, 0x60, 0x24, 0xd0, 0xb5, 0x33, 0xb9, - 0x7e, 0xa5, 0x9c, 0x04, 0x5e, 0x78, 0x2f, 0x95, 0xa4, 0xb6, 0x9b, 0x77, 0x61, 0xfd, 0xc2, 0x6b, - 0x27, 0x65, 0x28, 0x3d, 0x18, 0x3d, 0xad, 0x17, 0x08, 0xc0, 0xf2, 0xc8, 0xda, 0xbc, 0xb7, 0x3b, - 0xaa, 0x6b, 0xf2, 0x7b, 0x70, 0xcf, 0x1a, 0x7d, 0x7c, 0xb7, 0x5e, 0xdc, 0xf8, 0xb9, 0x08, 0xe5, - 0xd4, 0x39, 0x99, 0x42, 0xcd, 0x9c, 0x87, 0xd6, 0x8b, 0xd0, 0x91, 0xf9, 0x25, 0xdd, 0x7f, 0xcf, - 0x62, 0x52, 0xa9, 0x8d, 0xf7, 0xaf, 0x1c, 0x52, 0xee, 0xdc, 0xc1, 0x78, 0x07, 0x39, 0xa7, 0x13, - 0xec, 0x14, 0xba, 0xda, 0x6d, 0x8d, 0x38, 0x50, 0x3d, 0xad, 0x2c, 0xf2, 0x1f, 0xca, 0xaf, 0x71, - 0xc5, 0x99, 0xce, 0xbf, 0xc3, 0x4e, 0x81, 0x4c, 0x00, 0xd4, 0x34, 0x16, 0x31, 0xd2, 0x80, 0x5c, - 0xff, 0x8c, 0x8d, 0x6b, 0x5f, 0x3c, 0xb9, 0xcd, 0xe0, 0x83, 0xc3, 0x3f, 0x9b, 0x85, 0xc3, 0xe3, - 0xa6, 0xf6, 0xea, 0xb8, 0xa9, 0xbd, 0x3e, 0x6e, 0x6a, 0x7f, 0x1c, 0x37, 0xb5, 0x97, 0x27, 0xcd, - 0xc2, 0xab, 0x93, 0x66, 0xe1, 0xf5, 0x49, 0xb3, 0xf0, 0x4d, 0x2d, 0xf7, 0xfb, 0xf5, 0x77, 0x00, - 0x00, 0x00, 0xff, 0xff, 0xf4, 0x0c, 0x64, 0x7c, 0x2c, 0x0a, 0x00, 0x00, +func init() { proto.RegisterFile("sql/execinfrapb/api.proto", fileDescriptor_api_0f64f87ae4c87c57) } + +var fileDescriptor_api_0f64f87ae4c87c57 = []byte{ + // 1210 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x56, 0xcd, 0x6e, 0xdb, 0xc6, + 0x16, 0x16, 0x25, 0xd9, 0x96, 0x8e, 0x2c, 0x5f, 0x79, 0x92, 0x7b, 0xc1, 0xab, 0x85, 0x24, 0xa8, + 0x7f, 0x6a, 0x82, 0x4a, 0xa9, 0x93, 0x16, 0x45, 0x5b, 0xa0, 0x88, 0x2c, 0xa5, 0xb1, 0x53, 0x07, + 0x2e, 0x99, 0x06, 0x41, 0x17, 0x25, 0x46, 0xe4, 0xb1, 0x44, 0x84, 0xe4, 0x50, 0x9c, 0x91, 0x63, + 0xf7, 0x09, 0xba, 0xcc, 0x23, 0x74, 0xd1, 0x17, 0x29, 0xd0, 0x85, 0x97, 0x59, 0x74, 0x11, 0x74, + 0xe1, 0xb6, 0xf6, 0xbe, 0x0f, 0xd0, 0x55, 0x31, 0x43, 0x52, 0xa6, 0xe5, 0xda, 0x75, 0x77, 0x9c, + 0xf3, 0x7d, 0xdf, 0x99, 0x39, 0x3f, 0x33, 0x87, 0xf0, 0x7f, 0x3e, 0xf5, 0x7a, 0x78, 0x80, 0xb6, + 0x1b, 0xec, 0x45, 0x34, 0x1c, 0xf5, 0x68, 0xe8, 0x76, 0xc3, 0x88, 0x09, 0x46, 0x74, 0x9b, 0xd9, + 0xcf, 0x23, 0x46, 0xed, 0x49, 0x97, 0x4f, 0xbd, 0xae, 0xe3, 0x72, 0xc1, 0xa7, 0x5e, 0x34, 0x0b, + 0xea, 0x37, 0xc7, 0x6c, 0xcc, 0x14, 0xa9, 0x27, 0xbf, 0x62, 0x7e, 0xbd, 0x39, 0x66, 0x6c, 0xec, + 0x61, 0x4f, 0xad, 0x46, 0xb3, 0xbd, 0x9e, 0x70, 0x7d, 0xe4, 0x82, 0xfa, 0x61, 0x42, 0x20, 0xca, + 0x59, 0x38, 0xea, 0x39, 0x54, 0xd0, 0xc4, 0x56, 0x5f, 0xdc, 0x3f, 0x83, 0xb5, 0x16, 0xb1, 0x30, + 0x62, 0x36, 0x72, 0xce, 0x22, 0x1e, 0x33, 0xda, 0x3f, 0xe5, 0xa1, 0x66, 0xa2, 0x98, 0x85, 0x0f, + 0x3c, 0xf6, 0xc2, 0xc0, 0xe9, 0x0c, 0xb9, 0x20, 0x9f, 0x42, 0x71, 0xcf, 0x63, 0x2f, 0xf4, 0x42, + 0x4b, 0xeb, 0x54, 0x36, 0xda, 0xdd, 0xcb, 0xc2, 0xe8, 0x4a, 0x91, 0x19, 0xa2, 0xdd, 0x2f, 0x1e, + 0x1d, 0x37, 0x73, 0x86, 0x52, 0x91, 0x3b, 0xb0, 0xb2, 0x8f, 0x11, 0x77, 0x59, 0xa0, 0x2f, 0xb5, + 0xb4, 0x4e, 0xb5, 0xff, 0x3f, 0x09, 0xfe, 0x79, 0xdc, 0x5c, 0x1b, 0xb8, 0x5c, 0x98, 0x5f, 0x7e, + 0xf1, 0x34, 0x46, 0x8d, 0x94, 0x46, 0x76, 0xa0, 0x82, 0xfb, 0xd4, 0xdb, 0x64, 0x81, 0xc0, 0x03, + 0xa1, 0x2f, 0xab, 0x6d, 0xdf, 0xba, 0x7c, 0xdb, 0xe1, 0x19, 0x39, 0xd9, 0x39, 0xab, 0x27, 0x43, + 0x58, 0x13, 0x07, 0x81, 0x65, 0x33, 0x16, 0x39, 0x96, 0x8f, 0x82, 0xea, 0x2b, 0xca, 0x63, 0x33, + 0xe3, 0x31, 0x49, 0x64, 0xf7, 0xc9, 0x41, 0xb0, 0x29, 0x79, 0x3b, 0x28, 0xa8, 0xb1, 0x2a, 0x32, + 0x2b, 0xd2, 0x80, 0x95, 0x27, 0x11, 0xb5, 0xf1, 0xd1, 0x53, 0xbd, 0xd4, 0xd2, 0x3a, 0xa5, 0x64, + 0xab, 0xd4, 0xb8, 0x5d, 0x2c, 0x69, 0xb5, 0xfc, 0x76, 0xb1, 0x94, 0xaf, 0x15, 0xda, 0xa7, 0x1a, + 0x94, 0xd2, 0x64, 0x90, 0xf7, 0x61, 0x45, 0x26, 0xc2, 0x72, 0x1d, 0x5d, 0x6b, 0x69, 0x9d, 0xd5, + 0xbe, 0x2e, 0x85, 0xbf, 0x1c, 0x37, 0x97, 0x25, 0x65, 0x6b, 0x70, 0x32, 0xff, 0x32, 0x96, 0x25, + 0x71, 0xcb, 0x21, 0x3b, 0x00, 0x67, 0xa5, 0xd1, 0xf3, 0xad, 0x42, 0xa7, 0xb2, 0xf1, 0xce, 0xe5, + 0x09, 0xd8, 0x4d, 0xb9, 0x99, 0xe4, 0x67, 0x1c, 0x90, 0xaf, 0x60, 0x65, 0x4c, 0x05, 0xbe, 0xa0, + 0x87, 0xaa, 0x86, 0x4b, 0xfd, 0x4f, 0x92, 0x12, 0xdc, 0x1d, 0xbb, 0x62, 0x32, 0x1b, 0x75, 0x6d, + 0xe6, 0xf7, 0xe6, 0xde, 0x9d, 0xd1, 0xd9, 0x77, 0x2f, 0x7c, 0x3e, 0xee, 0xa5, 0xc9, 0x79, 0xcc, + 0x1c, 0xdc, 0x1a, 0x18, 0xa9, 0xaf, 0xf6, 0x1f, 0x45, 0xa8, 0x64, 0x72, 0x4f, 0xee, 0x01, 0xe1, + 0xc2, 0x17, 0x4f, 0xd2, 0x2e, 0x7d, 0x4c, 0x03, 0xc6, 0x55, 0xcc, 0x85, 0xe4, 0x50, 0x7f, 0x83, + 0x93, 0x0d, 0x58, 0x17, 0x07, 0xc1, 0x82, 0x28, 0x9f, 0x11, 0x5d, 0x84, 0x49, 0x0b, 0x4a, 0x1e, + 0xb3, 0xa9, 0x90, 0x4d, 0x55, 0x6c, 0x69, 0x9d, 0x72, 0x42, 0x9d, 0x5b, 0x25, 0x43, 0x36, 0xfe, + 0x88, 0x72, 0x54, 0x6d, 0x37, 0x67, 0xa4, 0x56, 0xd2, 0x84, 0x0a, 0x47, 0x1a, 0xd9, 0x13, 0x2b, + 0xa4, 0x62, 0xa2, 0x2f, 0xb7, 0x0a, 0x9d, 0xb2, 0x01, 0xb1, 0x69, 0x97, 0x8a, 0x09, 0xd1, 0xa1, + 0x38, 0xe3, 0x18, 0xa9, 0x6e, 0x49, 0xe5, 0xca, 0x42, 0xb6, 0xa1, 0xcc, 0x71, 0x6a, 0x71, 0x41, + 0x05, 0xaa, 0x66, 0xb8, 0xb2, 0x3a, 0xa6, 0xbc, 0x46, 0x81, 0x8d, 0xa6, 0xa4, 0xa7, 0xc7, 0xe0, + 0x38, 0x55, 0x6b, 0xd2, 0x83, 0x1a, 0x0d, 0x43, 0xcf, 0x8d, 0xcf, 0x6d, 0x05, 0xd4, 0x47, 0xbd, + 0x9c, 0xd9, 0xf1, 0x3f, 0x19, 0xf4, 0x31, 0xf5, 0x91, 0x50, 0xb8, 0x31, 0x3a, 0x14, 0xc8, 0x2d, + 0x0c, 0x6c, 0xe6, 0xa0, 0xb5, 0xc7, 0x22, 0x9f, 0x0a, 0x1d, 0x5a, 0x5a, 0x67, 0x6d, 0xe3, 0xf6, + 0xe5, 0xc7, 0xe8, 0x4b, 0xd1, 0x50, 0x69, 0x1e, 0x28, 0x49, 0x9a, 0xde, 0xd1, 0x22, 0x40, 0x36, + 0x80, 0xe0, 0x81, 0x88, 0xa8, 0xb5, 0xe7, 0x31, 0x2a, 0x2c, 0xc7, 0x1d, 0xbb, 0x82, 0xeb, 0x15, + 0xd5, 0x3a, 0xb1, 0xa8, 0xa6, 0xf0, 0x07, 0x12, 0x1e, 0x28, 0x94, 0xb4, 0xa1, 0xbc, 0x8f, 0xb6, + 0x60, 0x91, 0xfb, 0x2d, 0xea, 0xab, 0x19, 0xea, 0x99, 0x99, 0x7c, 0x04, 0xff, 0x15, 0xe8, 0x87, + 0x2c, 0xa2, 0xd1, 0xa1, 0xc5, 0xed, 0x09, 0xfa, 0x34, 0x0e, 0xb8, 0x9a, 0x09, 0xf8, 0xc6, 0x9c, + 0x62, 0x2a, 0x86, 0x0c, 0x7a, 0xbb, 0x58, 0x2a, 0xd4, 0x8a, 0xed, 0x9f, 0x35, 0xa8, 0x9e, 0xcb, + 0x26, 0xf9, 0x0c, 0x8a, 0x1c, 0xa7, 0xb2, 0xc9, 0xe4, 0x15, 0xb9, 0x7d, 0xcd, 0x22, 0xc8, 0x95, + 0xa1, 0x84, 0xe4, 0x0e, 0xdc, 0xf4, 0x28, 0x17, 0x96, 0xac, 0xa7, 0x1b, 0xd8, 0x11, 0xfa, 0x18, + 0x08, 0x74, 0x54, 0x03, 0x56, 0x0d, 0x22, 0x31, 0x13, 0xa7, 0x5b, 0x67, 0x48, 0x7d, 0x17, 0x0a, + 0x26, 0x4e, 0xc9, 0x9b, 0xb0, 0xac, 0x34, 0xf1, 0xa5, 0xae, 0xf6, 0xab, 0xf2, 0xf0, 0x27, 0xc7, + 0xcd, 0x25, 0x49, 0x1f, 0x18, 0x4b, 0x1c, 0xa7, 0x5b, 0x0e, 0x79, 0x03, 0xc0, 0xa3, 0x02, 0xb9, + 0xb0, 0xf6, 0xa9, 0x77, 0xae, 0xab, 0xcb, 0xb1, 0xfd, 0x29, 0xf5, 0xda, 0x9f, 0xc3, 0x9a, 0xe9, + 0xfa, 0xa1, 0x87, 0x06, 0xf2, 0x90, 0x05, 0x1c, 0xc9, 0x07, 0xb0, 0x84, 0x51, 0xc4, 0x22, 0xe5, + 0xfb, 0xfc, 0x4b, 0xb5, 0xf8, 0xf6, 0x49, 0x9a, 0x11, 0xb3, 0xdb, 0xdf, 0xe5, 0x61, 0x6d, 0x93, + 0x05, 0x7c, 0xe6, 0x63, 0x64, 0xba, 0xe3, 0x80, 0x7a, 0xe4, 0x11, 0x54, 0x9d, 0x88, 0xba, 0x81, + 0x15, 0xc5, 0x8f, 0x79, 0xe2, 0xf1, 0xed, 0xcb, 0x3d, 0x0e, 0x24, 0x3d, 0x79, 0xfa, 0x8d, 0x55, + 0x27, 0xb3, 0x22, 0xcf, 0x80, 0x70, 0x39, 0x1c, 0x2c, 0xf5, 0x9e, 0xa5, 0x1e, 0xf3, 0xca, 0xe3, + 0xad, 0xab, 0x72, 0x7f, 0x7e, 0xa0, 0x18, 0x35, 0xbe, 0x38, 0x62, 0xb6, 0xa0, 0x3c, 0xa1, 0x81, + 0xc3, 0x27, 0xf4, 0x39, 0x26, 0x73, 0xe6, 0x8a, 0x62, 0xa6, 0x31, 0x3e, 0x4c, 0x25, 0xc6, 0x99, + 0xfa, 0xe3, 0xe2, 0xd1, 0xf7, 0x4d, 0xad, 0xbd, 0x06, 0xab, 0xd9, 0x40, 0xda, 0x3f, 0xe4, 0x61, + 0xfd, 0x82, 0x8c, 0xdc, 0x05, 0x62, 0x27, 0x46, 0xd5, 0x8f, 0xce, 0xcc, 0xc3, 0xb8, 0xa0, 0xe9, + 0xf3, 0xbe, 0x9e, 0xe2, 0x66, 0x0a, 0x93, 0x6f, 0xa0, 0x7e, 0x41, 0x64, 0x39, 0x48, 0x1d, 0xcf, + 0x0d, 0x30, 0xc9, 0x46, 0xbd, 0x1b, 0xcf, 0xee, 0x6e, 0x3a, 0xbb, 0xbb, 0xf3, 0x17, 0xac, 0x5f, + 0x7c, 0xf9, 0x6b, 0x53, 0x33, 0xf4, 0x45, 0xc7, 0x83, 0xc4, 0x43, 0x76, 0x60, 0x16, 0xae, 0x37, + 0x30, 0x1f, 0xc2, 0x4d, 0xdf, 0x0d, 0x2c, 0x6a, 0xdb, 0x18, 0x0a, 0x74, 0xac, 0x54, 0x5e, 0xbc, + 0x52, 0x4e, 0x7c, 0x37, 0xb8, 0x9f, 0x48, 0x12, 0xdb, 0xad, 0x7b, 0xb0, 0x7e, 0xe1, 0x9d, 0x20, + 0x2b, 0x50, 0x78, 0x38, 0x7c, 0x56, 0xcb, 0x11, 0x80, 0xe5, 0xa1, 0xb9, 0x79, 0x7f, 0x77, 0x58, + 0xd3, 0xe4, 0x77, 0xff, 0xbe, 0x39, 0xfc, 0xf0, 0x5e, 0x2d, 0xbf, 0xf1, 0x63, 0x1e, 0x56, 0x12, + 0xe7, 0x64, 0x02, 0x15, 0x63, 0x16, 0x98, 0x87, 0x81, 0x2d, 0xeb, 0x4b, 0x3a, 0xff, 0x5c, 0xc5, + 0xb8, 0x53, 0xeb, 0xef, 0x5e, 0x39, 0xdf, 0x9c, 0x99, 0x8d, 0xd1, 0x0e, 0x72, 0x4e, 0xc7, 0xd8, + 0xce, 0x75, 0xb4, 0x3b, 0x1a, 0xb1, 0xa1, 0x3c, 0xef, 0x2c, 0xf2, 0x2f, 0xda, 0xaf, 0x7e, 0xc5, + 0x99, 0xce, 0xdf, 0xc3, 0x76, 0x8e, 0x8c, 0x01, 0xd4, 0x20, 0x17, 0x11, 0x52, 0x9f, 0x5c, 0xff, + 0x8c, 0xf5, 0x6b, 0x07, 0x1e, 0x47, 0xd3, 0x7f, 0xef, 0xe8, 0xf7, 0x46, 0xee, 0xe8, 0xa4, 0xa1, + 0xbd, 0x3a, 0x69, 0x68, 0xaf, 0x4f, 0x1a, 0xda, 0x6f, 0x27, 0x0d, 0xed, 0xe5, 0x69, 0x23, 0xf7, + 0xea, 0xb4, 0x91, 0x7b, 0x7d, 0xda, 0xc8, 0x7d, 0x5d, 0xc9, 0xfc, 0xb9, 0xfd, 0x15, 0x00, 0x00, + 0xff, 0xff, 0xbd, 0xcd, 0x1c, 0x89, 0x67, 0x0a, 0x00, 0x00, } diff --git a/pkg/sql/execinfrapb/api.proto b/pkg/sql/execinfrapb/api.proto index 689ad47fe8d1..9c3e60685fe3 100644 --- a/pkg/sql/execinfrapb/api.proto +++ b/pkg/sql/execinfrapb/api.proto @@ -67,7 +67,8 @@ message EvalContext { // parse timestamps. Used to init EvalContext.Location. optional string location = 4 [(gogoproto.nullable) = false]; optional string database = 5 [(gogoproto.nullable) = false]; - repeated string searchPath = 6; + repeated string search_path = 6; + optional string temporary_schema_name = 13 [(gogoproto.nullable) = false]; optional string user = 7 [(gogoproto.nullable) = false]; optional SequenceState seq_state = 8 [(gogoproto.nullable) = false]; optional string application_name = 9 [(gogoproto.nullable) = false]; diff --git a/pkg/sql/internal.go b/pkg/sql/internal.go index 4f70b08d1fcb..ed39c6f49faa 100644 --- a/pkg/sql/internal.go +++ b/pkg/sql/internal.go @@ -325,6 +325,11 @@ func (ie *InternalExecutor) ExecWithUser( return res.rowsAffected, res.err } +// Settings returns the cluster settings. +func (ie *InternalExecutor) Settings() *cluster.Settings { + return ie.s.cfg.Settings +} + // Query executes the supplied SQL statement and returns the resulting rows. // The statement is executed as the root user. // diff --git a/pkg/sql/lease.go b/pkg/sql/lease.go index 14c5789d856c..23e426ebcea7 100644 --- a/pkg/sql/lease.go +++ b/pkg/sql/lease.go @@ -1571,18 +1571,18 @@ func (m *LeaseManager) AcquireByName( func (m *LeaseManager) resolveName( ctx context.Context, timestamp hlc.Timestamp, dbID sqlbase.ID, tableName string, ) (sqlbase.ID, error) { - key := sqlbase.NewTableKey(dbID, tableName).Key() id := sqlbase.InvalidID if err := m.db.Txn(ctx, func(ctx context.Context, txn *client.Txn) error { txn.SetFixedTimestamp(ctx, timestamp) - gr, err := txn.Get(ctx, key) + var found bool + var err error + found, id, err = sqlbase.LookupPublicTableID(ctx, txn, dbID, tableName) if err != nil { return err } - if !gr.Exists() { + if !found { return nil } - id = sqlbase.ID(gr.ValueInt()) return nil }); err != nil { return id, err diff --git a/pkg/sql/logical_schema_accessors.go b/pkg/sql/logical_schema_accessors.go index 65d70f3a6d19..41d692ab1dde 100644 --- a/pkg/sql/logical_schema_accessors.go +++ b/pkg/sql/logical_schema_accessors.go @@ -14,6 +14,7 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/internal/client" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented" @@ -33,13 +34,15 @@ type LogicalSchemaAccessor struct { var _ SchemaAccessor = &LogicalSchemaAccessor{} // IsValidSchema implements the DatabaseLister interface. -func (l *LogicalSchemaAccessor) IsValidSchema(dbDesc *DatabaseDescriptor, scName string) bool { +func (l *LogicalSchemaAccessor) IsValidSchema( + ctx context.Context, txn *client.Txn, dbID sqlbase.ID, scName string, +) (bool, sqlbase.ID, error) { if _, ok := l.vt.getVirtualSchemaEntry(scName); ok { - return true + return true, sqlbase.InvalidID, nil } // Fallthrough. - return l.SchemaAccessor.IsValidSchema(dbDesc, scName) + return l.SchemaAccessor.IsValidSchema(ctx, txn, dbID, scName) } // GetObjectNames implements the DatabaseLister interface. @@ -68,7 +71,11 @@ func (l *LogicalSchemaAccessor) GetObjectNames( // GetObjectDesc implements the ObjectAccessor interface. func (l *LogicalSchemaAccessor) GetObjectDesc( - ctx context.Context, txn *client.Txn, name *ObjectName, flags tree.ObjectLookupFlags, + ctx context.Context, + txn *client.Txn, + settings *cluster.Settings, + name *ObjectName, + flags tree.ObjectLookupFlags, ) (ObjectDescriptor, error) { if scEntry, ok := l.vt.getVirtualSchemaEntry(name.Schema()); ok { tableName := name.Table() @@ -90,5 +97,5 @@ func (l *LogicalSchemaAccessor) GetObjectDesc( } // Fallthrough. - return l.SchemaAccessor.GetObjectDesc(ctx, txn, name, flags) + return l.SchemaAccessor.GetObjectDesc(ctx, txn, settings, name, flags) } diff --git a/pkg/sql/logictest/testdata/logic_test/crdb_internal b/pkg/sql/logictest/testdata/logic_test/crdb_internal index d5cac99fbe56..3caf0232f0f9 100644 --- a/pkg/sql/logictest/testdata/logic_test/crdb_internal +++ b/pkg/sql/logictest/testdata/logic_test/crdb_internal @@ -76,7 +76,7 @@ query IITTITTTTTTT colnames SELECT table_id, parent_id, name, database_name, version, format_version, state, sc_lease_node_id, sc_lease_expiration_time, drop_time, audit_mode, schema_name FROM crdb_internal.tables WHERE NAME = 'namespace' ---- table_id parent_id name database_name version format_version state sc_lease_node_id sc_lease_expiration_time drop_time audit_mode schema_name -2 1 namespace system 1 InterleavedFormatVersion PUBLIC NULL NULL NULL DISABLED public +30 1 namespace system 1 InterleavedFormatVersion PUBLIC NULL NULL NULL DISABLED public # Verify that table names are not double escaped. diff --git a/pkg/sql/logictest/testdata/logic_test/grant_table b/pkg/sql/logictest/testdata/logic_test/grant_table index 56f92c2a68f9..9ae5b03fef95 100644 --- a/pkg/sql/logictest/testdata/logic_test/grant_table +++ b/pkg/sql/logictest/testdata/logic_test/grant_table @@ -162,23 +162,10 @@ SELECT * FROM [SHOW GRANTS] WHERE schema_name NOT IN ('crdb_internal', 'pg_catalog', 'information_schema') ---- database_name schema_name table_name grantee privilege_type -system public NULL admin GRANT -system public NULL admin SELECT -system public NULL root GRANT -system public NULL root SELECT -defaultdb public NULL admin ALL -defaultdb public NULL root ALL -postgres public NULL admin ALL -postgres public NULL root ALL -test public NULL admin ALL -test public NULL root ALL -a public NULL admin ALL -a public NULL readwrite ALL -a public NULL root ALL -system public namespace admin GRANT -system public namespace admin SELECT -system public namespace root GRANT -system public namespace root SELECT +system public namespace_deprecated admin GRANT +system public namespace_deprecated admin SELECT +system public namespace_deprecated root GRANT +system public namespace_deprecated root SELECT system public descriptor admin GRANT system public descriptor admin SELECT system public descriptor root GRANT @@ -358,6 +345,23 @@ system public reports_meta root GRANT system public reports_meta root INSERT system public reports_meta root SELECT system public reports_meta root UPDATE +system public namespace admin GRANT +system public namespace admin SELECT +system public namespace root GRANT +system public namespace root SELECT +a public NULL admin ALL +a public NULL readwrite ALL +a public NULL root ALL +defaultdb public NULL admin ALL +defaultdb public NULL root ALL +postgres public NULL admin ALL +postgres public NULL root ALL +system public NULL admin GRANT +system public NULL admin SELECT +system public NULL root GRANT +system public NULL root SELECT +test public NULL admin ALL +test public NULL root ALL query TTTTT colnames SHOW GRANTS FOR root @@ -412,6 +416,8 @@ system public locations root SEL system public locations root UPDATE system public namespace root GRANT system public namespace root SELECT +system public namespace_deprecated root GRANT +system public namespace_deprecated root SELECT system public rangelog root DELETE system public rangelog root GRANT system public rangelog root INSERT diff --git a/pkg/sql/logictest/testdata/logic_test/information_schema b/pkg/sql/logictest/testdata/logic_test/information_schema index 972b9d6798c0..a2d18d86fb24 100644 --- a/pkg/sql/logictest/testdata/logic_test/information_schema +++ b/pkg/sql/logictest/testdata/logic_test/information_schema @@ -602,7 +602,7 @@ system pg_catalog pg_type SYSTEM VIE system pg_catalog pg_user SYSTEM VIEW NO 1 system pg_catalog pg_user_mapping SYSTEM VIEW NO 1 system pg_catalog pg_views SYSTEM VIEW NO 1 -system public namespace BASE TABLE YES 1 +system public namespace_deprecated BASE TABLE YES 1 system public descriptor BASE TABLE YES 1 system public users BASE TABLE YES 1 system public zones BASE TABLE YES 1 @@ -621,6 +621,7 @@ system public replication_constraint_stats BASE TABLE system public replication_critical_localities BASE TABLE YES 1 system public replication_stats BASE TABLE YES 1 system public reports_meta BASE TABLE YES 1 +system public namespace BASE TABLE YES 1 statement ok ALTER TABLE other_db.xyz ADD COLUMN j INT @@ -682,6 +683,7 @@ system public primary system public system public primary system public lease PRIMARY KEY NO NO system public primary system public locations PRIMARY KEY NO NO system public primary system public namespace PRIMARY KEY NO NO +system public primary system public namespace_deprecated PRIMARY KEY NO NO system public primary system public rangelog PRIMARY KEY NO NO system public primary system public replication_constraint_stats PRIMARY KEY NO NO system public primary system public replication_critical_localities PRIMARY KEY NO NO @@ -767,6 +769,9 @@ system public 630200280_28_1_not_null id IS NOT NULL system public 630200280_28_2_not_null generated IS NOT NULL system public 630200280_2_1_not_null parentID IS NOT NULL system public 630200280_2_2_not_null name IS NOT NULL +system public 630200280_30_1_not_null parentID IS NOT NULL +system public 630200280_30_2_not_null parentSchemaID IS NOT NULL +system public 630200280_30_3_not_null name IS NOT NULL system public 630200280_3_1_not_null id IS NOT NULL system public 630200280_4_1_not_null username IS NOT NULL system public 630200280_4_3_not_null isRole IS NOT NULL @@ -780,43 +785,46 @@ SELECT * FROM system.information_schema.constraint_column_usage ORDER BY TABLE_NAME, COLUMN_NAME, CONSTRAINT_NAME ---- -table_catalog table_schema table_name column_name constraint_catalog constraint_schema constraint_name -system public comments object_id system public primary -system public comments sub_id system public primary -system public comments type system public primary -system public descriptor id system public primary -system public eventlog timestamp system public primary -system public eventlog uniqueID system public primary -system public jobs id system public primary -system public lease descID system public primary -system public lease expiration system public primary -system public lease nodeID system public primary -system public lease version system public primary -system public locations localityKey system public primary -system public locations localityValue system public primary -system public namespace name system public primary -system public namespace parentID system public primary -system public rangelog timestamp system public primary -system public rangelog uniqueID system public primary -system public replication_constraint_stats config system public primary -system public replication_constraint_stats subzone_id system public primary -system public replication_constraint_stats type system public primary -system public replication_constraint_stats zone_id system public primary -system public replication_critical_localities locality system public primary -system public replication_critical_localities subzone_id system public primary -system public replication_critical_localities zone_id system public primary -system public replication_stats subzone_id system public primary -system public replication_stats zone_id system public primary -system public reports_meta id system public primary -system public role_members member system public primary -system public role_members role system public primary -system public settings name system public primary -system public table_statistics statisticID system public primary -system public table_statistics tableID system public primary -system public ui key system public primary -system public users username system public primary -system public web_sessions id system public primary -system public zones id system public primary +table_catalog table_schema table_name column_name constraint_catalog constraint_schema constraint_name +system public comments object_id system public primary +system public comments sub_id system public primary +system public comments type system public primary +system public descriptor id system public primary +system public eventlog timestamp system public primary +system public eventlog uniqueID system public primary +system public jobs id system public primary +system public lease descID system public primary +system public lease expiration system public primary +system public lease nodeID system public primary +system public lease version system public primary +system public locations localityKey system public primary +system public locations localityValue system public primary +system public namespace name system public primary +system public namespace parentID system public primary +system public namespace parentSchemaID system public primary +system public namespace_deprecated name system public primary +system public namespace_deprecated parentID system public primary +system public rangelog timestamp system public primary +system public rangelog uniqueID system public primary +system public replication_constraint_stats config system public primary +system public replication_constraint_stats subzone_id system public primary +system public replication_constraint_stats type system public primary +system public replication_constraint_stats zone_id system public primary +system public replication_critical_localities locality system public primary +system public replication_critical_localities subzone_id system public primary +system public replication_critical_localities zone_id system public primary +system public replication_stats subzone_id system public primary +system public replication_stats zone_id system public primary +system public reports_meta id system public primary +system public role_members member system public primary +system public role_members role system public primary +system public settings name system public primary +system public table_statistics statisticID system public primary +system public table_statistics tableID system public primary +system public ui key system public primary +system public users username system public primary +system public web_sessions id system public primary +system public zones id system public primary statement ok CREATE DATABASE constraint_db @@ -923,9 +931,13 @@ system public locations latitude system public locations localityKey 1 system public locations localityValue 2 system public locations longitude 4 -system public namespace id 3 -system public namespace name 2 +system public namespace id 4 +system public namespace name 3 system public namespace parentID 1 +system public namespace parentSchemaID 2 +system public namespace_deprecated id 3 +system public namespace_deprecated name 2 +system public namespace_deprecated parentID 1 system public rangelog eventType 4 system public rangelog info 6 system public rangelog otherRangeID 5 @@ -1467,6 +1479,10 @@ NULL admin system public namespace NULL admin system public namespace SELECT NULL YES NULL root system public namespace GRANT NULL NO NULL root system public namespace SELECT NULL YES +NULL admin system public namespace_deprecated GRANT NULL NO +NULL admin system public namespace_deprecated SELECT NULL YES +NULL root system public namespace_deprecated GRANT NULL NO +NULL root system public namespace_deprecated SELECT NULL YES NULL admin system public rangelog DELETE NULL NO NULL admin system public rangelog GRANT NULL NO NULL admin system public rangelog INSERT NULL NO @@ -1696,10 +1712,10 @@ NULL public system pg_catalog pg_type NULL public system pg_catalog pg_user SELECT NULL YES NULL public system pg_catalog pg_user_mapping SELECT NULL YES NULL public system pg_catalog pg_views SELECT NULL YES -NULL admin system public namespace GRANT NULL NO -NULL admin system public namespace SELECT NULL YES -NULL root system public namespace GRANT NULL NO -NULL root system public namespace SELECT NULL YES +NULL admin system public namespace_deprecated GRANT NULL NO +NULL admin system public namespace_deprecated SELECT NULL YES +NULL root system public namespace_deprecated GRANT NULL NO +NULL root system public namespace_deprecated SELECT NULL YES NULL admin system public descriptor GRANT NULL NO NULL admin system public descriptor SELECT NULL YES NULL root system public descriptor GRANT NULL NO @@ -1879,6 +1895,10 @@ NULL root system public reports_meta NULL root system public reports_meta INSERT NULL NO NULL root system public reports_meta SELECT NULL YES NULL root system public reports_meta UPDATE NULL NO +NULL admin system public namespace GRANT NULL NO +NULL admin system public namespace SELECT NULL YES +NULL root system public namespace GRANT NULL NO +NULL root system public namespace SELECT NULL YES statement ok CREATE TABLE other_db.xyz (i INT) diff --git a/pkg/sql/logictest/testdata/logic_test/pg_catalog b/pkg/sql/logictest/testdata/logic_test/pg_catalog index c3a6d9266139..d15c19d4e98e 100644 --- a/pkg/sql/logictest/testdata/logic_test/pg_catalog +++ b/pkg/sql/logictest/testdata/logic_test/pg_catalog @@ -641,6 +641,7 @@ indexrelid indrelid indnatts indisunique indisprimary indisexclusion indim 3706522183 11 4 true true false true false true false false true false 1 2 4 3 0 0 0 0 0 0 0 0 2 2 2 2 NULL NULL 3752917847 27 2 true true false true false true false false true false 1 2 0 0 0 0 2 2 NULL NULL 3966258450 14 1 true true false true false true false false true false 1 3903121477 0 2 NULL NULL +4012654114 30 3 true true false true false true false false true false 1 2 3 0 0 3903121477 0 0 0 2 2 2 NULL NULL 4225994721 13 2 true true false true false true false false true false 1 7 0 0 0 0 2 2 NULL NULL # From #26504 @@ -692,6 +693,9 @@ indexrelid operator_argument_type_oid operator_argument_position 3752917847 0 1 3752917847 0 2 3966258450 0 1 +4012654114 0 1 +4012654114 0 2 +4012654114 0 3 4225994721 0 1 4225994721 0 2 diff --git a/pkg/sql/logictest/testdata/logic_test/ranges b/pkg/sql/logictest/testdata/logic_test/ranges index e601c970912e..94824a7e854b 100644 --- a/pkg/sql/logictest/testdata/logic_test/ranges +++ b/pkg/sql/logictest/testdata/logic_test/ranges @@ -298,7 +298,9 @@ start_key start_pretty end_key [161] /Table/25 [162] /Table/26 system replication_constraint_stats · {1} 1 [162] /Table/26 [163] /Table/27 system replication_critical_localities · {1} 1 [163] /Table/27 [164] /Table/28 system replication_stats · {1} 1 -[164] /Table/28 [189 137] /Table/53/1 system reports_meta · {1} 1 +[164] /Table/28 [165] /Table/29 system reports_meta · {1} 1 +[165] /Table/29 [166] /NamespaceTable/30 · · · {1} 1 +[166] /NamespaceTable/30 [189 137] /Table/53/1 system namespace · {1} 1 [189 137] /Table/53/1 [189 137 137] /Table/53/1/1 test t · {1} 1 [189 137 137] /Table/53/1/1 [189 137 141 137] /Table/53/1/5/1 test t · {3,4} 3 [189 137 141 137] /Table/53/1/5/1 [189 137 141 138] /Table/53/1/5/2 test t · {1,2,3} 1 @@ -345,7 +347,9 @@ start_key start_pretty end_key [161] /Table/25 [162] /Table/26 system replication_constraint_stats · {1} 1 [162] /Table/26 [163] /Table/27 system replication_critical_localities · {1} 1 [163] /Table/27 [164] /Table/28 system replication_stats · {1} 1 -[164] /Table/28 [189 137] /Table/53/1 system reports_meta · {1} 1 +[164] /Table/28 [165] /Table/29 system reports_meta · {1} 1 +[165] /Table/29 [166] /NamespaceTable/30 · · · {1} 1 +[166] /NamespaceTable/30 [189 137] /Table/53/1 system namespace · {1} 1 [189 137] /Table/53/1 [189 137 137] /Table/53/1/1 test t · {1} 1 [189 137 137] /Table/53/1/1 [189 137 141 137] /Table/53/1/5/1 test t · {3,4} 3 [189 137 141 137] /Table/53/1/5/1 [189 137 141 138] /Table/53/1/5/2 test t · {1,2,3} 1 @@ -515,4 +519,3 @@ SELECT crdb_internal.pretty_key(crdb_internal.encode_key(70, 4, (1, )), 0) ---- /70/4/1/0 - diff --git a/pkg/sql/logictest/testdata/logic_test/show_source b/pkg/sql/logictest/testdata/logic_test/show_source index 1d54cdbe0b7a..67b1bccf7761 100644 --- a/pkg/sql/logictest/testdata/logic_test/show_source +++ b/pkg/sql/logictest/testdata/logic_test/show_source @@ -173,7 +173,7 @@ query T colnames,rowsort SELECT * FROM [SHOW TABLES FROM system] ---- table_name -namespace +namespace_deprecated descriptor users zones @@ -192,12 +192,13 @@ replication_constraint_stats replication_critical_localities replication_stats reports_meta +namespace query TT colnames,rowsort SELECT * FROM [SHOW TABLES FROM system WITH COMMENT] ---- table_name comment -namespace · +namespace_deprecated · descriptor · users · zones · @@ -216,6 +217,7 @@ replication_constraint_stats · replication_critical_localities · replication_stats · reports_meta · +namespace · query ITTT colnames SELECT node_id, user_name, application_name, active_queries diff --git a/pkg/sql/logictest/testdata/logic_test/system b/pkg/sql/logictest/testdata/logic_test/system index 36b9cd44fc1f..fd0056723442 100644 --- a/pkg/sql/logictest/testdata/logic_test/system +++ b/pkg/sql/logictest/testdata/logic_test/system @@ -16,6 +16,7 @@ jobs lease locations namespace +namespace_deprecated rangelog replication_constraint_stats replication_critical_localities @@ -29,33 +30,6 @@ users web_sessions zones -query ITI rowsort -SELECT * FROM system.namespace ----- -0 defaultdb 50 -0 postgres 51 -0 system 1 -0 test 52 -1 comments 24 -1 descriptor 3 -1 eventlog 12 -1 jobs 15 -1 lease 11 -1 locations 21 -1 namespace 2 -1 rangelog 13 -1 replication_constraint_stats 25 -1 replication_critical_localities 26 -1 replication_stats 27 -1 reports_meta 28 -1 role_members 23 -1 settings 6 -1 table_statistics 20 -1 ui 14 -1 users 4 -1 web_sessions 19 -1 zones 5 - query I rowsort SELECT id FROM system.descriptor ---- @@ -79,6 +53,7 @@ SELECT id FROM system.descriptor 26 27 28 +30 50 51 52 @@ -89,13 +64,6 @@ SELECT length(descriptor) * (id - 1) FROM system.descriptor WHERE id = 1 ---- 0 -# Verify format of system tables. -query TTBTTTB -SHOW COLUMNS FROM system.namespace ----- -parentID INT8 false NULL · {primary} false -name STRING false NULL · {primary} false -id INT8 true NULL · {} false query TTBTTTB SHOW COLUMNS FROM system.descriptor @@ -201,189 +169,193 @@ system public root SELECT query TTTTT SHOW GRANTS ON system.* ---- -system public comments admin DELETE -system public comments admin GRANT -system public comments admin INSERT -system public comments admin SELECT -system public comments admin UPDATE -system public comments public DELETE -system public comments public GRANT -system public comments public INSERT -system public comments public SELECT -system public comments public UPDATE -system public comments root DELETE -system public comments root GRANT -system public comments root INSERT -system public comments root SELECT -system public comments root UPDATE -system public descriptor admin GRANT -system public descriptor admin SELECT -system public descriptor root GRANT -system public descriptor root SELECT -system public eventlog admin DELETE -system public eventlog admin GRANT -system public eventlog admin INSERT -system public eventlog admin SELECT -system public eventlog admin UPDATE -system public eventlog root DELETE -system public eventlog root GRANT -system public eventlog root INSERT -system public eventlog root SELECT -system public eventlog root UPDATE -system public jobs admin DELETE -system public jobs admin GRANT -system public jobs admin INSERT -system public jobs admin SELECT -system public jobs admin UPDATE -system public jobs root DELETE -system public jobs root GRANT -system public jobs root INSERT -system public jobs root SELECT -system public jobs root UPDATE -system public lease admin DELETE -system public lease admin GRANT -system public lease admin INSERT -system public lease admin SELECT -system public lease admin UPDATE -system public lease root DELETE -system public lease root GRANT -system public lease root INSERT -system public lease root SELECT -system public lease root UPDATE -system public locations admin DELETE -system public locations admin GRANT -system public locations admin INSERT -system public locations admin SELECT -system public locations admin UPDATE -system public locations root DELETE -system public locations root GRANT -system public locations root INSERT -system public locations root SELECT -system public locations root UPDATE -system public namespace admin GRANT -system public namespace admin SELECT -system public namespace root GRANT -system public namespace root SELECT -system public rangelog admin DELETE -system public rangelog admin GRANT -system public rangelog admin INSERT -system public rangelog admin SELECT -system public rangelog admin UPDATE -system public rangelog root DELETE -system public rangelog root GRANT -system public rangelog root INSERT -system public rangelog root SELECT -system public rangelog root UPDATE -system public replication_constraint_stats admin DELETE -system public replication_constraint_stats admin GRANT -system public replication_constraint_stats admin INSERT -system public replication_constraint_stats admin SELECT -system public replication_constraint_stats admin UPDATE -system public replication_constraint_stats root DELETE -system public replication_constraint_stats root GRANT -system public replication_constraint_stats root INSERT -system public replication_constraint_stats root SELECT -system public replication_constraint_stats root UPDATE -system public replication_critical_localities admin DELETE -system public replication_critical_localities admin GRANT -system public replication_critical_localities admin INSERT -system public replication_critical_localities admin SELECT -system public replication_critical_localities admin UPDATE -system public replication_critical_localities root DELETE -system public replication_critical_localities root GRANT -system public replication_critical_localities root INSERT -system public replication_critical_localities root SELECT -system public replication_critical_localities root UPDATE -system public replication_stats admin DELETE -system public replication_stats admin GRANT -system public replication_stats admin INSERT -system public replication_stats admin SELECT -system public replication_stats admin UPDATE -system public replication_stats root DELETE -system public replication_stats root GRANT -system public replication_stats root INSERT -system public replication_stats root SELECT -system public replication_stats root UPDATE -system public reports_meta admin DELETE -system public reports_meta admin GRANT -system public reports_meta admin INSERT -system public reports_meta admin SELECT -system public reports_meta admin UPDATE -system public reports_meta root DELETE -system public reports_meta root GRANT -system public reports_meta root INSERT -system public reports_meta root SELECT -system public reports_meta root UPDATE -system public role_members admin DELETE -system public role_members admin GRANT -system public role_members admin INSERT -system public role_members admin SELECT -system public role_members admin UPDATE -system public role_members root DELETE -system public role_members root GRANT -system public role_members root INSERT -system public role_members root SELECT -system public role_members root UPDATE -system public settings admin DELETE -system public settings admin GRANT -system public settings admin INSERT -system public settings admin SELECT -system public settings admin UPDATE -system public settings root DELETE -system public settings root GRANT -system public settings root INSERT -system public settings root SELECT -system public settings root UPDATE -system public table_statistics admin DELETE -system public table_statistics admin GRANT -system public table_statistics admin INSERT -system public table_statistics admin SELECT -system public table_statistics admin UPDATE -system public table_statistics root DELETE -system public table_statistics root GRANT -system public table_statistics root INSERT -system public table_statistics root SELECT -system public table_statistics root UPDATE -system public ui admin DELETE -system public ui admin GRANT -system public ui admin INSERT -system public ui admin SELECT -system public ui admin UPDATE -system public ui root DELETE -system public ui root GRANT -system public ui root INSERT -system public ui root SELECT -system public ui root UPDATE -system public users admin DELETE -system public users admin GRANT -system public users admin INSERT -system public users admin SELECT -system public users admin UPDATE -system public users root DELETE -system public users root GRANT -system public users root INSERT -system public users root SELECT -system public users root UPDATE -system public web_sessions admin DELETE -system public web_sessions admin GRANT -system public web_sessions admin INSERT -system public web_sessions admin SELECT -system public web_sessions admin UPDATE -system public web_sessions root DELETE -system public web_sessions root GRANT -system public web_sessions root INSERT -system public web_sessions root SELECT -system public web_sessions root UPDATE -system public zones admin DELETE -system public zones admin GRANT -system public zones admin INSERT -system public zones admin SELECT -system public zones admin UPDATE -system public zones root DELETE -system public zones root GRANT -system public zones root INSERT -system public zones root SELECT -system public zones root UPDATE +system public comments admin DELETE +system public comments admin GRANT +system public comments admin INSERT +system public comments admin SELECT +system public comments admin UPDATE +system public comments public DELETE +system public comments public GRANT +system public comments public INSERT +system public comments public SELECT +system public comments public UPDATE +system public comments root DELETE +system public comments root GRANT +system public comments root INSERT +system public comments root SELECT +system public comments root UPDATE +system public descriptor admin GRANT +system public descriptor admin SELECT +system public descriptor root GRANT +system public descriptor root SELECT +system public eventlog admin DELETE +system public eventlog admin GRANT +system public eventlog admin INSERT +system public eventlog admin SELECT +system public eventlog admin UPDATE +system public eventlog root DELETE +system public eventlog root GRANT +system public eventlog root INSERT +system public eventlog root SELECT +system public eventlog root UPDATE +system public jobs admin DELETE +system public jobs admin GRANT +system public jobs admin INSERT +system public jobs admin SELECT +system public jobs admin UPDATE +system public jobs root DELETE +system public jobs root GRANT +system public jobs root INSERT +system public jobs root SELECT +system public jobs root UPDATE +system public lease admin DELETE +system public lease admin GRANT +system public lease admin INSERT +system public lease admin SELECT +system public lease admin UPDATE +system public lease root DELETE +system public lease root GRANT +system public lease root INSERT +system public lease root SELECT +system public lease root UPDATE +system public locations admin DELETE +system public locations admin GRANT +system public locations admin INSERT +system public locations admin SELECT +system public locations admin UPDATE +system public locations root DELETE +system public locations root GRANT +system public locations root INSERT +system public locations root SELECT +system public locations root UPDATE +system public namespace admin GRANT +system public namespace admin SELECT +system public namespace root GRANT +system public namespace root SELECT +system public namespace_deprecated admin GRANT +system public namespace_deprecated admin SELECT +system public namespace_deprecated root GRANT +system public namespace_deprecated root SELECT +system public rangelog admin DELETE +system public rangelog admin GRANT +system public rangelog admin INSERT +system public rangelog admin SELECT +system public rangelog admin UPDATE +system public rangelog root DELETE +system public rangelog root GRANT +system public rangelog root INSERT +system public rangelog root SELECT +system public rangelog root UPDATE +system public replication_constraint_stats admin DELETE +system public replication_constraint_stats admin GRANT +system public replication_constraint_stats admin INSERT +system public replication_constraint_stats admin SELECT +system public replication_constraint_stats admin UPDATE +system public replication_constraint_stats root DELETE +system public replication_constraint_stats root GRANT +system public replication_constraint_stats root INSERT +system public replication_constraint_stats root SELECT +system public replication_constraint_stats root UPDATE +system public replication_critical_localities admin DELETE +system public replication_critical_localities admin GRANT +system public replication_critical_localities admin INSERT +system public replication_critical_localities admin SELECT +system public replication_critical_localities admin UPDATE +system public replication_critical_localities root DELETE +system public replication_critical_localities root GRANT +system public replication_critical_localities root INSERT +system public replication_critical_localities root SELECT +system public replication_critical_localities root UPDATE +system public replication_stats admin DELETE +system public replication_stats admin GRANT +system public replication_stats admin INSERT +system public replication_stats admin SELECT +system public replication_stats admin UPDATE +system public replication_stats root DELETE +system public replication_stats root GRANT +system public replication_stats root INSERT +system public replication_stats root SELECT +system public replication_stats root UPDATE +system public reports_meta admin DELETE +system public reports_meta admin GRANT +system public reports_meta admin INSERT +system public reports_meta admin SELECT +system public reports_meta admin UPDATE +system public reports_meta root DELETE +system public reports_meta root GRANT +system public reports_meta root INSERT +system public reports_meta root SELECT +system public reports_meta root UPDATE +system public role_members admin DELETE +system public role_members admin GRANT +system public role_members admin INSERT +system public role_members admin SELECT +system public role_members admin UPDATE +system public role_members root DELETE +system public role_members root GRANT +system public role_members root INSERT +system public role_members root SELECT +system public role_members root UPDATE +system public settings admin DELETE +system public settings admin GRANT +system public settings admin INSERT +system public settings admin SELECT +system public settings admin UPDATE +system public settings root DELETE +system public settings root GRANT +system public settings root INSERT +system public settings root SELECT +system public settings root UPDATE +system public table_statistics admin DELETE +system public table_statistics admin GRANT +system public table_statistics admin INSERT +system public table_statistics admin SELECT +system public table_statistics admin UPDATE +system public table_statistics root DELETE +system public table_statistics root GRANT +system public table_statistics root INSERT +system public table_statistics root SELECT +system public table_statistics root UPDATE +system public ui admin DELETE +system public ui admin GRANT +system public ui admin INSERT +system public ui admin SELECT +system public ui admin UPDATE +system public ui root DELETE +system public ui root GRANT +system public ui root INSERT +system public ui root SELECT +system public ui root UPDATE +system public users admin DELETE +system public users admin GRANT +system public users admin INSERT +system public users admin SELECT +system public users admin UPDATE +system public users root DELETE +system public users root GRANT +system public users root INSERT +system public users root SELECT +system public users root UPDATE +system public web_sessions admin DELETE +system public web_sessions admin GRANT +system public web_sessions admin INSERT +system public web_sessions admin SELECT +system public web_sessions admin UPDATE +system public web_sessions root DELETE +system public web_sessions root GRANT +system public web_sessions root INSERT +system public web_sessions root SELECT +system public web_sessions root UPDATE +system public zones admin DELETE +system public zones admin GRANT +system public zones admin INSERT +system public zones admin SELECT +system public zones admin UPDATE +system public zones root DELETE +system public zones root GRANT +system public zones root INSERT +system public zones root SELECT +system public zones root UPDATE statement error user root does not have DROP privilege on database system ALTER DATABASE system RENAME TO not_system diff --git a/pkg/sql/logictest/testdata/logic_test/system_namespace b/pkg/sql/logictest/testdata/logic_test/system_namespace new file mode 100644 index 000000000000..8cb37a22935b --- /dev/null +++ b/pkg/sql/logictest/testdata/logic_test/system_namespace @@ -0,0 +1,44 @@ +# LogicTest: local local-vec-off fakedist fakedist-vec-off fakedist-disk fakedist-metadata +# The local-mixed-19.1-19.2 configuration is excluded for this file because system.namespace +# changed in 20.1 + +query IITI rowsort +SELECT * FROM system.namespace +---- +0 0 system 1 +1 0 public 29 +1 29 comments 24 +1 29 jobs 15 +1 29 lease 11 +1 29 locations 21 +1 29 namespace 30 +1 29 namespace_deprecated 2 +1 29 rangelog 13 +1 29 replication_constraint_stats 25 +1 29 replication_critical_localities 26 +1 29 settings 6 +1 29 table_statistics 20 +1 29 ui 14 +1 29 zones 5 +50 0 public 29 +51 0 public 29 +52 0 public 29 +0 0 defaultdb 50 +0 0 test 52 +1 29 users 4 +1 29 web_sessions 19 +0 0 postgres 51 +1 29 descriptor 3 +1 29 eventlog 12 +1 29 replication_stats 27 +1 29 reports_meta 28 +1 29 role_members 23 + +# Verify format of system tables. +query TTBTTTB +SHOW COLUMNS FROM system.namespace +---- +parentID INT8 false NULL · {primary} false +parentSchemaID INT8 false NULL · {primary} false +name STRING false NULL · {primary} false +id INT8 true NULL · {} false diff --git a/pkg/sql/logictest/testdata/logic_test/system_namespace_deprecated b/pkg/sql/logictest/testdata/logic_test/system_namespace_deprecated new file mode 100644 index 000000000000..01c007de9fa1 --- /dev/null +++ b/pkg/sql/logictest/testdata/logic_test/system_namespace_deprecated @@ -0,0 +1,41 @@ +# This should test the local-mixed-19.2-20.1 configuration because system.namespace +# changed in 20.1. +# TODO(whomever): #42931 uncomment these tests and only run them for +# local-mixed-19.2-20.1 configuration. + +# query ITI rowsort +# SELECT * FROM system.namespace +# ---- +# 0 defaultdb 50 +# 0 postgres 51 +# 0 system 1 +# 0 test 52 +# 1 comments 24 +# 1 descriptor 3 +# 1 eventlog 12 +# 1 jobs 15 +# 1 lease 11 +# 1 locations 21 +# 1 namespace 30 +# 1 namespace_deprecated 2 +# 1 rangelog 13 +# 1 replication_constraint_stats 25 +# 1 replication_critical_localities 26 +# 1 replication_stats 27 +# 1 reports_meta 28 +# 1 role_members 23 +# 1 settings 6 +# 1 table_statistics 20 +# 1 ui 14 +# 1 users 4 +# 1 web_sessions 19 +# 1 zones 5 + +# # Verify format of system namespace. +# query TTBTTTB +# SHOW COLUMNS FROM system.namespace +# ---- +# parentID INT8 false NULL · {primary} false +# parentSchemaID INT8 false NULL · {primary} false +# name STRING false NULL · {primary} false +# id INT8 true NULL · {} false diff --git a/pkg/sql/logictest/testdata/logic_test/vectorize b/pkg/sql/logictest/testdata/logic_test/vectorize index ea3eaf05e991..bdbb57ab38d8 100644 --- a/pkg/sql/logictest/testdata/logic_test/vectorize +++ b/pkg/sql/logictest/testdata/logic_test/vectorize @@ -558,10 +558,10 @@ SELECT "hashedPassword" FROM system.users LIMIT 1 ---- · -query ITI +query IITI SELECT * FROM system.namespace LIMIT 1 ---- -0 defaultdb 50 +0 0 defaultdb 50 # Regression test for issue with fetching from unique indexes with embedded # nulls. diff --git a/pkg/sql/opt/exec/execbuilder/testdata/autocommit b/pkg/sql/opt/exec/execbuilder/testdata/autocommit index a51e2f2c4e56..291c380522eb 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/autocommit +++ b/pkg/sql/opt/exec/execbuilder/testdata/autocommit @@ -37,7 +37,7 @@ WHERE message LIKE '%sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r24: sending batch 1 CPut, 1 EndTxn to (n1,s1):1 +dist sender send r26: sending batch 1 CPut, 1 EndTxn to (n1,s1):1 # Multi-row insert should auto-commit. query B @@ -58,7 +58,7 @@ WHERE message LIKE '%sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r24: sending batch 2 CPut, 1 EndTxn to (n1,s1):1 +dist sender send r26: sending batch 2 CPut, 1 EndTxn to (n1,s1):1 # No auto-commit inside a transaction. statement ok @@ -82,7 +82,7 @@ WHERE message LIKE '%sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r24: sending batch 2 CPut to (n1,s1):1 +dist sender send r26: sending batch 2 CPut to (n1,s1):1 statement ok ROLLBACK @@ -106,7 +106,7 @@ WHERE message LIKE '%sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r24: sending batch 2 CPut, 1 EndTxn to (n1,s1):1 +dist sender send r26: sending batch 2 CPut, 1 EndTxn to (n1,s1):1 # TODO(radu): allow non-side-effecting projections. query B @@ -128,8 +128,8 @@ WHERE message LIKE '%sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r24: sending batch 2 CPut to (n1,s1):1 -dist sender send r24: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r26: sending batch 2 CPut to (n1,s1):1 +dist sender send r26: sending batch 1 EndTxn to (n1,s1):1 # Insert with RETURNING statement with side-effects should not auto-commit. # In this case division can (in principle) error out. @@ -152,8 +152,8 @@ WHERE message LIKE '%sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r24: sending batch 2 CPut to (n1,s1):1 -dist sender send r24: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r26: sending batch 2 CPut to (n1,s1):1 +dist sender send r26: sending batch 1 EndTxn to (n1,s1):1 # Another way to test the scenario above: generate an error and ensure that the # mutation was not committed. @@ -188,7 +188,7 @@ WHERE message LIKE '%sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r24: sending batch 1 Put, 1 EndTxn to (n1,s1):1 +dist sender send r26: sending batch 1 Put, 1 EndTxn to (n1,s1):1 # Multi-row upsert should auto-commit. query B @@ -209,7 +209,7 @@ WHERE message LIKE '%sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r24: sending batch 2 Put, 1 EndTxn to (n1,s1):1 +dist sender send r26: sending batch 2 Put, 1 EndTxn to (n1,s1):1 # No auto-commit inside a transaction. statement ok @@ -233,7 +233,7 @@ WHERE message LIKE '%sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r24: sending batch 2 Put to (n1,s1):1 +dist sender send r26: sending batch 2 Put to (n1,s1):1 statement ok ROLLBACK @@ -257,7 +257,7 @@ WHERE message LIKE '%sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r24: sending batch 2 Put, 1 EndTxn to (n1,s1):1 +dist sender send r26: sending batch 2 Put, 1 EndTxn to (n1,s1):1 # TODO(radu): allow non-side-effecting projections. query B @@ -279,8 +279,8 @@ WHERE message LIKE '%sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r24: sending batch 2 Put to (n1,s1):1 -dist sender send r24: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r26: sending batch 2 Put to (n1,s1):1 +dist sender send r26: sending batch 1 EndTxn to (n1,s1):1 # Upsert with RETURNING statement with side-effects should not auto-commit. # In this case division can (in principle) error out. @@ -303,8 +303,8 @@ WHERE message LIKE '%sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r24: sending batch 2 Put to (n1,s1):1 -dist sender send r24: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r26: sending batch 2 Put to (n1,s1):1 +dist sender send r26: sending batch 1 EndTxn to (n1,s1):1 # Another way to test the scenario above: generate an error and ensure that the # mutation was not committed. @@ -339,8 +339,8 @@ WHERE message LIKE '%sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r24: sending batch 1 Scan to (n1,s1):1 -dist sender send r24: sending batch 2 Put, 1 EndTxn to (n1,s1):1 +dist sender send r26: sending batch 1 Scan to (n1,s1):1 +dist sender send r26: sending batch 2 Put, 1 EndTxn to (n1,s1):1 # No auto-commit inside a transaction. statement ok @@ -364,8 +364,8 @@ WHERE message LIKE '%sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r24: sending batch 1 Scan to (n1,s1):1 -dist sender send r24: sending batch 2 Put to (n1,s1):1 +dist sender send r26: sending batch 1 Scan to (n1,s1):1 +dist sender send r26: sending batch 2 Put to (n1,s1):1 statement ok ROLLBACK @@ -389,8 +389,8 @@ WHERE message LIKE '%sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r24: sending batch 1 Scan to (n1,s1):1 -dist sender send r24: sending batch 2 Put, 1 EndTxn to (n1,s1):1 +dist sender send r26: sending batch 1 Scan to (n1,s1):1 +dist sender send r26: sending batch 2 Put, 1 EndTxn to (n1,s1):1 # TODO(radu): allow non-side-effecting projections. query B @@ -412,9 +412,9 @@ WHERE message LIKE '%sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r24: sending batch 1 Scan to (n1,s1):1 -dist sender send r24: sending batch 2 Put to (n1,s1):1 -dist sender send r24: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r26: sending batch 1 Scan to (n1,s1):1 +dist sender send r26: sending batch 2 Put to (n1,s1):1 +dist sender send r26: sending batch 1 EndTxn to (n1,s1):1 # Update with RETURNING statement with side-effects should not auto-commit. # In this case division can (in principle) error out. @@ -437,9 +437,9 @@ WHERE message LIKE '%sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r24: sending batch 1 Scan to (n1,s1):1 -dist sender send r24: sending batch 2 Put to (n1,s1):1 -dist sender send r24: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r26: sending batch 1 Scan to (n1,s1):1 +dist sender send r26: sending batch 2 Put to (n1,s1):1 +dist sender send r26: sending batch 1 EndTxn to (n1,s1):1 # Another way to test the scenario above: generate an error and ensure that the # mutation was not committed. @@ -474,7 +474,7 @@ WHERE message LIKE '%sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r24: sending batch 1 DelRng, 1 EndTxn to (n1,s1):1 +dist sender send r26: sending batch 1 DelRng, 1 EndTxn to (n1,s1):1 # Multi-row delete should auto-commit. query B @@ -495,7 +495,7 @@ WHERE message LIKE '%sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r24: sending batch 1 DelRng, 1 EndTxn to (n1,s1):1 +dist sender send r26: sending batch 1 DelRng, 1 EndTxn to (n1,s1):1 # No auto-commit inside a transaction. statement ok @@ -519,7 +519,7 @@ WHERE message LIKE '%sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r24: sending batch 1 DelRng to (n1,s1):1 +dist sender send r26: sending batch 1 DelRng to (n1,s1):1 statement ok ROLLBACK @@ -543,8 +543,8 @@ WHERE message LIKE '%sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r24: sending batch 1 Scan to (n1,s1):1 -dist sender send r24: sending batch 2 Del, 1 EndTxn to (n1,s1):1 +dist sender send r26: sending batch 1 Scan to (n1,s1):1 +dist sender send r26: sending batch 2 Del, 1 EndTxn to (n1,s1):1 # TODO(radu): allow non-side-effecting projections. query B @@ -566,9 +566,9 @@ WHERE message LIKE '%sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r24: sending batch 1 Scan to (n1,s1):1 -dist sender send r24: sending batch 2 Del to (n1,s1):1 -dist sender send r24: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r26: sending batch 1 Scan to (n1,s1):1 +dist sender send r26: sending batch 2 Del to (n1,s1):1 +dist sender send r26: sending batch 1 EndTxn to (n1,s1):1 # Insert with RETURNING statement with side-effects should not auto-commit. # In this case division can (in principle) error out. @@ -591,9 +591,9 @@ WHERE message LIKE '%sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r24: sending batch 1 Scan to (n1,s1):1 -dist sender send r24: sending batch 2 Del to (n1,s1):1 -dist sender send r24: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r26: sending batch 1 Scan to (n1,s1):1 +dist sender send r26: sending batch 2 Del to (n1,s1):1 +dist sender send r26: sending batch 1 EndTxn to (n1,s1):1 statement ok INSERT INTO ab VALUES (12, 0); @@ -641,9 +641,9 @@ WHERE message LIKE '%sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r24: sending batch 2 CPut, 2 InitPut to (n1,s1):1 -dist sender send r24: sending batch 2 Scan to (n1,s1):1 -dist sender send r24: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r26: sending batch 2 CPut, 2 InitPut to (n1,s1):1 +dist sender send r26: sending batch 2 Scan to (n1,s1):1 +dist sender send r26: sending batch 1 EndTxn to (n1,s1):1 query B SELECT count(*) > 0 FROM [ @@ -664,10 +664,10 @@ WHERE message LIKE '%sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r24: sending batch 1 Scan to (n1,s1):1 -dist sender send r24: sending batch 1 Put, 1 CPut, 1 Del to (n1,s1):1 -dist sender send r24: sending batch 1 Scan to (n1,s1):1 -dist sender send r24: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r26: sending batch 1 Scan to (n1,s1):1 +dist sender send r26: sending batch 1 Put, 1 CPut, 1 Del to (n1,s1):1 +dist sender send r26: sending batch 1 Scan to (n1,s1):1 +dist sender send r26: sending batch 1 EndTxn to (n1,s1):1 query B SELECT count(*) > 0 FROM [ @@ -689,10 +689,10 @@ WHERE message LIKE '%sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r24: sending batch 1 Scan to (n1,s1):1 -dist sender send r24: sending batch 1 Del to (n1,s1):1 -dist sender send r24: sending batch 1 Scan to (n1,s1):1 -dist sender send r24: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r26: sending batch 1 Scan to (n1,s1):1 +dist sender send r26: sending batch 1 Del to (n1,s1):1 +dist sender send r26: sending batch 1 Scan to (n1,s1):1 +dist sender send r26: sending batch 1 EndTxn to (n1,s1):1 # ----------------------- # Multiple mutation tests @@ -720,9 +720,9 @@ WHERE message LIKE '%sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r24: sending batch 2 CPut to (n1,s1):1 -dist sender send r24: sending batch 2 CPut to (n1,s1):1 -dist sender send r24: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r26: sending batch 2 CPut to (n1,s1):1 +dist sender send r26: sending batch 2 CPut to (n1,s1):1 +dist sender send r26: sending batch 1 EndTxn to (n1,s1):1 query B SELECT count(*) > 0 FROM [ @@ -745,6 +745,6 @@ WHERE message LIKE '%sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r24: sending batch 2 CPut to (n1,s1):1 -dist sender send r24: sending batch 2 CPut to (n1,s1):1 -dist sender send r24: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r26: sending batch 2 CPut to (n1,s1):1 +dist sender send r26: sending batch 2 CPut to (n1,s1):1 +dist sender send r26: sending batch 1 EndTxn to (n1,s1):1 diff --git a/pkg/sql/opt/exec/execbuilder/testdata/delete_range b/pkg/sql/opt/exec/execbuilder/testdata/delete_range index cccc32fc8b40..a2447c306b4d 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/delete_range +++ b/pkg/sql/opt/exec/execbuilder/testdata/delete_range @@ -15,11 +15,12 @@ query TT SELECT operation, message FROM [SHOW KV TRACE FOR SESSION] WHERE message LIKE '%DelRange%' OR message LIKE '%sending batch%' ---- +dist sender send r26: sending batch 1 Get to (n1,s1):1 flow DelRange /Table/53/1 - /Table/53/2 -dist sender send r24: sending batch 1 DelRng to (n1,s1):1 +dist sender send r26: sending batch 1 DelRng to (n1,s1):1 flow DelRange /Table/53/1/601/0 - /Table/53/2 -dist sender send r24: sending batch 1 DelRng to (n1,s1):1 -dist sender send r24: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r26: sending batch 1 DelRng to (n1,s1):1 +dist sender send r26: sending batch 1 EndTxn to (n1,s1):1 # Ensure that DelRange requests are autocommitted when DELETE FROM happens on a # chunk of fewer than 600 keys. @@ -35,4 +36,4 @@ SELECT operation, message FROM [SHOW KV TRACE FOR SESSION] WHERE message LIKE '%DelRange%' OR message LIKE '%sending batch%' ---- flow DelRange /Table/53/1/5 - /Table/53/1/5/# -dist sender send r24: sending batch 1 DelRng, 1 EndTxn to (n1,s1):1 +dist sender send r26: sending batch 1 DelRng, 1 EndTxn to (n1,s1):1 diff --git a/pkg/sql/opt/exec/execbuilder/testdata/show_trace b/pkg/sql/opt/exec/execbuilder/testdata/show_trace index f99a74dfc08a..0743f7477878 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/show_trace +++ b/pkg/sql/opt/exec/execbuilder/testdata/show_trace @@ -17,8 +17,9 @@ WHERE message NOT LIKE '%Z/%' AND tag NOT LIKE '%IndexBackfiller%' AND operation != 'dist sender send' ---- -flow CPut /Table/2/1/0/"t"/3/1 -> 53 +flow CPut /NamespaceTable/30/1/0/0/"t"/4/1 -> 53 flow CPut /Table/3/1/53/2/1 -> database: users: > > +flow CPut /NamespaceTable/30/1/53/0/"public"/4/1 -> 29 exec stmt rows affected: 0 @@ -37,7 +38,7 @@ WHERE message NOT LIKE '%Z/%' AND tag NOT LIKE '%IndexBackfiller%' AND operation != 'dist sender send' ---- -flow CPut /Table/2/1/53/"kv"/3/1 -> 54 +flow CPut /NamespaceTable/30/1/53/29/"kv"/4/1 -> 54 flow CPut /Table/3/1/54/2/1 -> table: columns: > nullable:false hidden:false > columns: > nullable:true hidden:false > next_column_id:3 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD created_explicitly:false > next_index_id:2 privileges: users: > next_mutation_id:1 format_version:3 state:PUBLIC offline_reason:"" view_query:"" drop_time:0 replacement_of: > audit_mode:DISABLED drop_job_id:0 create_query:"" create_as_of_time:<> temporary:false > exec stmt rows affected: 0 @@ -76,7 +77,7 @@ SELECT operation, message FROM [SHOW KV TRACE FOR SESSION] ---- flow CPut /Table/54/1/1/0 -> /TUPLE/2:2:Int/2 flow InitPut /Table/54/2/2/0 -> /BYTES/0x89 -kv.DistSender: sending partial batch r24: sending batch 1 CPut, 1 EndTxn to (n1,s1):1 +kv.DistSender: sending partial batch r26: sending batch 1 CPut, 1 EndTxn to (n1,s1):1 flow fast path completed exec stmt rows affected: 1 @@ -91,7 +92,7 @@ SELECT operation, message FROM [SHOW KV TRACE FOR SESSION] ---- flow CPut /Table/54/1/1/0 -> /TUPLE/2:2:Int/2 flow InitPut /Table/54/2/2/0 -> /BYTES/0x89 -kv.DistSender: sending partial batch r24: sending batch 1 CPut, 1 EndTxn to (n1,s1):1 +kv.DistSender: sending partial batch r26: sending batch 1 CPut, 1 EndTxn to (n1,s1):1 exec stmt execution failed after 0 rows: duplicate key value (k)=(1) violates unique constraint "primary" statement error duplicate key value @@ -104,7 +105,7 @@ SELECT operation, message FROM [SHOW KV TRACE FOR SESSION] ---- flow CPut /Table/54/1/2/0 -> /TUPLE/2:2:Int/2 flow InitPut /Table/54/2/2/0 -> /BYTES/0x8a -kv.DistSender: sending partial batch r24: sending batch 1 CPut, 1 EndTxn to (n1,s1):1 +kv.DistSender: sending partial batch r26: sending batch 1 CPut, 1 EndTxn to (n1,s1):1 exec stmt execution failed after 0 rows: duplicate key value (v)=(2) violates unique constraint "woo" statement ok @@ -122,7 +123,7 @@ WHERE message NOT LIKE '%Z/%' AND operation != 'dist sender send' ---- table reader Scan /Table/54/{1-2} -flow CPut /Table/2/1/53/"kv2"/3/1 -> 55 +flow CPut /NamespaceTable/30/1/53/29/"kv2"/4/1 -> 55 flow CPut /Table/3/1/55/2/1 -> table: columns: > nullable:true hidden:false > columns: > nullable:true hidden:false > columns: > nullable:false default_expr:"unique_rowid()" hidden:true > next_column_id:4 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD created_explicitly:false > next_index_id:2 privileges: users: > next_mutation_id:1 format_version:3 state:ADD offline_reason:"" view_query:"" drop_time:0 replacement_of: > audit_mode:DISABLED drop_job_id:0 create_query:"TABLE t.public.kv" create_as_of_time:<> temporary:false > exec stmt rows affected: 0 @@ -185,7 +186,7 @@ table reader Scan /Table/54/{1-2} table reader fetched: /kv/primary/1/v -> /2 flow Del /Table/54/2/2/0 flow Del /Table/54/1/1/0 -kv.DistSender: sending partial batch r24: sending batch 1 Del to (n1,s1):1 +kv.DistSender: sending partial batch r26: sending batch 1 Del to (n1,s1):1 flow fast path completed exec stmt rows affected: 1 @@ -242,7 +243,7 @@ SET tracing = on; INSERT INTO t.kv3 (k, v) VALUES (1,1); SET tracing = off query T SELECT message FROM [SHOW TRACE FOR SESSION] WHERE message LIKE e'%1 CPut, 1 EndTxn%' AND message NOT LIKE e'%proposing command%' ---- -r25: sending batch 1 CPut, 1 EndTxn to (n1,s1):1 +r27: sending batch 1 CPut, 1 EndTxn to (n1,s1):1 1 CPut, 1 EndTxn ## TODO(tschottdorf): re-enable @@ -295,8 +296,9 @@ SELECT DISTINCT node_id, store_id, replica_id FROM [SHOW EXPERIMENTAL_REPLICA TRACE FOR SESSION] ---- node_id store_id replica_id +1 1 26 1 1 6 -1 1 25 +1 1 27 subtest system_table_lookup @@ -311,6 +313,10 @@ WHERE (message LIKE 'querying next range%' OR message LIKE '%batch%') AND message NOT LIKE '%SystemConfigSpan%' AND message NOT LIKE '%PushTxn%' ---- +dist sender send querying next range at /NamespaceTable/30/1/0/0/"test"/4/1 +dist sender send r26: sending batch 1 Get to (n1,s1):1 +dist sender send querying next range at /NamespaceTable/30/1/52/0/"system"/4/1 +dist sender send r26: sending batch 1 Get to (n1,s1):1 dist sender send querying next range at /Table/3/1/12/2/1 dist sender send r6: sending batch 1 Get to (n1,s1):1 dist sender send querying next range at /Table/3/1/1/2/1 diff --git a/pkg/sql/opt/exec/execbuilder/testdata/upsert b/pkg/sql/opt/exec/execbuilder/testdata/upsert index c9d073e3be7a..0557e589c7f6 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/upsert +++ b/pkg/sql/opt/exec/execbuilder/testdata/upsert @@ -250,7 +250,7 @@ SELECT operation, message FROM [SHOW KV TRACE FOR SESSION] table reader Scan /Table/57/1/2{-/#} flow CPut /Table/57/1/2/0 -> /TUPLE/2:2:Int/3 flow InitPut /Table/57/2/3/0 -> /BYTES/0x8a -kv.DistSender: sending partial batch r24: sending batch 1 CPut, 1 EndTxn to (n1,s1):1 +kv.DistSender: sending partial batch r26: sending batch 1 CPut, 1 EndTxn to (n1,s1):1 flow fast path completed exec stmt rows affected: 1 @@ -264,7 +264,7 @@ SELECT operation, message FROM [SHOW KV TRACE FOR SESSION] table reader Scan /Table/57/1/1{-/#} flow CPut /Table/57/1/1/0 -> /TUPLE/2:2:Int/2 flow InitPut /Table/57/2/2/0 -> /BYTES/0x89 -kv.DistSender: sending partial batch r24: sending batch 1 CPut, 1 EndTxn to (n1,s1):1 +kv.DistSender: sending partial batch r26: sending batch 1 CPut, 1 EndTxn to (n1,s1):1 flow fast path completed exec stmt rows affected: 1 @@ -281,7 +281,7 @@ table reader fetched: /kv/primary/2/v -> /3 flow Put /Table/57/1/2/0 -> /TUPLE/2:2:Int/2 flow Del /Table/57/2/3/0 flow CPut /Table/57/2/2/0 -> /BYTES/0x8a (expecting does not exist) -kv.DistSender: sending partial batch r24: sending batch 1 Put, 1 EndTxn to (n1,s1):1 +kv.DistSender: sending partial batch r26: sending batch 1 Put, 1 EndTxn to (n1,s1):1 exec stmt execution failed after 0 rows: duplicate key value (v)=(2) violates unique constraint "woo" diff --git a/pkg/sql/opt/optbuilder/create_table.go b/pkg/sql/opt/optbuilder/create_table.go index 842449bb0a7c..481e812e6add 100644 --- a/pkg/sql/opt/optbuilder/create_table.go +++ b/pkg/sql/opt/optbuilder/create_table.go @@ -26,9 +26,24 @@ import ( // statement. func (b *Builder) buildCreateTable(ct *tree.CreateTable, inScope *scope) (outScope *scope) { b.DisableMemoReuse = true + isTemp := resolveTemporaryStatus(&ct.Table, ct.Temporary) + if isTemp { + // Postgres allows using `pg_temp` as an alias for the session specific temp + // schema. In PG, the following are equivalent: + // CREATE TEMP TABLE t <=> CREATE TABLE pg_temp.t <=> CREATE TEMP TABLE pg_temp.t + // + // The temporary schema is created the first time a session creates + // a temporary object, so it is possible to use `pg_temp` in a fully + // qualified name when the temporary schema does not exist. To allow this, + // we explicitly set the SchemaName to `public` for temporary tables, as + // the public schema is guaranteed to exist. This ensures the FQN can be + // resolved correctly. + // TODO(whomever): Once it is possible to drop schemas, it will no longer be + // safe to set the schema name to `public`, as it may have been dropped. + ct.Table.TableNamePrefix.SchemaName = tree.PublicSchemaName + ct.Temporary = true + } sch, resName := b.resolveSchemaForCreate(&ct.Table) - // TODO(radu): we are modifying the AST in-place here. We should be storing - // the resolved name separately. ct.Table.TableNamePrefix = resName schID := b.factory.Metadata().AddSchema(sch) diff --git a/pkg/sql/opt/optbuilder/util.go b/pkg/sql/opt/optbuilder/util.go index fd921e623417..1ca325296680 100644 --- a/pkg/sql/opt/optbuilder/util.go +++ b/pkg/sql/opt/optbuilder/util.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/privilege" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented" @@ -426,6 +427,18 @@ func (b *Builder) resolveAndBuildScalar( return b.buildScalar(texpr, inScope, nil, nil, nil) } +// In Postgres, qualifying an object name with pg_temp is equivalent to explicitly +// specifying TEMP/TEMPORARY in the CREATE syntax. resolveTemporaryStatus returns +// true if either(or both) of these conditions are true. +func resolveTemporaryStatus(name *tree.TableName, explicitTemp bool) bool { + // An explicit schema can only be provided in the CREATE TEMP TABLE statement + // iff it is pg_temp. + if explicitTemp && name.ExplicitSchema && name.SchemaName != sessiondata.PgTempSchemaName { + panic(pgerror.New(pgcode.InvalidTableDefinition, "cannot create temporary relation in non-temporary schema")) + } + return name.SchemaName == sessiondata.PgTempSchemaName || explicitTemp +} + // resolveSchemaForCreate returns the schema that will contain a newly created // catalog object with the given name. If the current user does not have the // CREATE privilege, then resolveSchemaForCreate raises an error. diff --git a/pkg/sql/opt_catalog.go b/pkg/sql/opt_catalog.go index 12f8a1040a2c..a6663a6186eb 100644 --- a/pkg/sql/opt_catalog.go +++ b/pkg/sql/opt_catalog.go @@ -104,7 +104,9 @@ func (os *optSchema) Name() *cat.SchemaName { // GetDataSourceNames is part of the cat.Schema interface. func (os *optSchema) GetDataSourceNames(ctx context.Context) ([]cat.DataSourceName, error) { return GetObjectNames( - ctx, os.planner.Txn(), os.planner, os.desc, + ctx, os.planner.Txn(), + os.planner, + os.desc, os.name.Schema(), true, /* explicitPrefix */ ) @@ -128,6 +130,7 @@ func (oc *optCatalog) ResolveSchema( // more general error. oc.tn.TableName = "" oc.tn.TableNamePrefix = *name + found, desc, err := oc.tn.ResolveTarget( ctx, oc.planner, diff --git a/pkg/sql/pgwire/pgwire_test.go b/pkg/sql/pgwire/pgwire_test.go index 1e5dfa9ebf2c..cced7736a563 100644 --- a/pkg/sql/pgwire/pgwire_test.go +++ b/pkg/sql/pgwire/pgwire_test.go @@ -733,7 +733,7 @@ func TestPGPreparedQuery(t *testing.T) { baseTest.Results("users", "primary", false, 1, "username", "ASC", false, false), }}, {"SHOW TABLES FROM system", []preparedQueryTest{ - baseTest.Results("comments").Others(18), + baseTest.Results("comments").Others(19), }}, {"SHOW SCHEMAS FROM system", []preparedQueryTest{ baseTest.Results("crdb_internal").Others(3), diff --git a/pkg/sql/physical_schema_accessors.go b/pkg/sql/physical_schema_accessors.go index 8d6faebb568b..c3cb5fce8021 100644 --- a/pkg/sql/physical_schema_accessors.go +++ b/pkg/sql/physical_schema_accessors.go @@ -15,6 +15,8 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/internal/client" + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" "github.com/cockroachdb/cockroach/pkg/util/encoding" @@ -54,11 +56,10 @@ func (a UncachedPhysicalAccessor) GetDatabaseDesc( return &sysDB, nil } - descID, err := getDescriptorID(ctx, txn, sqlbase.NewDatabaseKey(name)) + found, descID, err := sqlbase.LookupDatabaseID(ctx, txn, name) if err != nil { return nil, err - } - if descID == sqlbase.InvalidID { + } else if !found { if flags.Required { return nil, sqlbase.NewUndefinedDatabaseError(name) } @@ -74,9 +75,22 @@ func (a UncachedPhysicalAccessor) GetDatabaseDesc( } // IsValidSchema implements the SchemaAccessor interface. -func (a UncachedPhysicalAccessor) IsValidSchema(dbDesc *DatabaseDescriptor, scName string) bool { - // At this point, only the public schema is recognized. - return scName == tree.PublicSchema +func (a UncachedPhysicalAccessor) IsValidSchema( + ctx context.Context, txn *client.Txn, dbID sqlbase.ID, scName string, +) (bool, sqlbase.ID, error) { + // Try to use the system name resolution bypass. Avoids a hotspot by explicitly + // checking for public schema. + if scName == tree.PublicSchema { + return true, keys.PublicSchemaID, nil + } + + sKey := sqlbase.NewSchemaKey(dbID, scName) + schemaID, err := getDescriptorID(ctx, txn, sKey) + if err != nil || schemaID == sqlbase.InvalidID { + return false, sqlbase.InvalidID, err + } + + return true, schemaID, nil } // GetObjectNames implements the SchemaAccessor interface. @@ -87,7 +101,8 @@ func (a UncachedPhysicalAccessor) GetObjectNames( scName string, flags tree.DatabaseListFlags, ) (TableNames, error) { - if ok := a.IsValidSchema(dbDesc, scName); !ok { + ok, schemaID, err := a.IsValidSchema(ctx, txn, dbDesc.ID, scName) + if !ok || err != nil { if flags.Required { tn := tree.MakeTableNameWithSchema(tree.Name(dbDesc.Name), tree.Name(scName), "") return nil, sqlbase.NewUnsupportedSchemaUsageError(tree.ErrString(&tn.TableNamePrefix)) @@ -96,19 +111,61 @@ func (a UncachedPhysicalAccessor) GetObjectNames( } log.Eventf(ctx, "fetching list of objects for %q", dbDesc.Name) - prefix := sqlbase.NewTableKey(dbDesc.ID, "").Key() + prefix := sqlbase.NewTableKey(dbDesc.ID, schemaID, "").Key() sr, err := txn.Scan(ctx, prefix, prefix.PrefixEnd(), 0) if err != nil { return nil, err } + // We scan both the deprecated and new system.namespace table to get the + // complete list of tables. Duplicate entries may be present in both the tables, + // so we filter those out. If a duplicate entry is present, it doesn't matter + // which table it is read from -- system.namespace entries are never modified, + // they are only added/deleted. Entries are written to only one table, so + // duplicate entries must have been copied over during migration. Thus, it + // doesn't matter which table (newer/deprecated) the value is read from. + // + // It may seem counter-intuitive to read both tables if we have found data in + // the newer version. The migration copied all entries from the deprecated + // system.namespace and all new entries after the cluster version bump are added + // to the new system.namespace. Why do we do this then? + // This is to account the scenario where a table was created before + // the cluster version was bumped, but after the older system.namespace was + // copied into the newer system.namespace. Objects created in this window + // will only be present in the older system.namespace. To account for this + // scenario, we must do this filtering logic. + // TODO(whomever): This complexity can be removed in 20.2. + dprefix := sqlbase.NewDeprecatedTableKey(dbDesc.ID, "").Key() + dsr, err := txn.Scan(ctx, dprefix, dprefix.PrefixEnd(), 0) + if err != nil { + return nil, err + } + alreadySeen := make(map[string]bool) var tableNames tree.TableNames + for _, row := range sr { + _, tableName, err := encoding.DecodeUnsafeStringAscending(bytes.TrimPrefix( + row.Key, prefix), nil) + if err != nil { + return nil, err + } + alreadySeen[tableName] = true + tn := tree.MakeTableName(tree.Name(dbDesc.Name), tree.Name(tableName)) + tn.ExplicitCatalog = flags.ExplicitPrefix + tn.ExplicitSchema = flags.ExplicitPrefix + tableNames = append(tableNames, tn) + } + + for _, row := range dsr { + // Decode using the deprecated key prefix. _, tableName, err := encoding.DecodeUnsafeStringAscending( - bytes.TrimPrefix(row.Key, prefix), nil) + bytes.TrimPrefix(row.Key, dprefix), nil) if err != nil { return nil, err } + if alreadySeen[tableName] { + continue + } tn := tree.MakeTableName(tree.Name(dbDesc.Name), tree.Name(tableName)) tn.ExplicitCatalog = flags.ExplicitPrefix tn.ExplicitSchema = flags.ExplicitPrefix @@ -119,16 +176,12 @@ func (a UncachedPhysicalAccessor) GetObjectNames( // GetObjectDesc implements the SchemaAccessor interface. func (a UncachedPhysicalAccessor) GetObjectDesc( - ctx context.Context, txn *client.Txn, name *ObjectName, flags tree.ObjectLookupFlags, + ctx context.Context, + txn *client.Txn, + settings *cluster.Settings, + name *ObjectName, + flags tree.ObjectLookupFlags, ) (ObjectDescriptor, error) { - // At this point, only the public schema is recognized. - if name.Schema() != tree.PublicSchema { - if flags.Required { - return nil, sqlbase.NewUnsupportedSchemaUsageError(tree.ErrString(name)) - } - return nil, nil - } - // Look up the database ID. dbID, err := getDatabaseID(ctx, txn, name.Catalog(), flags.Required) if err != nil || dbID == sqlbase.InvalidID { @@ -136,23 +189,32 @@ func (a UncachedPhysicalAccessor) GetObjectDesc( return nil, err } + ok, schemaID, err := a.IsValidSchema(ctx, txn, dbID, name.Schema()) + if !ok || err != nil { + if flags.Required { + return nil, sqlbase.NewUnsupportedSchemaUsageError(tree.ErrString(name)) + } + return nil, nil + } + // Try to use the system name resolution bypass. This avoids a hotspot. // Note: we can only bypass name to ID resolution. The desc // lookup below must still go through KV because system descriptors // can be modified on a running cluster. - descID := sqlbase.LookupSystemTableDescriptorID(dbID, name.Table()) + descID := sqlbase.LookupSystemTableDescriptorID(ctx, settings, dbID, name.Table()) if descID == sqlbase.InvalidID { - descID, err = getDescriptorID(ctx, txn, sqlbase.NewTableKey(dbID, name.Table())) + var found bool + found, descID, err = sqlbase.LookupObjectID(ctx, txn, dbID, schemaID, name.Table()) if err != nil { return nil, err } - } - if descID == sqlbase.InvalidID { - // KV name resolution failed. - if flags.Required { - return nil, sqlbase.NewUndefinedRelationError(name) + if !found { + // KV name resolution failed. + if flags.Required { + return nil, sqlbase.NewUndefinedRelationError(name) + } + return nil, nil } - return nil, nil } // Look up the table using the discovered database descriptor. @@ -173,7 +235,13 @@ func (a UncachedPhysicalAccessor) GetObjectDesc( // Immediately after a RENAME an old name still points to the // descriptor during the drain phase for the name. Do not // return a descriptor during draining. - if desc.Name == name.Table() { + // + // The second or condition ensures that clusters < 20.1 access the + // system.namespace_deprecated table when selecting from system.namespace. + // As this table can not be renamed by users, it is okay that the first + // check fails. + if desc.Name == name.Table() || + name.Table() == sqlbase.NamespaceTable.Name && name.Catalog() == sqlbase.SystemDB.Name { if flags.RequireMutable { return sqlbase.NewMutableExistingTableDescriptor(*desc), nil } @@ -215,8 +283,7 @@ func (a *CachedPhysicalAccessor) GetDatabaseDesc( // The database was not known in the uncommitted list. Have the db // cache look it up by name for us. - return a.tc.databaseCache.getDatabaseDesc(ctx, - a.tc.leaseMgr.db.Txn, name, flags.Required) + return a.tc.databaseCache.getDatabaseDesc(ctx, a.tc.leaseMgr.db.Txn, name, flags.Required) } // We avoided the cache. Go lower. @@ -225,8 +292,27 @@ func (a *CachedPhysicalAccessor) GetDatabaseDesc( // GetObjectDesc implements the SchemaAccessor interface. func (a *CachedPhysicalAccessor) GetObjectDesc( - ctx context.Context, txn *client.Txn, name *ObjectName, flags tree.ObjectLookupFlags, + ctx context.Context, + txn *client.Txn, + settings *cluster.Settings, + name *ObjectName, + flags tree.ObjectLookupFlags, ) (ObjectDescriptor, error) { + // TODO(arul): Actually fix this to return the cached descriptor, by adding a + // schema cache to table collection. Until this is fixed, public tables with + // the same name as temporary tables might return the wrong data, as the wrong descriptor + // might be cached. + if name.Schema() != tree.PublicSchema { + phyAccessor := UncachedPhysicalAccessor{} + obj, err := phyAccessor.GetObjectDesc(ctx, txn, settings, name, flags) + if obj == nil { + return nil, err + } + if flags.RequireMutable { + return obj.(*sqlbase.MutableTableDescriptor), err + } + return obj.(*sqlbase.ImmutableTableDescriptor), err + } if flags.RequireMutable { table, err := a.tc.getMutableTableDescriptor(ctx, txn, name, flags) if table == nil { diff --git a/pkg/sql/planner.go b/pkg/sql/planner.go index 55de0795c777..53a0bf19e33f 100644 --- a/pkg/sql/planner.go +++ b/pkg/sql/planner.go @@ -222,6 +222,7 @@ func newInternalPlanner( // leave it uninitialized. tables := &TableCollection{ leaseMgr: execCfg.LeaseManager, + settings: execCfg.Settings, } dataMutator := &sessionDataMutator{ data: sd, @@ -383,6 +384,16 @@ func (p *planner) User() string { return p.SessionData().User } +func (p *planner) TemporarySchemaName() string { + return fmt.Sprintf("pg_temp_%v%v", + p.ExtendedEvalContext().SessionID.Hi, + p.ExtendedEvalContext().SessionID.Lo) +} + +func (p *planner) SetTemporarySchemaName(scName string) { + p.sessionDataMutator.SetTemporarySchemaName(scName) +} + // DistSQLPlanner returns the DistSQLPlanner func (p *planner) DistSQLPlanner() *DistSQLPlanner { return p.extendedEvalCtx.DistSQLPlanner diff --git a/pkg/sql/rename_database.go b/pkg/sql/rename_database.go index d6def6ceebf6..d41195a7d4f3 100644 --- a/pkg/sql/rename_database.go +++ b/pkg/sql/rename_database.go @@ -86,8 +86,8 @@ func (n *renameDatabaseNode) startExec(params runParams) error { } lookupFlags.Required = false for i := range tbNames { - objDesc, err := phyAccessor.GetObjectDesc(ctx, p.txn, &tbNames[i], - tree.ObjectLookupFlags{CommonLookupFlags: lookupFlags}) + objDesc, err := phyAccessor.GetObjectDesc(ctx, p.txn, p.ExecCfg().Settings, + &tbNames[i], tree.ObjectLookupFlags{CommonLookupFlags: lookupFlags}) if err != nil { return err } diff --git a/pkg/sql/rename_table.go b/pkg/sql/rename_table.go index 7825f478f1a5..b0343da03ade 100644 --- a/pkg/sql/rename_table.go +++ b/pkg/sql/rename_table.go @@ -106,7 +106,8 @@ func (n *renameTableNode) startExec(params runParams) error { tableDesc.SetName(newTn.Table()) tableDesc.ParentID = targetDbDesc.ID - newTbKey := sqlbase.NewTableKey(targetDbDesc.ID, newTn.Table()).Key() + newTbKey := sqlbase.MakePublicTableNameKey(ctx, params.ExecCfg().Settings, + targetDbDesc.ID, newTn.Table()).Key() if err := tableDesc.Validate(ctx, p.txn); err != nil { return err diff --git a/pkg/sql/resolver.go b/pkg/sql/resolver.go index 06c6219ec49e..934e05514388 100644 --- a/pkg/sql/resolver.go +++ b/pkg/sql/resolver.go @@ -54,8 +54,7 @@ func (p *planner) ResolveUncachedDatabaseByName( ctx context.Context, dbName string, required bool, ) (res *UncachedDatabaseDescriptor, err error) { p.runWithOptions(resolveFlags{skipCache: true}, func() { - res, err = p.LogicalSchemaAccessor().GetDatabaseDesc(ctx, p.txn, dbName, - p.CommonLookupFlags(required)) + res, err = p.LogicalSchemaAccessor().GetDatabaseDesc(ctx, p.txn, dbName, p.CommonLookupFlags(required)) }) return res, err } @@ -271,7 +270,11 @@ func (p *planner) LookupSchema( if err != nil || dbDesc == nil { return false, nil, err } - return sc.IsValidSchema(dbDesc, scName), dbDesc, nil + found, _, err = sc.IsValidSchema(ctx, p.txn, dbDesc.ID, scName) + if err != nil { + return false, nil, err + } + return found, dbDesc, nil } // LookupObject implements the tree.TableNameExistingResolver interface. @@ -281,7 +284,7 @@ func (p *planner) LookupObject( sc := p.LogicalSchemaAccessor() p.tableName = tree.MakeTableNameWithSchema(tree.Name(dbName), tree.Name(scName), tree.Name(tbName)) lookupFlags.CommonLookupFlags = p.CommonLookupFlags(false /* required */) - objDesc, err := sc.GetObjectDesc(ctx, p.txn, &p.tableName, lookupFlags) + objDesc, err := sc.GetObjectDesc(ctx, p.txn, p.ExecCfg().Settings, &p.tableName, lookupFlags) return objDesc != nil, objDesc, err } diff --git a/pkg/sql/schema_accessors.go b/pkg/sql/schema_accessors.go index 525e191805e1..e89bd00e89bd 100644 --- a/pkg/sql/schema_accessors.go +++ b/pkg/sql/schema_accessors.go @@ -14,6 +14,7 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/internal/client" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" ) @@ -76,8 +77,8 @@ type SchemaAccessor interface { // an error is returned; otherwise a nil reference is returned. GetDatabaseDesc(ctx context.Context, txn *client.Txn, dbName string, flags tree.DatabaseLookupFlags) (*DatabaseDescriptor, error) - // IsValidSchema returns true if the given schema name is valid for the given database. - IsValidSchema(db *DatabaseDescriptor, scName string) bool + // IsValidSchema returns true and the SchemaID if the given schema name is valid for the given database. + IsValidSchema(ctx context.Context, txn *client.Txn, dbID sqlbase.ID, scName string) (bool, sqlbase.ID, error) // GetObjectNames returns the list of all objects in the given // database and schema. @@ -89,5 +90,5 @@ type SchemaAccessor interface { // descriptor and that of its parent database. If the object is not // found and flags.required is true, an error is returned, otherwise // a nil reference is returned. - GetObjectDesc(ctx context.Context, txn *client.Txn, name *ObjectName, flags tree.ObjectLookupFlags) (ObjectDescriptor, error) + GetObjectDesc(ctx context.Context, txn *client.Txn, settings *cluster.Settings, name *ObjectName, flags tree.ObjectLookupFlags) (ObjectDescriptor, error) } diff --git a/pkg/sql/schema_changer.go b/pkg/sql/schema_changer.go index b64569386f2e..20303111cfab 100644 --- a/pkg/sql/schema_changer.go +++ b/pkg/sql/schema_changer.go @@ -875,8 +875,10 @@ func (sc *SchemaChanger) drainNames(ctx context.Context) error { func(txn *client.Txn) error { b := txn.NewBatch() for _, drain := range namesToReclaim { - tbKey := sqlbase.NewTableKey(drain.ParentID, drain.Name).Key() - b.Del(tbKey) + err := sqlbase.RemovePublicTableNamespaceEntry(ctx, txn, drain.ParentID, drain.Name) + if err != nil { + return err + } } if dropJobID != 0 { diff --git a/pkg/sql/scrub.go b/pkg/sql/scrub.go index 49317d9d8217..a770d65fbee8 100644 --- a/pkg/sql/scrub.go +++ b/pkg/sql/scrub.go @@ -163,7 +163,7 @@ func (n *scrubNode) startScrubDatabase(ctx context.Context, p *planner, name *tr for i := range tbNames { tableName := &tbNames[i] - objDesc, err := p.LogicalSchemaAccessor().GetObjectDesc(ctx, p.txn, + objDesc, err := p.LogicalSchemaAccessor().GetObjectDesc(ctx, p.txn, p.ExecCfg().Settings, tableName, p.ObjectLookupFlags(true /*required*/, false /*requireMutable*/)) if err != nil { return err diff --git a/pkg/sql/sem/builtins/builtins.go b/pkg/sql/sem/builtins/builtins.go index bf68ffd6b8fe..b3c35eb11a80 100644 --- a/pkg/sql/sem/builtins/builtins.go +++ b/pkg/sql/sem/builtins/builtins.go @@ -2812,7 +2812,7 @@ may increase either contention or retry errors, or both.`, Fn: func(evalCtx *tree.EvalContext, args tree.Datums) (tree.Datum, error) { ctx := evalCtx.Ctx() curDb := evalCtx.SessionData.Database - iter := evalCtx.SessionData.SearchPath.IterWithoutImplicitPGCatalog() + iter := evalCtx.SessionData.SearchPath.IterWithoutImplicitPGSchemas() for scName, ok := iter.Next(); ok; scName, ok = iter.Next() { if found, _, err := evalCtx.Planner.LookupSchema(ctx, curDb, scName); found || err != nil { if err != nil { @@ -2835,6 +2835,8 @@ may increase either contention or retry errors, or both.`, // schemas in the search path to compute current_schemas. This is // not documented but can be verified by a SQL client against a pg // server. + // The argument supplied applies to all implicit pg schemas, which includes + // pg_catalog and pg_temp (if one exists). "current_schemas": makeBuiltin( tree.FunctionProperties{ Category: categorySystemInfo, @@ -2846,13 +2848,13 @@ may increase either contention or retry errors, or both.`, Fn: func(evalCtx *tree.EvalContext, args tree.Datums) (tree.Datum, error) { ctx := evalCtx.Ctx() curDb := evalCtx.SessionData.Database - includePgCatalog := *(args[0].(*tree.DBool)) + includeImplicitPgSchemas := *(args[0].(*tree.DBool)) schemas := tree.NewDArray(types.String) var iter sessiondata.SearchPathIter - if includePgCatalog { + if includeImplicitPgSchemas { iter = evalCtx.SessionData.SearchPath.Iter() } else { - iter = evalCtx.SessionData.SearchPath.IterWithoutImplicitPGCatalog() + iter = evalCtx.SessionData.SearchPath.IterWithoutImplicitPGSchemas() } for scName, ok := iter.Next(); ok; scName, ok = iter.Next() { if found, _, err := evalCtx.Planner.LookupSchema(ctx, curDb, scName); found || err != nil { diff --git a/pkg/sql/sem/tree/function_name_test.go b/pkg/sql/sem/tree/function_name_test.go index 6e95291507a3..eb96d2f61343 100644 --- a/pkg/sql/sem/tree/function_name_test.go +++ b/pkg/sql/sem/tree/function_name_test.go @@ -34,7 +34,7 @@ func TestResolveFunction(t *testing.T) { {`""`, ``, `invalid function name: ""`}, } - searchPath := sessiondata.MakeSearchPath([]string{"pg_catalog"}) + searchPath := sessiondata.MakeSearchPath([]string{"pg_catalog"}, sessiondata.DefaultTemporarySchemaName) for _, tc := range testCases { stmt, err := parser.ParseOne("SELECT " + tc.in + "(1)") if err != nil { diff --git a/pkg/sql/sem/tree/name_resolution.go b/pkg/sql/sem/tree/name_resolution.go index e152fbb738ef..e1f42263bca3 100644 --- a/pkg/sql/sem/tree/name_resolution.go +++ b/pkg/sql/sem/tree/name_resolution.go @@ -275,9 +275,16 @@ func (t *TableName) ResolveExisting( searchPath sessiondata.SearchPath, ) (bool, NameResolutionResult, error) { if t.ExplicitSchema { + // pg_temp can be used as an alias for the current sessions temporary schema. + // We must perform this resolution before looking up the object. This + // resolution only succeeds if the session already has a temporary schema. + scName, err := searchPath.MaybeResolveTemporarySchema(t.Schema()) + if err != nil { + return false, nil, err + } if t.ExplicitCatalog { // Already 3 parts: nothing to search. Delegate to the resolver. - return r.LookupObject(ctx, lookupFlags, t.Catalog(), t.Schema(), t.Table()) + return r.LookupObject(ctx, lookupFlags, t.Catalog(), scName, t.Table()) } // Two parts: D.T. // Try to use the current database, and be satisfied if it's sufficient to find the object. @@ -287,7 +294,8 @@ func (t *TableName) ResolveExisting( // database is not set. For example, `select * from // pg_catalog.pg_tables` is meant to show all tables across all // databases when there is no current database set. - if found, objMeta, err := r.LookupObject(ctx, lookupFlags, curDb, t.Schema(), t.Table()); found || err != nil { + + if found, objMeta, err := r.LookupObject(ctx, lookupFlags, curDb, scName, t.Table()); found || err != nil { if err == nil { t.CatalogName = Name(curDb) } @@ -326,13 +334,20 @@ func (t *TableName) ResolveTarget( ctx context.Context, r TableNameTargetResolver, curDb string, searchPath sessiondata.SearchPath, ) (found bool, scMeta SchemaMeta, err error) { if t.ExplicitSchema { + // pg_temp can be used as an alias for the current sessions temporary schema. + // We must perform this resolution before looking up the object. This + // resolution only succeeds if the session already has a temporary schema. + scName, err := searchPath.MaybeResolveTemporarySchema(t.Schema()) + if err != nil { + return false, nil, err + } if t.ExplicitCatalog { // Already 3 parts: nothing to do. - return r.LookupSchema(ctx, t.Catalog(), t.Schema()) + return r.LookupSchema(ctx, t.Catalog(), scName) } // Two parts: D.T. // Try to use the current database, and be satisfied if it's sufficient to find the object. - if found, scMeta, err = r.LookupSchema(ctx, curDb, t.Schema()); found || err != nil { + if found, scMeta, err = r.LookupSchema(ctx, curDb, scName); found || err != nil { if err == nil { t.CatalogName = Name(curDb) } @@ -353,7 +368,7 @@ func (t *TableName) ResolveTarget( // This is a naked table name. Use the current schema = the first // valid item in the search path. - iter := searchPath.IterWithoutImplicitPGCatalog() + iter := searchPath.IterWithoutImplicitPGSchemas() for scName, ok := iter.Next(); ok; scName, ok = iter.Next() { if found, scMeta, err = r.LookupSchema(ctx, curDb, scName); found || err != nil { if err == nil { @@ -372,14 +387,21 @@ func (tp *TableNamePrefix) Resolve( ctx context.Context, r TableNameTargetResolver, curDb string, searchPath sessiondata.SearchPath, ) (found bool, scMeta SchemaMeta, err error) { if tp.ExplicitSchema { + // pg_temp can be used as an alias for the current sessions temporary schema. + // We must perform this resolution before looking up the object. This + // resolution only succeeds if the session already has a temporary schema. + scName, err := searchPath.MaybeResolveTemporarySchema(tp.Schema()) + if err != nil { + return false, nil, err + } if tp.ExplicitCatalog { // Catalog name is explicit; nothing to do. - return r.LookupSchema(ctx, tp.Catalog(), tp.Schema()) + return r.LookupSchema(ctx, tp.Catalog(), scName) } // Try with the current database. This may be empty, because // virtual schemas exist even when the db name is empty // (CockroachDB extension). - if found, scMeta, err = r.LookupSchema(ctx, curDb, tp.Schema()); found || err != nil { + if found, scMeta, err = r.LookupSchema(ctx, curDb, scName); found || err != nil { if err == nil { tp.CatalogName = Name(curDb) } @@ -399,7 +421,7 @@ func (tp *TableNamePrefix) Resolve( } // This is a naked table name. Use the current schema = the first // valid item in the search path. - iter := searchPath.IterWithoutImplicitPGCatalog() + iter := searchPath.IterWithoutImplicitPGSchemas() for scName, ok := iter.Next(); ok; scName, ok = iter.Next() { if found, scMeta, err = r.LookupSchema(ctx, curDb, scName); found || err != nil { if err == nil { diff --git a/pkg/sql/sem/tree/name_resolution_test.go b/pkg/sql/sem/tree/name_resolution_test.go index 553a8c74d412..906723d41927 100644 --- a/pkg/sql/sem/tree/name_resolution_test.go +++ b/pkg/sql/sem/tree/name_resolution_test.go @@ -520,6 +520,10 @@ func newFakeMetadata() *fakeMetadata { {"public", []tree.Name{"foo"}}, {"extended", []tree.Name{"bar", "pg_tables"}}, }}, + {"db3", []knownSchema{ + {"public", []tree.Name{"foo", "bar"}}, + {"pg_temp_123", []tree.Name{"foo", "baz"}}, + }}, }, } } @@ -528,7 +532,13 @@ func TestResolveTablePatternOrName(t *testing.T) { defer leaktest.AfterTest(t)() type spath = sessiondata.SearchPath - var mpath = func(args ...string) spath { return sessiondata.MakeSearchPath(args) } + var mpath = func(args ...string) spath { + return sessiondata.MakeSearchPath(args, sessiondata.DefaultTemporarySchemaName) + } + + var tpath = func(tempSchemaName string, args ...string) spath { + return sessiondata.MakeSearchPath(args, tempSchemaName) + } testCases := []struct { // Test inputs. @@ -693,6 +703,68 @@ func TestResolveTablePatternOrName(t *testing.T) { {`db1.pg_catalog.*`, ``, mpath(), false, `db1.pg_catalog.*`, `db1.pg_catalog.*`, `db1.pg_catalog`, ``}, {`"".pg_catalog.*`, ``, mpath(), false, `"".pg_catalog.*`, `"".pg_catalog.*`, `.pg_catalog`, ``}, {`blix.pg_catalog.*`, ``, mpath("public"), false, ``, ``, ``, `prefix or object not found`}, + + // + // Tests for temporary table resolution + // + + // Names of length 1 + + {`foo`, `db3`, tpath("pg_temp_123", "public"), true, `foo`, `db3.pg_temp_123.foo`, `db3.pg_temp_123[0]`, ``}, + {`foo`, `db3`, tpath("pg_temp_123", "public", "pg_temp"), true, `foo`, `db3.public.foo`, `db3.public[0]`, ``}, + {`baz`, `db3`, tpath("pg_temp_123", "public"), true, `baz`, `db3.pg_temp_123.baz`, `db3.pg_temp_123[1]`, ``}, + {`bar`, `db3`, tpath("pg_temp_123", "public"), true, `bar`, `db3.public.bar`, `db3.public[1]`, ``}, + {`bar`, `db3`, tpath("pg_temp_123", "public", "pg_temp"), true, `bar`, `db3.public.bar`, `db3.public[1]`, ``}, + + // Names of length 2 + + {`public.foo`, `db3`, tpath("pg_temp_123", "public"), true, `public.foo`, `db3.public.foo`, `db3.public[0]`, ``}, + {`pg_temp.foo`, `db3`, tpath("pg_temp_123", "public"), true, `pg_temp.foo`, `db3.pg_temp.foo`, `db3.pg_temp[0]`, ``}, + {`pg_temp_123.foo`, `db3`, tpath("pg_temp_123", "public"), true, `pg_temp_123.foo`, `db3.pg_temp_123.foo`, `db3.pg_temp_123[0]`, ``}, + + // Wrongly qualifying a TT/PT as a PT/TT results in an error. + {`pg_temp.bar`, `db3`, tpath("pg_temp_123", "public"), true, ``, ``, ``, `prefix or object not found`}, + {`public.baz`, `db3`, tpath("pg_temp_123", "public"), true, ``, ``, ``, `prefix or object not found`}, + + // Cases where a session tries to access a temporary table of another session. + {`pg_temp_111.foo`, `db3`, tpath("pg_temp_123", "public"), true, ``, ``, ``, `cannot access temporary tables of other sessions`}, + {`pg_temp_111.foo`, `db3`, tpath("pg_temp_123", "public"), false, ``, ``, ``, `cannot access temporary tables of other sessions`}, + + // Case where the temporary table being created has the same name as an + // existing persistent table. + {`pg_temp.bar`, `db3`, tpath("pg_temp_123", "public"), false, `pg_temp.bar`, `db3.pg_temp.bar`, `db3.pg_temp_123`, ``}, + + // Case where the persistent table being created has the same name as an + // existing temporary table. + {`public.baz`, `db3`, tpath("pg_temp_123", "public"), false, `public.baz`, `db3.public.baz`, `db3.public`, ``}, + + // Cases where the temporary schema has not been created yet + {`pg_temp.foo`, `db3`, mpath("public"), false, ``, ``, ``, `prefix or object not found`}, + + // Names of length 3 + + {`db3.public.foo`, `db3`, tpath("pg_temp_123", "public"), true, `db3.public.foo`, `db3.public.foo`, `db3.public[0]`, ``}, + {`db3.pg_temp.foo`, `db3`, tpath("pg_temp_123", "public"), true, `db3.pg_temp.foo`, `db3.pg_temp.foo`, `db3.pg_temp[0]`, ``}, + {`db3.pg_temp_123.foo`, `db3`, tpath("pg_temp_123", "public"), true, `db3.pg_temp_123.foo`, `db3.pg_temp_123.foo`, `db3.pg_temp_123[0]`, ``}, + + // Wrongly qualifying a TT/PT as a PT/TT results in an error. + {`db3.pg_temp.bar`, `db3`, tpath("pg_temp_123", "public"), true, ``, ``, ``, `prefix or object not found`}, + {`db3.public.baz`, `db3`, tpath("pg_temp_123", "public"), true, ``, ``, ``, `prefix or object not found`}, + + // Cases where a session tries to access a temporary table of another session. + {`db3.pg_temp_111.foo`, `db3`, tpath("pg_temp_123", "public"), true, ``, ``, ``, `cannot access temporary tables of other sessions`}, + {`db3.pg_temp_111.foo`, `db3`, tpath("pg_temp_123", "public"), false, ``, ``, ``, `cannot access temporary tables of other sessions`}, + + // Case where the temporary table being created has the same name as an + // existing persistent table. + {`db3.pg_temp.bar`, `db3`, tpath("pg_temp_123", "public"), false, `db3.pg_temp.bar`, `db3.pg_temp.bar`, `db3.pg_temp_123`, ``}, + + // Case where the persistent table being created has the same name as an + // existing temporary table. + {`db3.public.baz`, `db3`, tpath("pg_temp_123", "public"), false, `db3.public.baz`, `db3.public.baz`, `db3.public`, ``}, + + // Cases where the temporary schema has not been created yet + {`db3.pg_temp.foo`, `db3`, mpath("public"), false, ``, ``, ``, `prefix or object not found`}, } fakeResolver := newFakeMetadata() diff --git a/pkg/sql/sessiondata/search_path.go b/pkg/sql/sessiondata/search_path.go index d2d6763d2e6a..8a2d8eaa77de 100644 --- a/pkg/sql/sessiondata/search_path.go +++ b/pkg/sql/sessiondata/search_path.go @@ -10,11 +10,23 @@ package sessiondata -import "strings" +import ( + "strings" + + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" +) // PgDatabaseName is the name of the default postgres system database. const PgDatabaseName = "postgres" +// DefaultTemporarySchemaName is the temporary schema new sessions that have not +// created a temporary table start off with. +// This is prefixed with `pg_` to ensure there is no clash with a user defined +// schema if/when CRDB supports them. In PG, schema names starting with `pg_` +// are "reserved", so this can never clash with an actual physical schema. +const DefaultTemporarySchemaName = "pg_no_temp_schema" + // DefaultDatabaseName is the name ofthe default CockroachDB database used // for connections without a current db set. const DefaultDatabaseName = "defaultdb" @@ -22,48 +34,110 @@ const DefaultDatabaseName = "defaultdb" // PgCatalogName is the name of the pg_catalog system schema. const PgCatalogName = "pg_catalog" +// PgTempSchemaName is the alias for temporary schemas across sessions. +const PgTempSchemaName = "pg_temp" + // SearchPath represents a list of namespaces to search builtins in. // The names must be normalized (as per Name.Normalize) already. type SearchPath struct { - paths []string - containsPgCatalog bool + paths []string + containsPgCatalog bool + containsPgTempSchema bool + tempSchemaName string } // MakeSearchPath returns a new immutable SearchPath struct. The paths slice // must not be modified after hand-off to MakeSearchPath. -func MakeSearchPath(paths []string) SearchPath { +func MakeSearchPath(paths []string, tempSchemaName string) SearchPath { containsPgCatalog := false + containsPgTempSchema := false for _, e := range paths { if e == PgCatalogName { containsPgCatalog = true - break + } else if e == PgTempSchemaName { + containsPgTempSchema = true } } return SearchPath{ - paths: paths, - containsPgCatalog: containsPgCatalog, + paths: paths, + containsPgCatalog: containsPgCatalog, + containsPgTempSchema: containsPgTempSchema, + tempSchemaName: tempSchemaName, + } +} + +// WithTemporarySchemaName returns a new immutable SearchPath struct with +// the tempSchemaName supplied and the same paths as before. +// This should be called every time a session creates a temporary schema +// for the first time. +func (s SearchPath) WithTemporarySchemaName(tempSchemaName string) SearchPath { + return SearchPath{ + paths: s.paths, + containsPgCatalog: s.containsPgCatalog, + containsPgTempSchema: s.containsPgTempSchema, + tempSchemaName: tempSchemaName, } } +// UpdatePaths returns a new immutable SearchPath struct with the paths supplied +// and the same tempSchemaName as before. +func (s SearchPath) UpdatePaths(paths []string) SearchPath { + return MakeSearchPath(paths, s.tempSchemaName) +} + +// MaybeResolveTemporarySchema returns the session specific temporary schema +// for the pg_temp alias (only if a temporary schema exists). It acts as a pass +// through for all other schema names. +func (s SearchPath) MaybeResolveTemporarySchema(schemaName string) (string, error) { + // Only allow access to the session specific temporary schema. + if strings.HasPrefix(schemaName, PgTempSchemaName) && schemaName != PgTempSchemaName && schemaName != s.tempSchemaName { + return schemaName, pgerror.New(pgcode.FeatureNotSupported, "cannot access temporary tables of other sessions") + } + // If the schemaName is pg_temp and the tempSchemaName has been set, pg_temp + // is an alias the session specific temp schema. + if schemaName == PgTempSchemaName && s.tempSchemaName != DefaultTemporarySchemaName { + return s.tempSchemaName, nil + } + return schemaName, nil +} + // Iter returns an iterator through the search path. We must include the -// implicit pg_catalog at the beginning of the search path, unless it has been -// explicitly set later by the user. +// implicit pg_catalog and temporary schema at the beginning of the search path, +// unless they have been explicitly set later by the user. // "The system catalog schema, pg_catalog, is always searched, whether it is // mentioned in the path or not. If it is mentioned in the path then it will be // searched in the specified order. If pg_catalog is not in the path then it // will be searched before searching any of the path items." +// "Likewise, the current session's temporary-table schema, pg_temp_nnn, is +// always searched if it exists. It can be explicitly listed in the path by +// using the alias pg_temp. If it is not listed in the path then it is searched +// first (even before pg_catalog)." // - https://www.postgresql.org/docs/9.1/static/runtime-config-client.html func (s SearchPath) Iter() SearchPathIter { - if s.containsPgCatalog { - return SearchPathIter{paths: s.paths, i: 0} + implicitPgTempSchema := !s.containsPgTempSchema && s.tempSchemaName != DefaultTemporarySchemaName + sp := SearchPathIter{ + paths: s.paths, + implicitPgCatalog: !s.containsPgCatalog, + implicitPgTempSchema: implicitPgTempSchema, + } + if s.tempSchemaName != DefaultTemporarySchemaName { + sp.tempSchemaName = s.tempSchemaName } - return SearchPathIter{paths: s.paths, i: -1} + return sp } -// IterWithoutImplicitPGCatalog is the same as Iter, but does not include the -// implicit pg_catalog. -func (s SearchPath) IterWithoutImplicitPGCatalog() SearchPathIter { - return SearchPathIter{paths: s.paths, i: 0} +// IterWithoutImplicitPGSchemas is the same as Iter, but does not include the +// implicit pg_temp and pg_catalog. +func (s SearchPath) IterWithoutImplicitPGSchemas() SearchPathIter { + sp := SearchPathIter{ + paths: s.paths, + implicitPgCatalog: false, + implicitPgTempSchema: false, + } + if s.tempSchemaName != DefaultTemporarySchemaName { + sp.tempSchemaName = s.tempSchemaName + } + return sp } // GetPathArray returns the underlying path array of this SearchPath. The @@ -72,14 +146,26 @@ func (s SearchPath) GetPathArray() []string { return s.paths } +// GetTemporarySchemaName returns the temporary schema specific to the current +// session. +func (s SearchPath) GetTemporarySchemaName() string { + return s.tempSchemaName +} + // Equals returns true if two SearchPaths are the same. func (s SearchPath) Equals(other *SearchPath) bool { if s.containsPgCatalog != other.containsPgCatalog { return false } + if s.containsPgTempSchema != other.containsPgTempSchema { + return false + } if len(s.paths) != len(other.paths) { return false } + if s.tempSchemaName != other.tempSchemaName { + return false + } // Fast path: skip the check if it is the same slice. if &s.paths[0] != &other.paths[0] { for i := range s.paths { @@ -98,20 +184,42 @@ func (s SearchPath) String() string { // SearchPathIter enables iteration over the search paths without triggering an // allocation. Use one of the SearchPath.Iter methods to get an instance of the // iterator, and then repeatedly call the Next method in order to iterate over -// each search path. +// each search path. The tempSchemaName in the iterator is only set if the session +// has created a temporary schema. type SearchPathIter struct { - paths []string - i int + paths []string + implicitPgCatalog bool + implicitPgTempSchema bool + tempSchemaName string + i int } // Next returns the next search path, or false if there are no remaining paths. func (iter *SearchPathIter) Next() (path string, ok bool) { - if iter.i == -1 { - iter.i++ + // If the session specific temporary schema has not been created, we can + // preempt the name resolution failure by simply skipping the implicit pg_temp. + if iter.implicitPgTempSchema && iter.tempSchemaName != "" { + iter.implicitPgTempSchema = false + return iter.tempSchemaName, true + } + if iter.implicitPgCatalog { + iter.implicitPgCatalog = false return PgCatalogName, true } + if iter.i < len(iter.paths) { iter.i++ + // If pg_temp is explicitly present in the paths, it must be resolved to the + // session specific temp schema (if one exists). tempSchemaName is set in the + // iterator iff the session has created a temporary schema. + if iter.paths[iter.i-1] == PgTempSchemaName { + // If the session specific temporary schema has not been created we can + // preempt the resolution failure and iterate to the next entry. + if iter.tempSchemaName == "" { + return iter.Next() + } + return iter.tempSchemaName, true + } return iter.paths[iter.i-1], true } return "", false diff --git a/pkg/sql/sessiondata/search_path_test.go b/pkg/sql/sessiondata/search_path_test.go index 0b2c628466db..f918d8cac6d5 100644 --- a/pkg/sql/sessiondata/search_path_test.go +++ b/pkg/sql/sessiondata/search_path_test.go @@ -18,62 +18,194 @@ import ( "github.com/stretchr/testify/assert" ) +// Tests the implied search path when no temporary schema has been created +// by the session. func TestImpliedSearchPath(t *testing.T) { + testTempSchemaName := `test_temp_schema` + testCases := []struct { - explicitSearchPath []string - expectedSearchPath []string - expectedSearchPathWithoutImplicitPgCatalog []string + explicitSearchPath []string + expectedSearchPath []string + expectedSearchPathWithoutImplicitPgSchemas []string + expectedSearchPathWhenTemporarySchemaExists []string + expectedSearchPathWithoutImplicitPgSchemasWhenTempSchemaExists []string }{ - {[]string{}, []string{`pg_catalog`}, []string{}}, - {[]string{`pg_catalog`}, []string{`pg_catalog`}, []string{`pg_catalog`}}, - {[]string{`foobar`, `pg_catalog`}, []string{`foobar`, `pg_catalog`}, []string{`foobar`, `pg_catalog`}}, - {[]string{`foobar`}, []string{`pg_catalog`, `foobar`}, []string{`foobar`}}, + { + explicitSearchPath: []string{}, + expectedSearchPath: []string{`pg_catalog`}, + expectedSearchPathWithoutImplicitPgSchemas: []string{}, + expectedSearchPathWhenTemporarySchemaExists: []string{testTempSchemaName, `pg_catalog`}, + expectedSearchPathWithoutImplicitPgSchemasWhenTempSchemaExists: []string{}, + }, + { + explicitSearchPath: []string{`pg_catalog`}, + expectedSearchPath: []string{`pg_catalog`}, + expectedSearchPathWithoutImplicitPgSchemas: []string{`pg_catalog`}, + expectedSearchPathWhenTemporarySchemaExists: []string{testTempSchemaName, `pg_catalog`}, + expectedSearchPathWithoutImplicitPgSchemasWhenTempSchemaExists: []string{`pg_catalog`}, + }, + { + explicitSearchPath: []string{`pg_catalog`, `pg_temp`}, + expectedSearchPath: []string{`pg_catalog`}, + expectedSearchPathWithoutImplicitPgSchemas: []string{`pg_catalog`}, + expectedSearchPathWhenTemporarySchemaExists: []string{`pg_catalog`, testTempSchemaName}, + expectedSearchPathWithoutImplicitPgSchemasWhenTempSchemaExists: []string{`pg_catalog`, testTempSchemaName}, + }, + { + explicitSearchPath: []string{`pg_temp`, `pg_catalog`}, + expectedSearchPath: []string{`pg_catalog`}, + expectedSearchPathWithoutImplicitPgSchemas: []string{`pg_catalog`}, + expectedSearchPathWhenTemporarySchemaExists: []string{testTempSchemaName, `pg_catalog`}, + expectedSearchPathWithoutImplicitPgSchemasWhenTempSchemaExists: []string{testTempSchemaName, `pg_catalog`}, + }, + { + explicitSearchPath: []string{`foobar`, `pg_catalog`}, + expectedSearchPath: []string{`foobar`, `pg_catalog`}, + expectedSearchPathWithoutImplicitPgSchemas: []string{`foobar`, `pg_catalog`}, + expectedSearchPathWhenTemporarySchemaExists: []string{testTempSchemaName, `foobar`, `pg_catalog`}, + expectedSearchPathWithoutImplicitPgSchemasWhenTempSchemaExists: []string{`foobar`, `pg_catalog`}, + }, + { + explicitSearchPath: []string{`foobar`, `pg_temp`}, + expectedSearchPath: []string{`pg_catalog`, `foobar`}, + expectedSearchPathWithoutImplicitPgSchemas: []string{`foobar`}, + expectedSearchPathWhenTemporarySchemaExists: []string{`pg_catalog`, `foobar`, testTempSchemaName}, + expectedSearchPathWithoutImplicitPgSchemasWhenTempSchemaExists: []string{`foobar`, testTempSchemaName}, + }, + { + explicitSearchPath: []string{`foobar`}, + expectedSearchPath: []string{`pg_catalog`, `foobar`}, + expectedSearchPathWithoutImplicitPgSchemas: []string{`foobar`}, + expectedSearchPathWhenTemporarySchemaExists: []string{testTempSchemaName, `pg_catalog`, `foobar`}, + expectedSearchPathWithoutImplicitPgSchemasWhenTempSchemaExists: []string{`foobar`}, + }, } - for _, tc := range testCases { + for tcNum, tc := range testCases { t.Run(strings.Join(tc.explicitSearchPath, ","), func(t *testing.T) { - searchPath := MakeSearchPath(tc.explicitSearchPath) + searchPath := MakeSearchPath(tc.explicitSearchPath, DefaultTemporarySchemaName) actualSearchPath := make([]string, 0) iter := searchPath.Iter() for p, ok := iter.Next(); ok; p, ok = iter.Next() { actualSearchPath = append(actualSearchPath, p) } if !reflect.DeepEqual(tc.expectedSearchPath, actualSearchPath) { - t.Errorf(`Expected search path to be %#v, but was %#v.`, tc.expectedSearchPath, actualSearchPath) + t.Errorf( + `#%d: Expected search path to be %#v, but was %#v.`, + tcNum, + tc.expectedSearchPath, + actualSearchPath, + ) + } + }) + + t.Run(strings.Join(tc.explicitSearchPath, ",")+"/no-pg-schemas", func(t *testing.T) { + searchPath := MakeSearchPath(tc.explicitSearchPath, DefaultTemporarySchemaName) + actualSearchPath := make([]string, 0) + iter := searchPath.IterWithoutImplicitPGSchemas() + for p, ok := iter.Next(); ok; p, ok = iter.Next() { + actualSearchPath = append(actualSearchPath, p) + } + if !reflect.DeepEqual(tc.expectedSearchPathWithoutImplicitPgSchemas, actualSearchPath) { + t.Errorf( + `#%d: Expected search path to be %#v, but was %#v.`, + tcNum, + tc.expectedSearchPathWithoutImplicitPgSchemas, + actualSearchPath, + ) } }) - t.Run(strings.Join(tc.explicitSearchPath, ",")+"/no-pg-catalog", func(t *testing.T) { - searchPath := MakeSearchPath(tc.explicitSearchPath) + t.Run(strings.Join(tc.explicitSearchPath, ",")+"/temp-schema-exists", func(t *testing.T) { + searchPath := MakeSearchPath(tc.explicitSearchPath, testTempSchemaName) actualSearchPath := make([]string, 0) - iter := searchPath.IterWithoutImplicitPGCatalog() + iter := searchPath.Iter() for p, ok := iter.Next(); ok; p, ok = iter.Next() { actualSearchPath = append(actualSearchPath, p) } - if !reflect.DeepEqual(tc.expectedSearchPathWithoutImplicitPgCatalog, actualSearchPath) { - t.Errorf(`Expected search path to be %#v, but was %#v.`, tc.expectedSearchPathWithoutImplicitPgCatalog, actualSearchPath) + if !reflect.DeepEqual(tc.expectedSearchPathWhenTemporarySchemaExists, actualSearchPath) { + t.Errorf( + `#%d: Expected search path to be %#v, but was %#v.`, + tcNum, + tc.expectedSearchPathWhenTemporarySchemaExists, + actualSearchPath, + ) + } + }) + + t.Run(strings.Join(tc.explicitSearchPath, ",")+"/no-pg-schemas/temp-schema-exists", func(t *testing.T) { + searchPath := MakeSearchPath(tc.explicitSearchPath, testTempSchemaName) + actualSearchPath := make([]string, 0) + iter := searchPath.IterWithoutImplicitPGSchemas() + for p, ok := iter.Next(); ok; p, ok = iter.Next() { + actualSearchPath = append(actualSearchPath, p) + } + if !reflect.DeepEqual(tc.expectedSearchPathWithoutImplicitPgSchemasWhenTempSchemaExists, actualSearchPath) { + t.Errorf( + `#%d: Expected search path to be %#v, but was %#v.`, + tcNum, + tc.expectedSearchPathWithoutImplicitPgSchemasWhenTempSchemaExists, + actualSearchPath, + ) } }) } } func TestSearchPathEquals(t *testing.T) { - a1 := MakeSearchPath([]string{"x", "y", "z"}) - a2 := MakeSearchPath([]string{"x", "y", "z"}) + testTempSchemaName := `test_temp_schema` + + a1 := MakeSearchPath([]string{"x", "y", "z"}, DefaultTemporarySchemaName) + a2 := MakeSearchPath([]string{"x", "y", "z"}, DefaultTemporarySchemaName) assert.True(t, a1.Equals(&a1)) assert.True(t, a2.Equals(&a2)) assert.True(t, a1.Equals(&a2)) assert.True(t, a2.Equals(&a1)) - b := MakeSearchPath([]string{"x", "z", "y"}) + b := MakeSearchPath([]string{"x", "z", "y"}, DefaultTemporarySchemaName) assert.False(t, a1.Equals(&b)) - c1 := MakeSearchPath([]string{"x", "y", "pg_catalog"}) - c2 := MakeSearchPath([]string{"x", "y", "pg_catalog"}) + c1 := MakeSearchPath([]string{"x", "y", "z", "pg_catalog"}, DefaultTemporarySchemaName) + c2 := MakeSearchPath([]string{"x", "y", "z", "pg_catalog"}, DefaultTemporarySchemaName) assert.True(t, c1.Equals(&c2)) assert.False(t, a1.Equals(&c1)) - d := MakeSearchPath([]string{"x"}) + d := MakeSearchPath([]string{"x"}, DefaultTemporarySchemaName) assert.False(t, a1.Equals(&d)) + + e1 := MakeSearchPath([]string{"x", "y", "z"}, testTempSchemaName) + e2 := MakeSearchPath([]string{"x", "y", "z"}, testTempSchemaName) + assert.True(t, e1.Equals(&e1)) + assert.True(t, e1.Equals(&e2)) + assert.False(t, e1.Equals(&a1)) + + f := MakeSearchPath([]string{"x", "z", "y"}, testTempSchemaName) + assert.False(t, e1.Equals(&f)) + + g := MakeSearchPath([]string{"x", "y", "z", "pg_temp"}, DefaultTemporarySchemaName) + assert.False(t, e1.Equals(&g)) + assert.False(t, g.Equals(&c1)) + + h := MakeSearchPath([]string{"x", "y", "z", "pg_temp"}, testTempSchemaName) + assert.False(t, g.Equals(&h)) + + i := MakeSearchPath([]string{"x", "y", "z", "pg_temp", "pg_catalog"}, testTempSchemaName) + assert.False(t, i.Equals(&h)) + assert.False(t, i.Equals(&c1)) +} + +func TestWithTemporarySchema(t *testing.T) { + testTempSchemaName := `test_temp_schema` + + sp := MakeSearchPath([]string{"x", "y", "z"}, DefaultTemporarySchemaName) + sp = sp.UpdatePaths([]string{"x", "pg_catalog"}) + assert.True(t, sp.GetTemporarySchemaName() == DefaultTemporarySchemaName) + + sp = sp.WithTemporarySchemaName(testTempSchemaName) + sp = sp.UpdatePaths([]string{"pg_catalog"}) + assert.True(t, sp.GetTemporarySchemaName() == testTempSchemaName) + + sp = sp.UpdatePaths([]string{"x", "pg_temp"}) + assert.True(t, sp.GetTemporarySchemaName() == testTempSchemaName) } diff --git a/pkg/sql/set_zone_config.go b/pkg/sql/set_zone_config.go index cc020c0ede0a..02f2914d6526 100644 --- a/pkg/sql/set_zone_config.go +++ b/pkg/sql/set_zone_config.go @@ -324,7 +324,9 @@ func (n *setZoneConfigNode) startExec(params runParams) error { if err != nil { return err } - if targetID != keys.SystemDatabaseID && sqlbase.IsSystemConfigID(targetID) { + // NamespaceTableID is not in the system gossip range, but users should not + // be allowed to set zone configs on it. + if targetID != keys.SystemDatabaseID && sqlbase.IsSystemConfigID(targetID) || targetID == keys.NamespaceTableID { return pgerror.Newf(pgcode.CheckViolation, `cannot set zone configs for system config tables; `+ `try setting your config on the entire "system" database instead`) diff --git a/pkg/sql/sqlbase/constants.go b/pkg/sql/sqlbase/constants.go index 050838a3c31d..8e1307010533 100644 --- a/pkg/sql/sqlbase/constants.go +++ b/pkg/sql/sqlbase/constants.go @@ -17,7 +17,7 @@ import ( ) // DefaultSearchPath is the search path used by virgin sessions. -var DefaultSearchPath = sessiondata.MakeSearchPath([]string{"public"}) +var DefaultSearchPath = sessiondata.MakeSearchPath([]string{"public"}, sessiondata.DefaultTemporarySchemaName) // AdminRole is the default (and non-droppable) role with superuser privileges. var AdminRole = "admin" diff --git a/pkg/sql/sqlbase/keys.go b/pkg/sql/sqlbase/keys.go index e33225708904..0a15d76a0acf 100644 --- a/pkg/sql/sqlbase/keys.go +++ b/pkg/sql/sqlbase/keys.go @@ -18,16 +18,32 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/encoding" ) -// MakeNameMetadataKey returns the key for the name. Pass name == "" in order -// to generate the prefix key to use to scan over all of the names for the -// specified parentID. -func MakeNameMetadataKey(parentID ID, name string) roachpb.Key { +// MakeNameMetadataKey returns the key for the name, as expected by +// versions >= 20.1. +// Pass name == "" in order to generate the prefix key to use to scan over all +// of the names for the specified parentID. +func MakeNameMetadataKey(parentID ID, parentSchemaID ID, name string) roachpb.Key { k := keys.MakeTablePrefix(uint32(NamespaceTable.ID)) k = encoding.EncodeUvarintAscending(k, uint64(NamespaceTable.PrimaryIndex.ID)) k = encoding.EncodeUvarintAscending(k, uint64(parentID)) + k = encoding.EncodeUvarintAscending(k, uint64(parentSchemaID)) if name != "" { k = encoding.EncodeBytesAscending(k, []byte(name)) - k = keys.MakeFamilyKey(k, uint32(NamespaceTable.Columns[2].ID)) + k = keys.MakeFamilyKey(k, uint32(NamespaceTable.Columns[3].ID)) + } + return k +} + +// MakeDeprecatedNameMetadataKey returns the key for a name, as expected by +// versions < 20.1. Pass name == "" in order to generate the prefix key to use +// to scan over all of the names for the specified parentID. +func MakeDeprecatedNameMetadataKey(parentID ID, name string) roachpb.Key { + k := keys.MakeTablePrefix(uint32(DeprecatedNamespaceTable.ID)) + k = encoding.EncodeUvarintAscending(k, uint64(DeprecatedNamespaceTable.PrimaryIndex.ID)) + k = encoding.EncodeUvarintAscending(k, uint64(parentID)) + if name != "" { + k = encoding.EncodeBytesAscending(k, []byte(name)) + k = keys.MakeFamilyKey(k, uint32(DeprecatedNamespaceTable.Columns[2].ID)) } return k } diff --git a/pkg/sql/sqlbase/keys_test.go b/pkg/sql/sqlbase/keys_test.go index c66a2912eebd..0c539c5a9a54 100644 --- a/pkg/sql/sqlbase/keys_test.go +++ b/pkg/sql/sqlbase/keys_test.go @@ -23,12 +23,12 @@ func TestKeyAddress(t *testing.T) { testCases := []struct { key roachpb.Key }{ - {NewTableKey(0, "BAR").Key()}, - {NewTableKey(1, "BAR").Key()}, - {NewTableKey(1, "foo").Key()}, - {NewTableKey(2, "foo").Key()}, {MakeDescMetadataKey(123)}, {MakeDescMetadataKey(124)}, + {NewPublicTableKey(0, "BAR").Key()}, + {NewPublicTableKey(1, "BAR").Key()}, + {NewPublicTableKey(1, "foo").Key()}, + {NewPublicTableKey(2, "foo").Key()}, } var lastKey roachpb.Key for i, test := range testCases { diff --git a/pkg/sql/sqlbase/metadata.go b/pkg/sql/sqlbase/metadata.go index 7107f29014c7..e44a39a1f264 100644 --- a/pkg/sql/sqlbase/metadata.go +++ b/pkg/sql/sqlbase/metadata.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/protoutil" ) @@ -121,7 +122,9 @@ func (ms MetadataSchema) SystemDescriptorCount() int { // a bootstrapping cluster in order to create the tables contained // in the schema. Also returns a list of split points (a split for each SQL // table descriptor part of the initial values). Both returned sets are sorted. -func (ms MetadataSchema) GetInitialValues() ([]roachpb.KeyValue, []roachpb.RKey) { +func (ms MetadataSchema) GetInitialValues( + bootstrapVersion cluster.ClusterVersion, +) ([]roachpb.KeyValue, []roachpb.RKey) { var ret []roachpb.KeyValue var splits []roachpb.RKey @@ -140,10 +143,37 @@ func (ms MetadataSchema) GetInitialValues() ([]roachpb.KeyValue, []roachpb.RKey) // Create name metadata key. value := roachpb.Value{} value.SetInt(int64(desc.GetID())) - ret = append(ret, roachpb.KeyValue{ - Key: NewTableKey(parentID, desc.GetName()).Key(), - Value: value, - }) + + // TODO(whomever): This if/else can be removed in 20.2, as there will be no + // need to support the deprecated namespace table. + if bootstrapVersion.IsActive(cluster.VersionNamespaceTableWithSchemas) { + if parentID != keys.RootNamespaceID { + ret = append(ret, roachpb.KeyValue{ + Key: NewPublicTableKey(parentID, desc.GetName()).Key(), + Value: value, + }) + } else { + // Initializing a database. Databases must be initialized with + // the public schema, as all tables are scoped under the public schema. + publicSchemaValue := roachpb.Value{} + publicSchemaValue.SetInt(int64(keys.PublicSchemaID)) + ret = append( + ret, + roachpb.KeyValue{ + Key: NewDatabaseKey(desc.GetName()).Key(), + Value: value, + }, + roachpb.KeyValue{ + Key: NewPublicSchemaKey(desc.GetID()).Key(), + Value: publicSchemaValue, + }) + } + } else { + ret = append(ret, roachpb.KeyValue{ + Key: NewDeprecatedTableKey(parentID, desc.GetName()).Key(), + Value: value, + }) + } // Create descriptor metadata key. value = roachpb.Value{} @@ -217,10 +247,18 @@ var systemTableIDCache = func() map[string]ID { // LookupSystemTableDescriptorID uses the lookup cache above // to bypass a KV lookup when resolving the name of system tables. -func LookupSystemTableDescriptorID(dbID ID, tableName string) ID { +func LookupSystemTableDescriptorID( + ctx context.Context, settings *cluster.Settings, dbID ID, tableName string, +) ID { if dbID != SystemDB.ID { return InvalidID } + + if settings != nil && + !cluster.Version.IsActive(ctx, settings, cluster.VersionNamespaceTableWithSchemas) && + tableName == NamespaceTable.Name { + return DeprecatedNamespaceTable.ID + } dbID, ok := systemTableIDCache[tableName] if !ok { return InvalidID diff --git a/pkg/sql/sqlbase/namespace.go b/pkg/sql/sqlbase/namespace.go new file mode 100644 index 000000000000..f444c3a094b9 --- /dev/null +++ b/pkg/sql/sqlbase/namespace.go @@ -0,0 +1,206 @@ +// Copyright 2019 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 sqlbase + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/internal/client" + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/util/log" +) + +// This file abstracts all accesses to system.namespace. Entries in +// system.namespace are never modified. We only write new entries or delete +// existing entries. +// +// As of 20.1, the older system.namespace table is marked deprecated. It is +// replaced by a new system.namespace table that has an additional parentSchemaID +// column, which allows support for additional physical schemas. The new +// namespace table is also created outside the system config range, so it is no +// longer gossiped. +// +// To ensure accesses are seamless across mixed version clusters, >= 20.1 clusters, +// and during the upgrade process, the following functions should be used +// for adding/removing entries. +// TODO(whomever): The fallback semantics will no longer be required in 20.2. +// This code should be cleaned up then, to only access the new system.namespace +// table. + +// Deleting entries from system.namespace: +// Entries are deleted from both the deprecated and newer system.namespace, if +// they exist in them. +// Entries may be in one/both of the tables. +// - In a mixed version (19.2/20.1) cluster, the entry only exists in the older +// system.namespace. +// - In a 20.1 cluster, if the entry was created before upgrade, the entry exists +// in both the tables. +// - In a 20.1 cluster, if the entry was created after upgrade, it exists only +// in the newer system.namespace. +// +// Adding entries to system.namespace: +// Entries are added to either the new system.namespace or the deprecated +// system.namespace, depending on the cluster version. Methods supplied by +// this file only abstract key construction based on the cluster settings. +// It is not safe to construct keys and do removals/lookups using them, as +// this can cause issues in mixed version clusters. Please use the provided +// removal/lookup methods for those cases. + +// RemoveObjectNamespaceEntry removes entries from both the deprecated and +// new system.namespace table (if one exists). +func RemoveObjectNamespaceEntry( + ctx context.Context, txn *client.Txn, parentID ID, parentSchemaID ID, name string, KVTrace bool, +) error { + b := txn.NewBatch() + var toDelete []DescriptorKey + // The (parentID, name) mapping could be in either the new system.namespace + // or the deprecated version. Thus we try to remove the mapping from both. + if parentID == keys.RootNamespaceID { + toDelete = append(toDelete, NewDatabaseKey(name)) + // TODO(whomever): This can be completely removed in 20.2. + toDelete = append(toDelete, NewDeprecatedDatabaseKey(name)) + } else if parentSchemaID == keys.RootNamespaceID { + // Schemas were introduced in 20.1. + toDelete = append(toDelete, NewSchemaKey(parentID, name)) + } else { + toDelete = append(toDelete, NewTableKey(parentID, parentSchemaID, name)) + // TODO(whomever): This can be completely removed in 20.2. + toDelete = append(toDelete, NewDeprecatedTableKey(parentID, name)) + } + for _, delKey := range toDelete { + if KVTrace { + log.VEventf(ctx, 2, "Del %s", delKey) + } + b.Del(delKey.Key()) + } + return txn.Run(ctx, b) +} + +// RemovePublicTableNamespaceEntry is a wrapper around RemoveObjectNamespaceEntry +// for public tables. +func RemovePublicTableNamespaceEntry( + ctx context.Context, txn *client.Txn, parentID ID, name string, +) error { + return RemoveObjectNamespaceEntry(ctx, txn, parentID, keys.PublicSchemaID, name, false /* KVTrace */) +} + +// RemoveDatabaseNamespaceEntry is a wrapper around RemoveObjectNamespaceEntry +// for databases. +func RemoveDatabaseNamespaceEntry( + ctx context.Context, txn *client.Txn, name string, KVTrace bool, +) error { + return RemoveObjectNamespaceEntry(ctx, txn, keys.RootNamespaceID, keys.RootNamespaceID, name, KVTrace) +} + +// MakeObjectNameKey returns a key in the system.namespace table for +// a given parentID and name, based on the cluster version. +// - If cluster version >= 20.1, the key is in the new system.namespace table. +// - If cluster version < 20.1, the key is in the deprecated system.namespace table. +// - The parentSchemaID field is ignored in < 20.1 clusters. +func MakeObjectNameKey( + ctx context.Context, settings *cluster.Settings, parentID ID, parentSchemaID ID, name string, +) DescriptorKey { + // TODO(whomever): This if condition can be removed in 20.2 + if !cluster.Version.IsActive(ctx, settings, cluster.VersionNamespaceTableWithSchemas) { + return NewDeprecatedTableKey(parentID, name) + } + var key DescriptorKey + if parentID == keys.RootNamespaceID { + key = NewDatabaseKey(name) + } else if parentSchemaID == keys.RootNamespaceID { + key = NewSchemaKey(parentID, name) + } else { + key = NewTableKey(parentID, parentSchemaID, name) + } + return key +} + +// MakePublicTableNameKey is a wrapper around MakeObjectNameKey for public tables. +func MakePublicTableNameKey( + ctx context.Context, settings *cluster.Settings, parentID ID, name string, +) DescriptorKey { + return MakeObjectNameKey(ctx, settings, parentID, keys.PublicSchemaID, name) +} + +// MakeDatabaseNameKey is a wrapper around MakeObjectNameKey for databases. +func MakeDatabaseNameKey( + ctx context.Context, settings *cluster.Settings, name string, +) DescriptorKey { + return MakeObjectNameKey(ctx, settings, keys.RootNamespaceID, keys.RootNamespaceID, name) +} + +// LookupObjectID returns the ObjectID for the given +// (parentID, parentSchemaID, name) supplied. If cluster version < 20.1, +// the parentSchemaID is ignored. +func LookupObjectID( + ctx context.Context, txn *client.Txn, parentID ID, parentSchemaID ID, name string, +) (bool, ID, error) { + var key DescriptorKey + if parentID == keys.RootNamespaceID { + key = NewDatabaseKey(name) + } else if parentSchemaID == keys.RootNamespaceID { + key = NewSchemaKey(parentID, name) + } else { + key = NewTableKey(parentID, parentSchemaID, name) + } + log.Eventf(ctx, "looking up descriptor ID for name key %q", key.Key()) + res, err := txn.Get(ctx, key.Key()) + if err != nil { + return false, InvalidID, err + } + if res.Exists() { + return true, ID(res.ValueInt()), nil + } + // If the key wasn't found in the new system.namespace table, it may still + // exist in the deprecated system.namespace in the case of mixed version clusters. + // TODO(whomever): This can be removed in 20.2. + + // This fallback logic is only required if the table is under the public schema. + // Without this check, we can run into the following problem: + // - Persistent table `t` was created before the cluster upgrade, so it is + // present in both the old & new system.namespace table. + // - A session creates a temporary table `u`, which means the session has a + // valid temporary schema. + // - If this session explicitly accesses `pg_temp.t`, it should fail -- but + // without this check, `pg_temp.t` will return the permanent table instead. + if parentSchemaID != keys.PublicSchemaID { + return false, InvalidID, nil + } + + var dKey DescriptorKey + if parentID == keys.RootNamespaceID { + dKey = NewDeprecatedDatabaseKey(name) + } else { + dKey = NewDeprecatedTableKey(parentID, name) + } + log.Eventf(ctx, "looking up descriptor ID for name key %q", dKey.Key()) + res, err = txn.Get(ctx, dKey.Key()) + if err != nil { + return false, InvalidID, err + } + if res.Exists() { + return true, ID(res.ValueInt()), nil + } + return false, InvalidID, nil +} + +// LookupPublicTableID is a wrapper around LookupObjectID for public tables. +func LookupPublicTableID( + ctx context.Context, txn *client.Txn, parentID ID, name string, +) (bool, ID, error) { + return LookupObjectID(ctx, txn, parentID, keys.PublicSchemaID, name) +} + +// LookupDatabaseID is a wrapper around LookupObjectID for databases. +func LookupDatabaseID(ctx context.Context, txn *client.Txn, name string) (bool, ID, error) { + return LookupObjectID(ctx, txn, keys.RootNamespaceID, keys.RootNamespaceID, name) +} diff --git a/pkg/sql/sqlbase/structured.go b/pkg/sql/sqlbase/structured.go index 377e27a03b8a..3e57e7545f15 100644 --- a/pkg/sql/sqlbase/structured.go +++ b/pkg/sql/sqlbase/structured.go @@ -3689,12 +3689,12 @@ type DatabaseKey struct { // NewDatabaseKey returns a new DatabaseKey. func NewDatabaseKey(name string) DatabaseKey { - return DatabaseKey{name} + return DatabaseKey{name: name} } // Key implements DescriptorKey interface. func (dk DatabaseKey) Key() roachpb.Key { - return MakeNameMetadataKey(keys.RootNamespaceID, dk.name) + return MakeNameMetadataKey(keys.RootNamespaceID, keys.RootNamespaceID, dk.name) } // Name implements DescriptorKey interface. @@ -3704,21 +3704,94 @@ func (dk DatabaseKey) Name() string { // TableKey implements DescriptorKey interface. type TableKey struct { - parentID ID - name string + parentID ID + parentSchemaID ID + name string +} + +// NewPublicTableKey returns a new TableKey scoped under the public schema. +func NewPublicTableKey(parentID ID, name string) TableKey { + return TableKey{parentID: parentID, parentSchemaID: keys.PublicSchemaID, name: name} } -// NewTableKey returns a new TableKey. -func NewTableKey(parentID ID, name string) TableKey { - return TableKey{parentID, name} +// NewTableKey returns a new TableKey +func NewTableKey(parentID ID, parentSchemaID ID, name string) TableKey { + return TableKey{parentID: parentID, parentSchemaID: parentSchemaID, name: name} } // Key implements DescriptorKey interface. func (tk TableKey) Key() roachpb.Key { - return MakeNameMetadataKey(tk.parentID, tk.name) + return MakeNameMetadataKey(tk.parentID, tk.parentSchemaID, tk.name) } // Name implements DescriptorKey interface. func (tk TableKey) Name() string { return tk.name } + +// SchemaKey implements DescriptorKey interface. +type SchemaKey struct { + parentID ID + name string +} + +// NewSchemaKey returns a new SchemaKey +func NewSchemaKey(parentID ID, name string) SchemaKey { + return SchemaKey{parentID: parentID, name: name} +} + +// NewPublicSchemaKey returns a new SchemaKey specific to the public schema. +func NewPublicSchemaKey(parentID ID) SchemaKey { + return SchemaKey{parentID: parentID, name: tree.PublicSchema} +} + +// Key implements DescriptorKey interface. +func (sk SchemaKey) Key() roachpb.Key { + return MakeNameMetadataKey(sk.parentID, keys.RootNamespaceID, sk.name) +} + +// Name implements DescriptorKey interface. +func (sk SchemaKey) Name() string { + return sk.name +} + +// DeprecatedTableKey implements DescriptorKey interface. +type DeprecatedTableKey struct { + parentID ID + name string +} + +// NewDeprecatedTableKey returns a new DeprecatedTableKey +func NewDeprecatedTableKey(parentID ID, name string) DeprecatedTableKey { + return DeprecatedTableKey{parentID, name} +} + +// Key implements DescriptorKey interface. +func (dtk DeprecatedTableKey) Key() roachpb.Key { + return MakeDeprecatedNameMetadataKey(dtk.parentID, dtk.name) +} + +// Name implements DescriptorKey interface. +func (dtk DeprecatedTableKey) Name() string { + return dtk.name +} + +// DeprecatedDatabaseKey implements DescriptorKey interface. +type DeprecatedDatabaseKey struct { + name string +} + +// NewDeprecatedDatabaseKey returns a new DeprecatedDatabaseKey +func NewDeprecatedDatabaseKey(name string) DeprecatedDatabaseKey { + return DeprecatedDatabaseKey{name: name} +} + +// Key implements DescriptorKey interface. +func (ddk DeprecatedDatabaseKey) Key() roachpb.Key { + return MakeDeprecatedNameMetadataKey(keys.RootNamespaceID, ddk.name) +} + +// Name implements DescriptorKey interface. +func (ddk DeprecatedDatabaseKey) Name() string { + return ddk.name +} diff --git a/pkg/sql/sqlbase/structured.pb.go b/pkg/sql/sqlbase/structured.pb.go index 6a5667f471a8..f81bdbbc3ff1 100644 --- a/pkg/sql/sqlbase/structured.pb.go +++ b/pkg/sql/sqlbase/structured.pb.go @@ -73,7 +73,7 @@ func (x *ConstraintValidity) UnmarshalJSON(data []byte) error { return nil } func (ConstraintValidity) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_8340276ca0eb828c, []int{0} + return fileDescriptor_structured_5e07f3284b96005b, []int{0} } type ForeignKeyReference_Action int32 @@ -118,7 +118,7 @@ func (x *ForeignKeyReference_Action) UnmarshalJSON(data []byte) error { return nil } func (ForeignKeyReference_Action) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_8340276ca0eb828c, []int{0, 0} + return fileDescriptor_structured_5e07f3284b96005b, []int{0, 0} } // Match is the algorithm used to compare composite keys. @@ -158,7 +158,7 @@ func (x *ForeignKeyReference_Match) UnmarshalJSON(data []byte) error { return nil } func (ForeignKeyReference_Match) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_8340276ca0eb828c, []int{0, 1} + return fileDescriptor_structured_5e07f3284b96005b, []int{0, 1} } // The direction of a column in the index. @@ -195,7 +195,7 @@ func (x *IndexDescriptor_Direction) UnmarshalJSON(data []byte) error { return nil } func (IndexDescriptor_Direction) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_8340276ca0eb828c, []int{6, 0} + return fileDescriptor_structured_5e07f3284b96005b, []int{6, 0} } // The type of the index. @@ -232,7 +232,7 @@ func (x *IndexDescriptor_Type) UnmarshalJSON(data []byte) error { return nil } func (IndexDescriptor_Type) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_8340276ca0eb828c, []int{6, 1} + return fileDescriptor_structured_5e07f3284b96005b, []int{6, 1} } type ConstraintToUpdate_ConstraintType int32 @@ -275,7 +275,7 @@ func (x *ConstraintToUpdate_ConstraintType) UnmarshalJSON(data []byte) error { return nil } func (ConstraintToUpdate_ConstraintType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_8340276ca0eb828c, []int{7, 0} + return fileDescriptor_structured_5e07f3284b96005b, []int{7, 0} } // A descriptor within a mutation is unavailable for reads, writes @@ -340,7 +340,7 @@ func (x *DescriptorMutation_State) UnmarshalJSON(data []byte) error { return nil } func (DescriptorMutation_State) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_8340276ca0eb828c, []int{8, 0} + return fileDescriptor_structured_5e07f3284b96005b, []int{8, 0} } // Direction of mutation. @@ -383,7 +383,7 @@ func (x *DescriptorMutation_Direction) UnmarshalJSON(data []byte) error { return nil } func (DescriptorMutation_Direction) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_8340276ca0eb828c, []int{8, 1} + return fileDescriptor_structured_5e07f3284b96005b, []int{8, 1} } // State is set if this TableDescriptor is in the process of being added or deleted. @@ -434,7 +434,7 @@ func (x *TableDescriptor_State) UnmarshalJSON(data []byte) error { return nil } func (TableDescriptor_State) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_8340276ca0eb828c, []int{9, 0} + return fileDescriptor_structured_5e07f3284b96005b, []int{9, 0} } // AuditMode indicates which auditing actions to take when this table is used. @@ -471,7 +471,7 @@ func (x *TableDescriptor_AuditMode) UnmarshalJSON(data []byte) error { return nil } func (TableDescriptor_AuditMode) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_8340276ca0eb828c, []int{9, 1} + return fileDescriptor_structured_5e07f3284b96005b, []int{9, 1} } type ForeignKeyReference struct { @@ -493,7 +493,7 @@ func (m *ForeignKeyReference) Reset() { *m = ForeignKeyReference{} } func (m *ForeignKeyReference) String() string { return proto.CompactTextString(m) } func (*ForeignKeyReference) ProtoMessage() {} func (*ForeignKeyReference) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_8340276ca0eb828c, []int{0} + return fileDescriptor_structured_5e07f3284b96005b, []int{0} } func (m *ForeignKeyReference) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -553,7 +553,7 @@ func (m *ForeignKeyConstraint) Reset() { *m = ForeignKeyConstraint{} } func (m *ForeignKeyConstraint) String() string { return proto.CompactTextString(m) } func (*ForeignKeyConstraint) ProtoMessage() {} func (*ForeignKeyConstraint) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_8340276ca0eb828c, []int{1} + return fileDescriptor_structured_5e07f3284b96005b, []int{1} } func (m *ForeignKeyConstraint) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -600,7 +600,7 @@ func (m *ColumnDescriptor) Reset() { *m = ColumnDescriptor{} } func (m *ColumnDescriptor) String() string { return proto.CompactTextString(m) } func (*ColumnDescriptor) ProtoMessage() {} func (*ColumnDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_8340276ca0eb828c, []int{2} + return fileDescriptor_structured_5e07f3284b96005b, []int{2} } func (m *ColumnDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -656,7 +656,7 @@ func (m *ColumnFamilyDescriptor) Reset() { *m = ColumnFamilyDescriptor{} func (m *ColumnFamilyDescriptor) String() string { return proto.CompactTextString(m) } func (*ColumnFamilyDescriptor) ProtoMessage() {} func (*ColumnFamilyDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_8340276ca0eb828c, []int{3} + return fileDescriptor_structured_5e07f3284b96005b, []int{3} } func (m *ColumnFamilyDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -702,7 +702,7 @@ func (m *InterleaveDescriptor) Reset() { *m = InterleaveDescriptor{} } func (m *InterleaveDescriptor) String() string { return proto.CompactTextString(m) } func (*InterleaveDescriptor) ProtoMessage() {} func (*InterleaveDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_8340276ca0eb828c, []int{4} + return fileDescriptor_structured_5e07f3284b96005b, []int{4} } func (m *InterleaveDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -746,7 +746,7 @@ func (m *InterleaveDescriptor_Ancestor) Reset() { *m = InterleaveDescrip func (m *InterleaveDescriptor_Ancestor) String() string { return proto.CompactTextString(m) } func (*InterleaveDescriptor_Ancestor) ProtoMessage() {} func (*InterleaveDescriptor_Ancestor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_8340276ca0eb828c, []int{4, 0} + return fileDescriptor_structured_5e07f3284b96005b, []int{4, 0} } func (m *InterleaveDescriptor_Ancestor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -791,7 +791,7 @@ func (m *PartitioningDescriptor) Reset() { *m = PartitioningDescriptor{} func (m *PartitioningDescriptor) String() string { return proto.CompactTextString(m) } func (*PartitioningDescriptor) ProtoMessage() {} func (*PartitioningDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_8340276ca0eb828c, []int{5} + return fileDescriptor_structured_5e07f3284b96005b, []int{5} } func (m *PartitioningDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -834,7 +834,7 @@ func (m *PartitioningDescriptor_List) Reset() { *m = PartitioningDescrip func (m *PartitioningDescriptor_List) String() string { return proto.CompactTextString(m) } func (*PartitioningDescriptor_List) ProtoMessage() {} func (*PartitioningDescriptor_List) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_8340276ca0eb828c, []int{5, 0} + return fileDescriptor_structured_5e07f3284b96005b, []int{5, 0} } func (m *PartitioningDescriptor_List) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -879,7 +879,7 @@ func (m *PartitioningDescriptor_Range) Reset() { *m = PartitioningDescri func (m *PartitioningDescriptor_Range) String() string { return proto.CompactTextString(m) } func (*PartitioningDescriptor_Range) ProtoMessage() {} func (*PartitioningDescriptor_Range) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_8340276ca0eb828c, []int{5, 1} + return fileDescriptor_structured_5e07f3284b96005b, []int{5, 1} } func (m *PartitioningDescriptor_Range) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1016,7 +1016,7 @@ func (m *IndexDescriptor) Reset() { *m = IndexDescriptor{} } func (m *IndexDescriptor) String() string { return proto.CompactTextString(m) } func (*IndexDescriptor) ProtoMessage() {} func (*IndexDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_8340276ca0eb828c, []int{6} + return fileDescriptor_structured_5e07f3284b96005b, []int{6} } func (m *IndexDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1067,7 +1067,7 @@ func (m *ConstraintToUpdate) Reset() { *m = ConstraintToUpdate{} } func (m *ConstraintToUpdate) String() string { return proto.CompactTextString(m) } func (*ConstraintToUpdate) ProtoMessage() {} func (*ConstraintToUpdate) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_8340276ca0eb828c, []int{7} + return fileDescriptor_structured_5e07f3284b96005b, []int{7} } func (m *ConstraintToUpdate) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1121,7 +1121,7 @@ func (m *DescriptorMutation) Reset() { *m = DescriptorMutation{} } func (m *DescriptorMutation) String() string { return proto.CompactTextString(m) } func (*DescriptorMutation) ProtoMessage() {} func (*DescriptorMutation) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_8340276ca0eb828c, []int{8} + return fileDescriptor_structured_5e07f3284b96005b, []int{8} } func (m *DescriptorMutation) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1422,7 +1422,7 @@ type TableDescriptor struct { // inbound_fks contains all foreign key constraints that have this table as // the referenced table. InboundFKs []ForeignKeyConstraint `protobuf:"bytes,37,rep,name=inbound_fks,json=inboundFks" json:"inbound_fks"` - // Temporary table support will be added to CRDB starting from 20.1 . The temporary + // Temporary table support will be added to CRDB starting from 20.1. The temporary // flag is set to true for all temporary tables. All table descriptors created // before 20.1 refer to persistent tables, so lack of the flag being set implies // the table is persistent. @@ -1433,7 +1433,7 @@ func (m *TableDescriptor) Reset() { *m = TableDescriptor{} } func (m *TableDescriptor) String() string { return proto.CompactTextString(m) } func (*TableDescriptor) ProtoMessage() {} func (*TableDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_8340276ca0eb828c, []int{9} + return fileDescriptor_structured_5e07f3284b96005b, []int{9} } func (m *TableDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1725,7 +1725,7 @@ func (m *TableDescriptor_SchemaChangeLease) Reset() { *m = TableDescript func (m *TableDescriptor_SchemaChangeLease) String() string { return proto.CompactTextString(m) } func (*TableDescriptor_SchemaChangeLease) ProtoMessage() {} func (*TableDescriptor_SchemaChangeLease) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_8340276ca0eb828c, []int{9, 0} + return fileDescriptor_structured_5e07f3284b96005b, []int{9, 0} } func (m *TableDescriptor_SchemaChangeLease) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1763,7 +1763,7 @@ func (m *TableDescriptor_CheckConstraint) Reset() { *m = TableDescriptor func (m *TableDescriptor_CheckConstraint) String() string { return proto.CompactTextString(m) } func (*TableDescriptor_CheckConstraint) ProtoMessage() {} func (*TableDescriptor_CheckConstraint) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_8340276ca0eb828c, []int{9, 1} + return fileDescriptor_structured_5e07f3284b96005b, []int{9, 1} } func (m *TableDescriptor_CheckConstraint) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1866,7 +1866,7 @@ func (m *TableDescriptor_NameInfo) Reset() { *m = TableDescriptor_NameIn func (m *TableDescriptor_NameInfo) String() string { return proto.CompactTextString(m) } func (*TableDescriptor_NameInfo) ProtoMessage() {} func (*TableDescriptor_NameInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_8340276ca0eb828c, []int{9, 2} + return fileDescriptor_structured_5e07f3284b96005b, []int{9, 2} } func (m *TableDescriptor_NameInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1906,7 +1906,7 @@ func (m *TableDescriptor_Reference) Reset() { *m = TableDescriptor_Refer func (m *TableDescriptor_Reference) String() string { return proto.CompactTextString(m) } func (*TableDescriptor_Reference) ProtoMessage() {} func (*TableDescriptor_Reference) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_8340276ca0eb828c, []int{9, 3} + return fileDescriptor_structured_5e07f3284b96005b, []int{9, 3} } func (m *TableDescriptor_Reference) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1943,7 +1943,7 @@ func (m *TableDescriptor_MutationJob) Reset() { *m = TableDescriptor_Mut func (m *TableDescriptor_MutationJob) String() string { return proto.CompactTextString(m) } func (*TableDescriptor_MutationJob) ProtoMessage() {} func (*TableDescriptor_MutationJob) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_8340276ca0eb828c, []int{9, 4} + return fileDescriptor_structured_5e07f3284b96005b, []int{9, 4} } func (m *TableDescriptor_MutationJob) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1986,7 +1986,7 @@ func (m *TableDescriptor_SequenceOpts) Reset() { *m = TableDescriptor_Se func (m *TableDescriptor_SequenceOpts) String() string { return proto.CompactTextString(m) } func (*TableDescriptor_SequenceOpts) ProtoMessage() {} func (*TableDescriptor_SequenceOpts) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_8340276ca0eb828c, []int{9, 5} + return fileDescriptor_structured_5e07f3284b96005b, []int{9, 5} } func (m *TableDescriptor_SequenceOpts) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2026,7 +2026,7 @@ func (m *TableDescriptor_SequenceOpts_SequenceOwner) String() string { } func (*TableDescriptor_SequenceOpts_SequenceOwner) ProtoMessage() {} func (*TableDescriptor_SequenceOpts_SequenceOwner) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_8340276ca0eb828c, []int{9, 5, 0} + return fileDescriptor_structured_5e07f3284b96005b, []int{9, 5, 0} } func (m *TableDescriptor_SequenceOpts_SequenceOwner) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2066,7 +2066,7 @@ func (m *TableDescriptor_Replacement) Reset() { *m = TableDescriptor_Rep func (m *TableDescriptor_Replacement) String() string { return proto.CompactTextString(m) } func (*TableDescriptor_Replacement) ProtoMessage() {} func (*TableDescriptor_Replacement) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_8340276ca0eb828c, []int{9, 6} + return fileDescriptor_structured_5e07f3284b96005b, []int{9, 6} } func (m *TableDescriptor_Replacement) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2103,7 +2103,7 @@ func (m *TableDescriptor_GCDescriptorMutation) Reset() { *m = TableDescr func (m *TableDescriptor_GCDescriptorMutation) String() string { return proto.CompactTextString(m) } func (*TableDescriptor_GCDescriptorMutation) ProtoMessage() {} func (*TableDescriptor_GCDescriptorMutation) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_8340276ca0eb828c, []int{9, 7} + return fileDescriptor_structured_5e07f3284b96005b, []int{9, 7} } func (m *TableDescriptor_GCDescriptorMutation) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2142,7 +2142,7 @@ func (m *DatabaseDescriptor) Reset() { *m = DatabaseDescriptor{} } func (m *DatabaseDescriptor) String() string { return proto.CompactTextString(m) } func (*DatabaseDescriptor) ProtoMessage() {} func (*DatabaseDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_8340276ca0eb828c, []int{10} + return fileDescriptor_structured_5e07f3284b96005b, []int{10} } func (m *DatabaseDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2200,7 +2200,7 @@ func (m *Descriptor) Reset() { *m = Descriptor{} } func (m *Descriptor) String() string { return proto.CompactTextString(m) } func (*Descriptor) ProtoMessage() {} func (*Descriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_8340276ca0eb828c, []int{11} + return fileDescriptor_structured_5e07f3284b96005b, []int{11} } func (m *Descriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -11097,10 +11097,10 @@ var ( ) func init() { - proto.RegisterFile("sql/sqlbase/structured.proto", fileDescriptor_structured_8340276ca0eb828c) + proto.RegisterFile("sql/sqlbase/structured.proto", fileDescriptor_structured_5e07f3284b96005b) } -var fileDescriptor_structured_8340276ca0eb828c = []byte{ +var fileDescriptor_structured_5e07f3284b96005b = []byte{ // 3430 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x5a, 0xcb, 0x6f, 0x1b, 0x47, 0x78, 0xd7, 0xf2, 0xcd, 0x8f, 0xaf, 0xe5, 0xc8, 0x76, 0x68, 0xc5, 0x11, 0x25, 0x3a, 0x4e, 0x94, diff --git a/pkg/sql/sqlbase/structured.proto b/pkg/sql/sqlbase/structured.proto index 419efa50cf62..a6effc63a47d 100644 --- a/pkg/sql/sqlbase/structured.proto +++ b/pkg/sql/sqlbase/structured.proto @@ -863,7 +863,7 @@ message TableDescriptor { // the referenced table. repeated ForeignKeyConstraint inbound_fks = 37 [(gogoproto.nullable) = false, (gogoproto.customname) = "InboundFKs"]; - // Temporary table support will be added to CRDB starting from 20.1 . The temporary + // Temporary table support will be added to CRDB starting from 20.1. The temporary // flag is set to true for all temporary tables. All table descriptors created // before 20.1 refer to persistent tables, so lack of the flag being set implies // the table is persistent. diff --git a/pkg/sql/sqlbase/system.go b/pkg/sql/sqlbase/system.go index 57d5ba4277f4..fdb843f481ce 100644 --- a/pkg/sql/sqlbase/system.go +++ b/pkg/sql/sqlbase/system.go @@ -63,9 +63,10 @@ const ( NamespaceTableSchema = ` CREATE TABLE system.namespace ( "parentID" INT8, + "parentSchemaID" INT8, name STRING, id INT8, - PRIMARY KEY ("parentID", name) + PRIMARY KEY ("parentID", "parentSchemaID", name) );` DescriptorTableSchema = ` @@ -243,11 +244,12 @@ func pk(name string) IndexDescriptor { // system object. Super users (root and admin) must have exactly the specified privileges, // other users must not exceed the specified privileges. var SystemAllowedPrivileges = map[ID]privilege.List{ - keys.SystemDatabaseID: privilege.ReadData, - keys.NamespaceTableID: privilege.ReadData, - keys.DescriptorTableID: privilege.ReadData, - keys.UsersTableID: privilege.ReadWriteData, - keys.ZonesTableID: privilege.ReadWriteData, + keys.SystemDatabaseID: privilege.ReadData, + keys.NamespaceTableID: privilege.ReadData, + keys.DeprecatedNamespaceTableID: privilege.ReadData, + keys.DescriptorTableID: privilege.ReadData, + keys.UsersTableID: privilege.ReadWriteData, + keys.ZonesTableID: privilege.ReadWriteData, // We eventually want to migrate the table to appear read-only to force the // the use of a validating, logging accessor, so we'll go ahead and tolerate // read-only privs to make that migration possible later. @@ -297,10 +299,10 @@ var ( // SystemDB is the descriptor for the system database. SystemDB = MakeSystemDatabaseDesc() - // NamespaceTable is the descriptor for the namespace table. - NamespaceTable = TableDescriptor{ - Name: "namespace", - ID: keys.NamespaceTableID, + // DeprecatedNamespaceTable is the descriptor for the deprecated namespace table. + DeprecatedNamespaceTable = TableDescriptor{ + Name: "namespace_deprecated", + ID: keys.DeprecatedNamespaceTableID, ParentID: keys.SystemDatabaseID, Version: 1, Columns: []ColumnDescriptor{ @@ -324,6 +326,39 @@ var ( Version: SecondaryIndexFamilyFormatVersion, }, NextIndexID: 2, + Privileges: NewCustomSuperuserPrivilegeDescriptor(SystemAllowedPrivileges[keys.DeprecatedNamespaceTableID]), + FormatVersion: InterleavedFormatVersion, + NextMutationID: 1, + } + + // NamespaceTable is the descriptor for the namespace table. + NamespaceTable = TableDescriptor{ + Name: "namespace", + ID: keys.NamespaceTableID, + ParentID: keys.SystemDatabaseID, + Version: 1, + Columns: []ColumnDescriptor{ + {Name: "parentID", ID: 1, Type: *types.Int}, + {Name: "parentSchemaID", ID: 2, Type: *types.Int}, + {Name: "name", ID: 3, Type: *types.String}, + {Name: "id", ID: 4, Type: *types.Int, Nullable: true}, + }, + NextColumnID: 5, + Families: []ColumnFamilyDescriptor{ + {Name: "primary", ID: 0, ColumnNames: []string{"parentID", "parentSchemaID", "name"}, ColumnIDs: []ColumnID{1, 2, 3}}, + {Name: "fam_4_id", ID: 4, ColumnNames: []string{"id"}, ColumnIDs: []ColumnID{4}, DefaultColumnID: 4}, + }, + NextFamilyID: 5, + PrimaryIndex: IndexDescriptor{ + Name: "primary", + ID: 1, + Unique: true, + ColumnNames: []string{"parentID", "parentSchemaID", "name"}, + ColumnDirections: []IndexDescriptor_Direction{IndexDescriptor_ASC, IndexDescriptor_ASC, IndexDescriptor_ASC}, + ColumnIDs: []ColumnID{1, 2, 3}, + Version: SecondaryIndexFamilyFormatVersion, + }, + NextIndexID: 2, Privileges: NewCustomSuperuserPrivilegeDescriptor(SystemAllowedPrivileges[keys.NamespaceTableID]), FormatVersion: InterleavedFormatVersion, NextMutationID: 1, @@ -1093,6 +1128,7 @@ func addSystemDescriptorsToSchema(target *MetadataSchema) { target.AddDescriptor(keys.RootNamespaceID, &SystemDB) // Add system config tables. + target.AddDescriptor(keys.SystemDatabaseID, &DeprecatedNamespaceTable) target.AddDescriptor(keys.SystemDatabaseID, &NamespaceTable) target.AddDescriptor(keys.SystemDatabaseID, &DescriptorTable) target.AddDescriptor(keys.SystemDatabaseID, &UsersTable) diff --git a/pkg/sql/sqlbase/testutils.go b/pkg/sql/sqlbase/testutils.go index 1fe515ca666f..e3c2454e9e91 100644 --- a/pkg/sql/sqlbase/testutils.go +++ b/pkg/sql/sqlbase/testutils.go @@ -48,6 +48,7 @@ import ( // GetTableDescriptor retrieves a table descriptor directly from the KV layer. func GetTableDescriptor(kvDB *client.DB, database string, table string) *TableDescriptor { // log.VEventf(context.TODO(), 2, "GetTableDescriptor %q %q", database, table) + // testutil, so we pass settings as nil for both database and table name keys. dKey := NewDatabaseKey(database) ctx := context.TODO() gr, err := kvDB.Get(ctx, dKey.Key()) @@ -59,7 +60,7 @@ func GetTableDescriptor(kvDB *client.DB, database string, table string) *TableDe } dbDescID := ID(gr.ValueInt()) - tKey := NewTableKey(dbDescID, table) + tKey := NewPublicTableKey(dbDescID, table) gr, err = kvDB.Get(ctx, tKey.Key()) if err != nil { panic(err) diff --git a/pkg/sql/table.go b/pkg/sql/table.go index 95c2dccc0c74..a3aeddcab5ee 100644 --- a/pkg/sql/table.go +++ b/pkg/sql/table.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -137,6 +138,11 @@ type TableCollection struct { // allDatabaseDescriptors is a slice of all available database descriptors. // These are purged at the same time as allDescriptors. allDatabaseDescriptors []*sqlbase.DatabaseDescriptor + + // settings are required to correctly resolve system.namespace accesses in + // mixed version (19.2/20.1) clusters. + // TODO(whomever): This field could maybe be removed in 20.2. + settings *cluster.Settings } type dbCacheSubscriber interface { @@ -173,8 +179,7 @@ func (tc *TableCollection) getMutableTableDescriptor( if dbID == sqlbase.InvalidID && tc.databaseCache != nil { // Resolve the database from the database cache when the transaction // hasn't modified the database. - dbID, err = tc.databaseCache.getDatabaseID(ctx, - tc.leaseMgr.db.Txn, tn.Catalog(), flags.Required) + dbID, err = tc.databaseCache.getDatabaseID(ctx, tc.leaseMgr.db.Txn, tn.Catalog(), flags.Required) if err != nil || dbID == sqlbase.InvalidID { // dbID can still be invalid if required is false and the database is not found. return nil, err @@ -189,7 +194,7 @@ func (tc *TableCollection) getMutableTableDescriptor( } phyAccessor := UncachedPhysicalAccessor{} - obj, err := phyAccessor.GetObjectDesc(ctx, txn, tn, flags) + obj, err := phyAccessor.GetObjectDesc(ctx, txn, tc.settings, tn, flags) if obj == nil { return nil, err } @@ -229,8 +234,7 @@ func (tc *TableCollection) getTableVersion( if dbID == sqlbase.InvalidID && tc.databaseCache != nil { // Resolve the database from the database cache when the transaction // hasn't modified the database. - dbID, err = tc.databaseCache.getDatabaseID(ctx, - tc.leaseMgr.db.Txn, tn.Catalog(), flags.Required) + dbID, err = tc.databaseCache.getDatabaseID(ctx, tc.leaseMgr.db.Txn, tn.Catalog(), flags.Required) if err != nil || dbID == sqlbase.InvalidID { // dbID can still be invalid if required is false and the database is not found. return nil, err @@ -265,7 +269,7 @@ func (tc *TableCollection) getTableVersion( readTableFromStore := func() (*sqlbase.ImmutableTableDescriptor, error) { phyAccessor := UncachedPhysicalAccessor{} - obj, err := phyAccessor.GetObjectDesc(ctx, txn, tn, flags) + obj, err := phyAccessor.GetObjectDesc(ctx, txn, tc.settings, tn, flags) if obj == nil { return nil, err } diff --git a/pkg/sql/temporary_schema.go b/pkg/sql/temporary_schema.go new file mode 100644 index 000000000000..e5564da39ce5 --- /dev/null +++ b/pkg/sql/temporary_schema.go @@ -0,0 +1,47 @@ +// Copyright 2019 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/internal/client" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" + "github.com/cockroachdb/cockroach/pkg/util/log" +) + +func createTempSchema(params runParams, sKey sqlbase.DescriptorKey) (sqlbase.ID, error) { + id, err := GenerateUniqueDescID(params.ctx, params.extendedEvalCtx.ExecCfg.DB) + if err != nil { + return sqlbase.InvalidID, err + } + if err := params.p.createSchemaWithID(params.ctx, sKey.Key(), id); err != nil { + return sqlbase.InvalidID, err + } + + params.p.SetTemporarySchemaName(sKey.Name()) + + return id, nil +} + +func (p *planner) createSchemaWithID( + ctx context.Context, schemaNameKey roachpb.Key, schemaID sqlbase.ID, +) error { + if p.ExtendedEvalContext().Tracing.KVTracingEnabled() { + log.VEventf(ctx, 2, "CPut %s -> %d", schemaNameKey, schemaID) + } + + b := &client.Batch{} + b.CPut(schemaNameKey, schemaID, nil) + + return p.txn.Run(ctx, b) +} diff --git a/pkg/sql/tests/system_table_test.go b/pkg/sql/tests/system_table_test.go index 0355dd257730..865e6ca174ed 100644 --- a/pkg/sql/tests/system_table_test.go +++ b/pkg/sql/tests/system_table_test.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/privilege" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" @@ -28,12 +29,11 @@ import ( func TestInitialKeys(t *testing.T) { defer leaktest.AfterTest(t)() - const keysPerDesc = 2 - const nonDescKeys = 9 + const nonDescKeys = 10 ms := sqlbase.MakeMetadataSchema(config.DefaultZoneConfigRef(), config.DefaultSystemZoneConfigRef()) - kv, _ /* splits */ := ms.GetInitialValues() + kv, _ /* splits */ := ms.GetInitialValues(cluster.TestingClusterVersion) expected := nonDescKeys + keysPerDesc*ms.SystemDescriptorCount() if actual := len(kv); actual != expected { t.Fatalf("Wrong number of initial sql kv pairs: %d, wanted %d", actual, expected) @@ -52,7 +52,7 @@ func TestInitialKeys(t *testing.T) { t.Fatal(err) } ms.AddDescriptor(keys.SystemDatabaseID, &desc) - kv, _ /* splits */ = ms.GetInitialValues() + kv, _ /* splits */ = ms.GetInitialValues(cluster.TestingClusterVersion) expected = nonDescKeys + keysPerDesc*ms.SystemDescriptorCount() if actual := len(kv); actual != expected { t.Fatalf("Wrong number of initial sql kv pairs: %d, wanted %d", actual, expected) diff --git a/pkg/sql/truncate.go b/pkg/sql/truncate.go index 303c5e7e1ad1..2ba65ddbfb74 100644 --- a/pkg/sql/truncate.go +++ b/pkg/sql/truncate.go @@ -211,7 +211,9 @@ func (p *planner) truncateTable( // // TODO(vivek): Fix properly along with #12123. zoneKey := config.MakeZoneKey(uint32(tableDesc.ID)) - nameKey := sqlbase.NewTableKey(tableDesc.ParentID, tableDesc.GetName()).Key() + nameKey := sqlbase.MakePublicTableNameKey(ctx, p.ExecCfg().Settings, tableDesc.ParentID, tableDesc.GetName()).Key() + key := sqlbase.MakePublicTableNameKey(ctx, p.ExecCfg().Settings, newTableDesc.ParentID, newTableDesc.Name).Key() + b := &client.Batch{} // Use CPut because we want to remove a specific name -> id map. if traceKV { @@ -270,7 +272,6 @@ func (p *planner) truncateTable( newTableDesc.Mutations = nil newTableDesc.GCMutations = nil newTableDesc.ModificationTime = p.txn.CommitTimestamp() - key := sqlbase.NewTableKey(newTableDesc.ParentID, newTableDesc.Name).Key() if err := p.createDescriptorWithID( ctx, key, newID, newTableDesc, p.ExtendedEvalContext().Settings); err != nil { return err diff --git a/pkg/sql/vars.go b/pkg/sql/vars.go index 0fe3289cf393..ebe062fddc06 100644 --- a/pkg/sql/vars.go +++ b/pkg/sql/vars.go @@ -185,8 +185,8 @@ var varGen = map[string]sessionVar{ if len(dbName) != 0 { // Verify database descriptor exists. - if _, err := evalCtx.schemaAccessors.logical.GetDatabaseDesc(ctx, evalCtx.Txn, dbName, - tree.DatabaseLookupFlags{Required: true}); err != nil { + if _, err := evalCtx.schemaAccessors.logical.GetDatabaseDesc(ctx, evalCtx.Txn, + dbName, tree.DatabaseLookupFlags{Required: true}); err != nil { return "", err } } @@ -618,7 +618,7 @@ var varGen = map[string]sessionVar{ }, Set: func(_ context.Context, m *sessionDataMutator, s string) error { paths := strings.Split(s, ",") - m.SetSearchPath(sessiondata.MakeSearchPath(paths)) + m.UpdateSearchPath(paths) return nil }, Get: func(evalCtx *extendedEvalContext) string { diff --git a/pkg/sql/virtual_schema.go b/pkg/sql/virtual_schema.go index ddcb46a31a98..c8db49cf8aee 100644 --- a/pkg/sql/virtual_schema.go +++ b/pkg/sql/virtual_schema.go @@ -240,8 +240,8 @@ func (e virtualDefEntry) getPlanInfo() (sqlbase.ResultColumns, virtualTableConst var dbDesc *DatabaseDescriptor if dbName != "" { var err error - dbDesc, err = p.LogicalSchemaAccessor().GetDatabaseDesc(ctx, p.txn, dbName, - tree.DatabaseLookupFlags{Required: true, AvoidCached: p.avoidCachedDescriptors}) + dbDesc, err = p.LogicalSchemaAccessor().GetDatabaseDesc(ctx, p.txn, + dbName, tree.DatabaseLookupFlags{Required: true, AvoidCached: p.avoidCachedDescriptors}) if err != nil { return nil, err } diff --git a/pkg/sql/zone_config.go b/pkg/sql/zone_config.go index 7837c073ef54..a04b4e650c65 100644 --- a/pkg/sql/zone_config.go +++ b/pkg/sql/zone_config.go @@ -262,18 +262,13 @@ func resolveZone(ctx context.Context, txn *client.Txn, zs *tree.ZoneSpecifier) ( errMissingKey := errors.New("missing key") id, err := config.ResolveZoneSpecifier(zs, func(parentID uint32, name string) (uint32, error) { - tKey := sqlbase.NewTableKey(sqlbase.ID(parentID), name) - kv, err := txn.Get(ctx, tKey.Key()) + found, id, err := sqlbase.LookupPublicTableID(ctx, txn, sqlbase.ID(parentID), name) if err != nil { return 0, err } - if kv.Value == nil { + if !found { return 0, errMissingKey } - id, err := kv.Value.GetInt() - if err != nil { - return 0, err - } return uint32(id), nil }, ) diff --git a/pkg/sqlmigrations/migrations.go b/pkg/sqlmigrations/migrations.go index 3d716112af54..5f91441b32ac 100644 --- a/pkg/sqlmigrations/migrations.go +++ b/pkg/sqlmigrations/migrations.go @@ -265,7 +265,10 @@ func databaseIDs(names ...string) func(ctx context.Context, db db) ([]sqlbase.ID return func(ctx context.Context, db db) ([]sqlbase.ID, error) { var ids []sqlbase.ID for _, name := range names { - kv, err := db.Get(ctx, sqlbase.NewTableKey(keys.RootNamespaceID, name).Key()) + // This runs as part of an older migration (introduced in 2.1). We use + // the DeprecatedDatabaseKey, and let the 20.1 migration handle moving + // from the old namespace table into the new one. + kv, err := db.Get(ctx, sqlbase.NewDeprecatedDatabaseKey(name).Key()) if err != nil { return nil, err } @@ -576,7 +579,8 @@ func createSystemTable(ctx context.Context, r runner, desc sqlbase.TableDescript // the reserved ID space. (The SQL layer doesn't allow this.) err := r.db.Txn(ctx, func(ctx context.Context, txn *client.Txn) error { b := txn.NewBatch() - b.CPut(sqlbase.NewTableKey(desc.GetParentID(), desc.GetName()).Key(), desc.GetID(), nil) + tKey := sqlbase.MakePublicTableNameKey(ctx, r.sqlExecutor.Settings(), desc.GetParentID(), desc.GetName()) + b.CPut(tKey.Key(), desc.GetID(), nil) b.CPut(sqlbase.MakeDescMetadataKey(desc.GetID()), sqlbase.WrapDescriptor(&desc), nil) if err := txn.SetSystemConfigTrigger(); err != nil { return err diff --git a/pkg/sqlmigrations/migrations_test.go b/pkg/sqlmigrations/migrations_test.go index d2f84a975eef..631a347f40b9 100644 --- a/pkg/sqlmigrations/migrations_test.go +++ b/pkg/sqlmigrations/migrations_test.go @@ -499,7 +499,7 @@ func TestCreateSystemTable(t *testing.T) { sqlbase.SystemAllowedPrivileges[table.ID] = sqlbase.SystemAllowedPrivileges[keys.NamespaceTableID] table.Name = "dummy" - nameKey := sqlbase.NewTableKey(table.ParentID, table.Name).Key() + nameKey := sqlbase.NewPublicTableKey(table.ParentID, table.Name).Key() descKey := sqlbase.MakeDescMetadataKey(table.ID) descVal := sqlbase.WrapDescriptor(&table) diff --git a/pkg/storage/client_split_test.go b/pkg/storage/client_split_test.go index 151d065b4dd6..be9db2f022c6 100644 --- a/pkg/storage/client_split_test.go +++ b/pkg/storage/client_split_test.go @@ -32,6 +32,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc" "github.com/cockroachdb/cockroach/pkg/server" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/storage/abortspan" @@ -1279,7 +1280,7 @@ func TestStoreRangeSystemSplits(t *testing.T) { return err } descTablePrefix := keys.MakeTablePrefix(keys.DescriptorTableID) - kvs, _ /* splits */ := schema.GetInitialValues() + kvs, _ /* splits */ := schema.GetInitialValues(cluster.TestingClusterVersion) for _, kv := range kvs { if !bytes.HasPrefix(kv.Key, descTablePrefix) { continue diff --git a/pkg/storage/client_test.go b/pkg/storage/client_test.go index 51aa5d33c34a..66f57178e08d 100644 --- a/pkg/storage/client_test.go +++ b/pkg/storage/client_test.go @@ -186,7 +186,8 @@ func createTestStoreWithOpts( if !opts.dontBootstrap { var kvs []roachpb.KeyValue var splits []roachpb.RKey - kvs, tableSplits := sqlbase.MakeMetadataSchema(storeCfg.DefaultZoneConfig, storeCfg.DefaultSystemZoneConfig).GetInitialValues() + bootstrapVersion := cluster.ClusterVersion{Version: cluster.BinaryServerVersion} + kvs, tableSplits := sqlbase.MakeMetadataSchema(storeCfg.DefaultZoneConfig, storeCfg.DefaultSystemZoneConfig).GetInitialValues(bootstrapVersion) if !opts.dontCreateSystemRanges { splits = config.StaticSplits() splits = append(splits, tableSplits...) @@ -868,7 +869,8 @@ func (m *multiTestContext) addStore(idx int) { if needBootstrap && idx == 0 { // Bootstrap the initial range on the first engine. var splits []roachpb.RKey - kvs, tableSplits := sqlbase.MakeMetadataSchema(cfg.DefaultZoneConfig, cfg.DefaultSystemZoneConfig).GetInitialValues() + bootstrapVersion := cluster.ClusterVersion{Version: cluster.BinaryServerVersion} + kvs, tableSplits := sqlbase.MakeMetadataSchema(cfg.DefaultZoneConfig, cfg.DefaultSystemZoneConfig).GetInitialValues(bootstrapVersion) if !m.startWithSingleRange { splits = config.StaticSplits() splits = append(splits, tableSplits...) diff --git a/pkg/storage/store_test.go b/pkg/storage/store_test.go index 1d5ab11873cd..7618129948b6 100644 --- a/pkg/storage/store_test.go +++ b/pkg/storage/store_test.go @@ -225,7 +225,8 @@ func createTestStoreWithoutStart( t.Fatal(err) } var splits []roachpb.RKey - kvs, tableSplits := sqlbase.MakeMetadataSchema(cfg.DefaultZoneConfig, cfg.DefaultSystemZoneConfig).GetInitialValues() + bootstrapVersion := cluster.ClusterVersion{Version: cluster.BinaryServerVersion} + kvs, tableSplits := sqlbase.MakeMetadataSchema(cfg.DefaultZoneConfig, cfg.DefaultSystemZoneConfig).GetInitialValues(bootstrapVersion) if opts.createSystemRanges { splits = config.StaticSplits() splits = append(splits, tableSplits...) @@ -441,7 +442,8 @@ func TestStoreInitAndBootstrap(t *testing.T) { // Bootstrap the system ranges. var splits []roachpb.RKey - kvs, tableSplits := sqlbase.MakeMetadataSchema(cfg.DefaultZoneConfig, cfg.DefaultSystemZoneConfig).GetInitialValues() + bootstrapVersion := cluster.ClusterVersion{Version: cluster.BinaryServerVersion} + kvs, tableSplits := sqlbase.MakeMetadataSchema(cfg.DefaultZoneConfig, cfg.DefaultSystemZoneConfig).GetInitialValues(bootstrapVersion) splits = config.StaticSplits() splits = append(splits, tableSplits...) sort.Slice(splits, func(i, j int) bool { diff --git a/pkg/testutils/localtestcluster/local_test_cluster.go b/pkg/testutils/localtestcluster/local_test_cluster.go index 6a1f91cf76f7..f58d87fc22cf 100644 --- a/pkg/testutils/localtestcluster/local_test_cluster.go +++ b/pkg/testutils/localtestcluster/local_test_cluster.go @@ -182,7 +182,8 @@ func (ltc *LocalTestCluster) Start(t testing.TB, baseCtx *base.Config, initFacto if !ltc.DontCreateSystemRanges { schema := sqlbase.MakeMetadataSchema(cfg.DefaultZoneConfig, cfg.DefaultSystemZoneConfig) var tableSplits []roachpb.RKey - initialValues, tableSplits = schema.GetInitialValues() + bootstrapVersion := cluster.ClusterVersion{Version: cluster.BinaryServerVersion} + initialValues, tableSplits = schema.GetInitialValues(bootstrapVersion) splits = append(config.StaticSplits(), tableSplits...) sort.Slice(splits, func(i, j int) bool { return splits[i].Less(splits[j])