diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html
index 2d0f0e7b8cd4..811963ae5b9b 100644
--- a/docs/generated/settings/settings.html
+++ b/docs/generated/settings/settings.html
@@ -133,6 +133,6 @@
trace.debug.enable | boolean | false | if set, traces for recent requests can be seen in the /debug page |
trace.lightstep.token | string |
| if set, traces go to Lightstep using this token |
trace.zipkin.collector | string |
| if set, traces go to the given Zipkin instance (example: '127.0.0.1:9411'); ignored if trace.lightstep.token is set |
-version | custom validation | 19.1-11 | set the active cluster version in the format '.' |
+version | custom validation | 20.1 | set the active cluster version in the format '.' |
diff --git a/pkg/ccl/backupccl/backup.go b/pkg/ccl/backupccl/backup.go
index 7bd0654a6ccf..67867eb99723 100644
--- a/pkg/ccl/backupccl/backup.go
+++ b/pkg/ccl/backupccl/backup.go
@@ -1370,10 +1370,12 @@ 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, *cluster.Settings) error {
+ return nil
+}
// OnSuccess is part of the jobs.Resumer interface.
-func (b *backupResumer) OnSuccess(context.Context, *client.Txn) error { return nil }
+func (b *backupResumer) OnSuccess(context.Context, *client.Txn, *cluster.Settings) error { return nil }
// OnTerminal is part of the jobs.Resumer interface.
func (b *backupResumer) OnTerminal(
diff --git a/pkg/ccl/backupccl/restore.go b/pkg/ccl/backupccl/restore.go
index 5281b2626afd..026af3d0f0c1 100644
--- a/pkg/ccl/backupccl/restore.go
+++ b/pkg/ccl/backupccl/restore.go
@@ -369,7 +369,7 @@ 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)
+ dKey := sqlbase.NewDatabaseKey(name, p.ExecCfg().Settings)
existingDatabaseID, err := txn.Get(ctx, dKey.Key())
if err != nil {
return err
@@ -397,7 +397,7 @@ func allocateTableRewrites(
} else {
var parentID sqlbase.ID
{
- dKey := sqlbase.NewDatabaseKey(targetDB)
+ dKey := sqlbase.NewDatabaseKey(targetDB, p.ExecCfg().Settings)
existingDatabaseID, err := txn.Get(ctx, dKey.Key())
if err != nil {
return err
@@ -416,7 +416,7 @@ func allocateTableRewrites(
// Check that the table name is _not_ in use.
// This would fail the CPut later anyway, but this yields a prettier error.
- if err := CheckTableExists(ctx, txn, parentID, table.Name); err != nil {
+ if err := CheckTableExists(ctx, txn, p.ExecCfg().Settings, parentID, table.Name); err != nil {
return err
}
@@ -486,9 +486,13 @@ func allocateTableRewrites(
// CheckTableExists returns an error if a table already exists with given
// parent and name.
func CheckTableExists(
- ctx context.Context, txn *client.Txn, parentID sqlbase.ID, name string,
+ ctx context.Context,
+ txn *client.Txn,
+ settings *cluster.Settings,
+ parentID sqlbase.ID,
+ name string,
) error {
- tKey := sqlbase.NewTableKey(parentID, name)
+ tKey := sqlbase.NewPublicTableKey(parentID, name, settings)
res, err := txn.Get(ctx, tKey.Key())
if err != nil {
return err
@@ -1057,7 +1061,7 @@ 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)
+ b.CPut(sqlbase.NewDatabaseKey(desc.Name, settings).Key(), desc.ID, nil)
}
for i := range tables {
if wrote, ok := wroteDBs[tables[i].ParentID]; ok {
@@ -1079,7 +1083,7 @@ 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)
+ b.CPut(sqlbase.NewPublicTableKey(tables[i].ParentID, tables[i].Name, settings).Key(), tables[i].ID, nil)
}
for _, kv := range extra {
b.InitPut(kv.Key, &kv.Value, false)
@@ -1770,7 +1774,9 @@ func (r *restoreResumer) Resume(
// has been committed from a restore that has failed or been canceled. It does
// this by adding the table descriptors in DROP state, which causes the schema
// change stuff to delete the keys in the background.
-func (r *restoreResumer) OnFailOrCancel(ctx context.Context, txn *client.Txn) error {
+func (r *restoreResumer) OnFailOrCancel(
+ ctx context.Context, txn *client.Txn, settings *cluster.Settings,
+) error {
details := r.job.Details().(jobspb.RestoreDetails)
// No need to mark the tables as dropped if they were not even created in the
@@ -1791,7 +1797,7 @@ func (r *restoreResumer) OnFailOrCancel(ctx context.Context, txn *client.Txn) er
var existingIDVal roachpb.Value
existingIDVal.SetInt(int64(tableDesc.ID))
b.CPut(
- sqlbase.NewTableKey(tableDesc.ParentID, tableDesc.Name).Key(),
+ sqlbase.NewPublicTableKey(tableDesc.ParentID, tableDesc.Name, settings).Key(),
nil,
&existingIDVal,
)
@@ -1813,7 +1819,9 @@ func (r *restoreResumer) OnFailOrCancel(ctx context.Context, txn *client.Txn) er
}
// OnSuccess is part of the jobs.Resumer interface.
-func (r *restoreResumer) OnSuccess(ctx context.Context, txn *client.Txn) error {
+func (r *restoreResumer) OnSuccess(
+ ctx context.Context, txn *client.Txn, _ *cluster.Settings,
+) error {
log.Event(ctx, "making tables live")
if err := stats.InsertNewStats(ctx, r.exec, txn, r.latestStats); err != nil {
diff --git a/pkg/ccl/backupccl/targets_test.go b/pkg/ccl/backupccl/targets_test.go
index c86676699fcf..a0468f8165fa 100644
--- a/pkg/ccl/backupccl/targets_test.go
+++ b/pkg/ccl/backupccl/targets_test.go
@@ -104,7 +104,7 @@ func TestDescriptorsMatchingTargets(t *testing.T) {
// {"", `TABLE system."FOO"`, []string{"system"}},
// {"system", `TABLE "FOO"`, []string{"system"}},
}
- searchPath := sessiondata.MakeSearchPath([]string{"public", "pg_catalog"})
+ searchPath := sessiondata.MakeSearchPath([]string{"public", "pg_catalog"}, sessiondata.DefaultTemporarySchema)
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/changefeedccl/changefeed_stmt.go b/pkg/ccl/changefeedccl/changefeed_stmt.go
index 31809d0bebdf..8b94d8490c60 100644
--- a/pkg/ccl/changefeedccl/changefeed_stmt.go
+++ b/pkg/ccl/changefeedccl/changefeed_stmt.go
@@ -552,10 +552,14 @@ func (b *changefeedResumer) Resume(
}
// OnFailOrCancel is part of the jobs.Resumer interface.
-func (b *changefeedResumer) OnFailOrCancel(context.Context, *client.Txn) error { return nil }
+func (b *changefeedResumer) OnFailOrCancel(context.Context, *client.Txn, *cluster.Settings) error {
+ return nil
+}
// OnSuccess is part of the jobs.Resumer interface.
-func (b *changefeedResumer) OnSuccess(context.Context, *client.Txn) error { return nil }
+func (b *changefeedResumer) OnSuccess(context.Context, *client.Txn, *cluster.Settings) error {
+ return nil
+}
// OnTerminal is part of the jobs.Resumer interface.
func (b *changefeedResumer) OnTerminal(context.Context, jobs.Status, chan<- tree.Datums) {}
diff --git a/pkg/ccl/importccl/import_stmt.go b/pkg/ccl/importccl/import_stmt.go
index 97dba267133c..7c301375c3c4 100644
--- a/pkg/ccl/importccl/import_stmt.go
+++ b/pkg/ccl/importccl/import_stmt.go
@@ -614,7 +614,7 @@ func prepareNewTableDescsForIngestion(
) ([]*sqlbase.TableDescriptor, error) {
var tableDescs []*sqlbase.TableDescriptor
for _, i := range tables {
- if err := backupccl.CheckTableExists(ctx, txn, parentID, i.Desc.Name); err != nil {
+ if err := backupccl.CheckTableExists(ctx, txn, p.ExecCfg().Settings, parentID, i.Desc.Name); err != nil {
return nil, err
}
tableDescs = append(tableDescs, i.Desc)
@@ -865,7 +865,9 @@ func (r *importResumer) Resume(
// been committed from a import that has failed or been canceled. It does this
// by adding the table descriptors in DROP state, which causes the schema change
// stuff to delete the keys in the background.
-func (r *importResumer) OnFailOrCancel(ctx context.Context, txn *client.Txn) error {
+func (r *importResumer) OnFailOrCancel(
+ ctx context.Context, txn *client.Txn, settings *cluster.Settings,
+) error {
details := r.job.Details().(jobspb.ImportDetails)
// Needed to trigger the schema change manager.
@@ -921,7 +923,7 @@ func (r *importResumer) OnFailOrCancel(ctx context.Context, txn *client.Txn) err
tableDesc.DropTime = 1
var existingIDVal roachpb.Value
existingIDVal.SetInt(int64(tableDesc.ID))
- tKey := sqlbase.NewTableKey(tableDesc.ParentID, tableDesc.Name)
+ tKey := sqlbase.NewPublicTableKey(tableDesc.ParentID, tableDesc.Name, settings)
b.CPut(tKey.Key(), nil, &existingIDVal)
} else {
// IMPORT did not create this table, so we should not drop it.
@@ -944,7 +946,7 @@ func (r *importResumer) OnFailOrCancel(ctx context.Context, txn *client.Txn) err
}
// OnSuccess is part of the jobs.Resumer interface.
-func (r *importResumer) OnSuccess(ctx context.Context, txn *client.Txn) error {
+func (r *importResumer) OnSuccess(ctx context.Context, txn *client.Txn, _ *cluster.Settings) error {
log.Event(ctx, "making tables live")
details := r.job.Details().(jobspb.ImportDetails)
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 206b018ba891..19e7782b3967 100644
--- a/pkg/cli/cli_test.go
+++ b/pkg/cli/cli_test.go
@@ -2215,6 +2215,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
@@ -2225,6 +2227,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..da2e82ca9d37 100644
--- a/pkg/config/system_test.go
+++ b/pkg/config/system_test.go
@@ -293,7 +293,9 @@ func TestComputeSplitKeyTableIDs(t *testing.T) {
kvs, _ /* splits */ := schema.GetInitialValues()
userSQL := append(kvs, descriptor(start), descriptor(start+1), descriptor(start+5))
// Real system tables and partitioned user tables.
- subzoneSQL := append(userSQL,
+ var userSQLCopy = make([]roachpb.KeyValue, len(userSQL))
+ copy(userSQLCopy, userSQL)
+ subzoneSQL := append(userSQLCopy,
zoneConfig(start+1, subzone("a", ""), subzone("c", "e")),
zoneConfig(start+5, subzone("b", ""), subzone("c", "d"), subzone("d", "")))
@@ -404,13 +406,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},
diff --git a/pkg/jobs/helpers_test.go b/pkg/jobs/helpers_test.go
index 33dd2835f114..956b4fb4df4b 100644
--- a/pkg/jobs/helpers_test.go
+++ b/pkg/jobs/helpers_test.go
@@ -15,6 +15,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/internal/client"
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
+ "github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
)
@@ -41,21 +42,21 @@ func (d FakeResumer) Resume(_ context.Context, _ interface{}, _ chan<- tree.Datu
return nil
}
-func (d FakeResumer) OnFailOrCancel(_ context.Context, _ *client.Txn) error {
+func (d FakeResumer) OnFailOrCancel(_ context.Context, _ *client.Txn, _ *cluster.Settings) error {
if d.Fail != nil {
return d.Fail()
}
return nil
}
-func (d FakeResumer) OnSuccess(_ context.Context, _ *client.Txn) error {
+func (d FakeResumer) OnSuccess(context.Context, *client.Txn, *cluster.Settings) error {
if d.Success != nil {
return d.Success()
}
return nil
}
-func (d FakeResumer) OnTerminal(_ context.Context, _ Status, _ chan<- tree.Datums) {
+func (d FakeResumer) OnTerminal(context.Context, Status, chan<- tree.Datums) {
if d.Terminal != nil {
d.Terminal()
}
diff --git a/pkg/jobs/jobs.go b/pkg/jobs/jobs.go
index df50e1e30771..e88c856c3ec8 100644
--- a/pkg/jobs/jobs.go
+++ b/pkg/jobs/jobs.go
@@ -16,6 +16,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/internal/client"
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
+ "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/hlc"
@@ -314,7 +315,9 @@ func (j *Job) resumed(ctx context.Context) error {
// Canceled sets the status of the tracked job to canceled. It does not directly
// cancel the job; like job.Paused, it expects the job to call job.Progressed
// soon, observe a "job is canceled" error, and abort further work.
-func (j *Job) canceled(ctx context.Context, fn func(context.Context, *client.Txn) error) error {
+func (j *Job) canceled(
+ ctx context.Context, fn func(context.Context, *client.Txn, *cluster.Settings) error,
+) error {
return j.Update(ctx, func(txn *client.Txn, md JobMetadata, ju *JobUpdater) error {
if md.Status == StatusCanceled {
// Already canceled - do nothing.
@@ -327,7 +330,7 @@ func (j *Job) canceled(ctx context.Context, fn func(context.Context, *client.Txn
return fmt.Errorf("job with status %s cannot be canceled", md.Status)
}
if fn != nil {
- if err := fn(ctx, txn); err != nil {
+ if err := fn(ctx, txn, j.registry.settings); err != nil {
return err
}
}
@@ -340,18 +343,18 @@ func (j *Job) canceled(ctx context.Context, fn func(context.Context, *client.Txn
// NoopFn is an empty function that can be used for Failed and Succeeded. It indicates
// no transactional callback should be made during these operations.
-var NoopFn = func(context.Context, *client.Txn) error { return nil }
+var NoopFn = func(context.Context, *client.Txn, *cluster.Settings) error { return nil }
// Failed marks the tracked job as having failed with the given error.
func (j *Job) Failed(
- ctx context.Context, err error, fn func(context.Context, *client.Txn) error,
+ ctx context.Context, err error, fn func(context.Context, *client.Txn, *cluster.Settings) error,
) error {
return j.Update(ctx, func(txn *client.Txn, md JobMetadata, ju *JobUpdater) error {
if md.Status.Terminal() {
// Already done - do nothing.
return nil
}
- if err := fn(ctx, txn); err != nil {
+ if err := fn(ctx, txn, j.registry.settings); err != nil {
return err
}
ju.UpdateStatus(StatusFailed)
@@ -364,13 +367,15 @@ func (j *Job) Failed(
// Succeeded marks the tracked job as having succeeded and sets its fraction
// completed to 1.0.
-func (j *Job) Succeeded(ctx context.Context, fn func(context.Context, *client.Txn) error) error {
+func (j *Job) Succeeded(
+ ctx context.Context, fn func(context.Context, *client.Txn, *cluster.Settings) error,
+) error {
return j.Update(ctx, func(txn *client.Txn, md JobMetadata, ju *JobUpdater) error {
if md.Status.Terminal() {
// Already done - do nothing.
return nil
}
- if err := fn(ctx, txn); err != nil {
+ if err := fn(ctx, txn, j.registry.settings); err != nil {
return err
}
ju.UpdateStatus(StatusSucceeded)
diff --git a/pkg/jobs/registry.go b/pkg/jobs/registry.go
index 7c0c485a4693..27b0b8ea4892 100644
--- a/pkg/jobs/registry.go
+++ b/pkg/jobs/registry.go
@@ -516,7 +516,7 @@ type Resumer interface {
//
// Any work this function does must still be correct if the txn is aborted at
// a later time.
- OnSuccess(ctx context.Context, txn *client.Txn) error
+ OnSuccess(ctx context.Context, txn *client.Txn, settings *cluster.Settings) error
// OnTerminal is called after a job has successfully been marked as
// terminal. It should be used to perform optional cleanup and return final
@@ -533,7 +533,7 @@ type Resumer interface {
// This method can be called during cancellation, which is not guaranteed to
// run on the node where the job is running. So it cannot assume that any
// other methods have been called on this Resumer object.
- OnFailOrCancel(ctx context.Context, txn *client.Txn) error
+ OnFailOrCancel(ctx context.Context, txn *client.Txn, settings *cluster.Settings) error
}
// Constructor creates a resumable job of a certain type. The Resumer is
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 76db6981b767..15e6604e9a5e 100644
--- a/pkg/server/admin_test.go
+++ b/pkg/server/admin_test.go
@@ -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/settings/cluster/cockroach_versions.go b/pkg/settings/cluster/cockroach_versions.go
index 249c84c4e4cf..e7b63307a35d 100644
--- a/pkg/settings/cluster/cockroach_versions.go
+++ b/pkg/settings/cluster/cockroach_versions.go
@@ -46,6 +46,7 @@ const (
VersionAtomicChangeReplicas
VersionTableDescModificationTimeFromMVCC
VersionPartitionedBackup
+ VersionNamespaceTableUngossip
// Add new versions here (step one of two).
@@ -551,6 +552,12 @@ var versionsSingleton = keyedVersions([]keyedVersion{
Key: VersionPartitionedBackup,
Version: roachpb.Version{Major: 19, Minor: 1, Unstable: 11},
},
+ {
+ // TODO(arul): fill this
+ // VersionNamespaceTableUngossip is
+ Key: VersionNamespaceTableUngossip,
+ Version: roachpb.Version{Major: 20, Minor: 1},
+ },
// 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 0b734f192b9e..441068a4d41f 100644
--- a/pkg/settings/cluster/versionkey_string.go
+++ b/pkg/settings/cluster/versionkey_string.go
@@ -23,11 +23,12 @@ func _() {
_ = x[VersionAtomicChangeReplicas-12]
_ = x[VersionTableDescModificationTimeFromMVCC-13]
_ = x[VersionPartitionedBackup-14]
+ _ = x[VersionNamespaceTableUngossip-15]
}
-const _VersionKey_name = "Version2_1VersionUnreplicatedRaftTruncatedStateVersionSideloadedStorageNoReplicaIDVersion19_1VersionStart19_2VersionQueryTxnTimestampVersionStickyBitVersionParallelCommitsVersionGenerationComparableVersionLearnerReplicasVersionTopLevelForeignKeysVersionAtomicChangeReplicasTriggerVersionAtomicChangeReplicasVersionTableDescModificationTimeFromMVCCVersionPartitionedBackup"
+const _VersionKey_name = "Version2_1VersionUnreplicatedRaftTruncatedStateVersionSideloadedStorageNoReplicaIDVersion19_1VersionStart19_2VersionQueryTxnTimestampVersionStickyBitVersionParallelCommitsVersionGenerationComparableVersionLearnerReplicasVersionTopLevelForeignKeysVersionAtomicChangeReplicasTriggerVersionAtomicChangeReplicasVersionTableDescModificationTimeFromMVCCVersionPartitionedBackupVersionNamespaceTableUngossip"
-var _VersionKey_index = [...]uint16{0, 10, 47, 82, 93, 109, 133, 149, 171, 198, 220, 246, 280, 307, 347, 371}
+var _VersionKey_index = [...]uint16{0, 10, 47, 82, 93, 109, 133, 149, 171, 198, 220, 246, 280, 307, 347, 371, 400}
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 b0d6ae40bbaf..31fd5737a05b 100644
--- a/pkg/sql/backfill.go
+++ b/pkg/sql/backfill.go
@@ -1302,7 +1302,7 @@ func runSchemaChangesInTxn(
// cleanup for later.
b := planner.Txn().NewBatch()
for _, drain := range tableDesc.DrainingNames {
- tbKey := sqlbase.NewTableKey(drain.ParentID, drain.Name).Key()
+ tbKey := sqlbase.NewPublicTableKey(drain.ParentID, drain.Name, planner.ExecCfg().Settings).Key()
b.Del(tbKey)
}
tableDesc.DrainingNames = nil
diff --git a/pkg/sql/conn_executor.go b/pkg/sql/conn_executor.go
index cb2317101387..522a2a63b481 100644
--- a/pkg/sql/conn_executor.go
+++ b/pkg/sql/conn_executor.go
@@ -2041,7 +2041,7 @@ func (ex *connExecutor) txnStateTransitionsApplyWrapper(
}
// Wait for the cache to reflect the dropped databases if any.
- ex.extraTxnState.tables.waitForCacheToDropDatabases(ex.Ctx())
+ ex.extraTxnState.tables.waitForCacheToDropDatabases(ex.Ctx(), ex.server.cfg.Settings)
fallthrough
case txnRestart, txnAborted:
diff --git a/pkg/sql/create_sequence.go b/pkg/sql/create_sequence.go
index 49e5e08229b3..2f97606a2b7e 100644
--- a/pkg/sql/create_sequence.go
+++ b/pkg/sql/create_sequence.go
@@ -51,7 +51,7 @@ 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.NewPublicTableKey(n.dbDesc.ID, n.n.Name.Table(), params.ExecCfg().Settings)
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.
@@ -91,7 +91,7 @@ func doCreateSequence(
// makeSequenceTableDesc already validates the table. No call to
// desc.ValidateTable() needed here.
- key := sqlbase.NewTableKey(dbDesc.ID, name.Table()).Key()
+ key := sqlbase.NewPublicTableKey(dbDesc.ID, name.Table(), params.ExecCfg().Settings).Key()
if err = params.p.createDescriptorWithID(params.ctx, key, id, &desc, params.EvalContext().Settings); err != nil {
return err
}
diff --git a/pkg/sql/create_stats.go b/pkg/sql/create_stats.go
index 00d24ded7bb5..a54800cf8948 100644
--- a/pkg/sql/create_stats.go
+++ b/pkg/sql/create_stats.go
@@ -455,12 +455,14 @@ func checkRunningJobs(ctx context.Context, job *jobs.Job, p *planner) error {
}
// OnFailOrCancel is part of the jobs.Resumer interface.
-func (r *createStatsResumer) OnFailOrCancel(ctx context.Context, txn *client.Txn) error {
+func (r *createStatsResumer) OnFailOrCancel(context.Context, *client.Txn, *cluster.Settings) error {
return nil
}
// OnSuccess is part of the jobs.Resumer interface.
-func (r *createStatsResumer) OnSuccess(ctx context.Context, _ *client.Txn) error {
+func (r *createStatsResumer) OnSuccess(
+ ctx context.Context, _ *client.Txn, _ *cluster.Settings,
+) error {
details := r.job.Details().(jobspb.CreateStatsDetails)
// Invalidate the local cache synchronously; this guarantees that the next
diff --git a/pkg/sql/create_table.go b/pkg/sql/create_table.go
index 844c8458d11c..66f63ab7b531 100644
--- a/pkg/sql/create_table.go
+++ b/pkg/sql/create_table.go
@@ -26,6 +26,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/row"
"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"
@@ -62,15 +63,37 @@ type createTableRun struct {
}
func (n *createTableNode) startExec(params runParams) error {
- temporary := false
- if n.n.Temporary {
+ temporary := n.n.Temporary || n.n.Table.SchemaName == sessiondata.PgTempSchemaName
+ tKey := sqlbase.NewPublicTableKey(n.dbDesc.ID, n.n.Table.Table(), params.ExecCfg().Settings)
+
+ // If a user specifies the pg_temp schema, even without the TEMPORARY keyword,
+ // a temporary table should be created.
+ if n.n.Temporary || n.n.Table.SchemaName == sessiondata.PgTempSchemaName {
+ // An explicit schema can only be provided in the CREATE TEMP TABLE statement
+ // iff it is pg_temp.
+ if n.n.Temporary && n.n.Table.ExplicitSchema && n.n.Table.SchemaName != sessiondata.PgTempSchemaName {
+ return pgerror.New(pgcode.InvalidTableDefinition, "cannot create temporary relation in non-temporary schema")
+ }
if !params.SessionData().TempTablesEnabled {
return unimplemented.NewWithIssuef(5807,
"temporary tables are unsupported")
}
- temporary = true
+
+ tempSchemaName := params.p.TemporarySchema()
+ 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(), params.ExecCfg().Settings)
}
- 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 {
diff --git a/pkg/sql/create_test.go b/pkg/sql/create_test.go
index 614d1a4edde1..12354b2945e9 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.
@@ -52,7 +51,7 @@ func TestDatabaseDescriptor(t *testing.T) {
}
// Database name.
- nameKey := sqlbase.NewDatabaseKey("test").Key()
+ nameKey := sqlbase.NewDatabaseKey("test", nil /* settings */).Key()
if gr, err := kvDB.Get(ctx, nameKey); err != nil {
t.Fatal(err)
} else if gr.Exists() {
@@ -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 migration associated with it yet. 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..7c402794d7c4 100644
--- a/pkg/sql/create_view.go
+++ b/pkg/sql/create_view.go
@@ -46,7 +46,7 @@ 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.NewPublicTableKey(n.dbDesc.ID, viewName, params.ExecCfg().Settings)
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..6f52df83e44f 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"
@@ -75,10 +76,10 @@ 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,
+ dbDesc *sqlbase.DatabaseDescriptor, settings *cluster.Settings,
) (zoneKey roachpb.Key, nameKey roachpb.Key, descKey roachpb.Key) {
zoneKey = config.MakeZoneKey(uint32(dbDesc.ID))
- nameKey = sqlbase.NewDatabaseKey(dbDesc.GetName()).Key()
+ nameKey = sqlbase.NewDatabaseKey(dbDesc.GetName(), settings).Key()
descKey = sqlbase.MakeDescMetadataKey(dbDesc.ID)
return
}
@@ -86,12 +87,12 @@ func getKeysForDatabaseDescriptor(
// getDatabaseID resolves a database name into a database ID.
// Returns InvalidID on failure.
func getDatabaseID(
- ctx context.Context, txn *client.Txn, name string, required bool,
+ ctx context.Context, txn *client.Txn, settings *cluster.Settings, name string, required bool,
) (sqlbase.ID, error) {
if name == sqlbase.SystemDB.Name {
return sqlbase.SystemDB.ID, nil
}
- dbID, err := getDescriptorID(ctx, txn, sqlbase.NewDatabaseKey(name))
+ dbID, err := getDescriptorID(ctx, txn, sqlbase.NewDatabaseKey(name, settings))
if err != nil {
return sqlbase.InvalidID, err
}
@@ -132,8 +133,10 @@ func MustGetDatabaseDescByID(
// getCachedDatabaseDesc looks up the database descriptor from the descriptor cache,
// given its name. Returns nil and no error if the name is not present in the
// cache.
-func (dc *databaseCache) getCachedDatabaseDesc(name string) (*sqlbase.DatabaseDescriptor, error) {
- dbID, err := dc.getCachedDatabaseID(name)
+func (dc *databaseCache) getCachedDatabaseDesc(
+ name string, settings *cluster.Settings,
+) (*sqlbase.DatabaseDescriptor, error) {
+ dbID, err := dc.getCachedDatabaseID(name, settings)
if dbID == sqlbase.InvalidID || err != nil {
return nil, err
}
@@ -176,6 +179,7 @@ func (dc *databaseCache) getCachedDatabaseDescByID(
// if it exists in the cache, otherwise falls back to KV operations.
func (dc *databaseCache) getDatabaseDesc(
ctx context.Context,
+ settings *cluster.Settings,
txnRunner func(context.Context, func(context.Context, *client.Txn) error) error,
name string,
required bool,
@@ -184,14 +188,14 @@ func (dc *databaseCache) getDatabaseDesc(
// isn't present. The cache might cause the usage of a recently renamed
// database, but that's a race that could occur anyways.
// The cache lookup may fail.
- desc, err := dc.getCachedDatabaseDesc(name)
+ desc, err := dc.getCachedDatabaseDesc(name, settings)
if err != nil {
return nil, err
}
if desc == nil {
if err := txnRunner(ctx, func(ctx context.Context, txn *client.Txn) error {
a := UncachedPhysicalAccessor{}
- desc, err = a.GetDatabaseDesc(ctx, txn, name,
+ desc, err = a.GetDatabaseDesc(ctx, txn, settings, name,
tree.DatabaseLookupFlags{Required: required})
return err
}); err != nil {
@@ -224,18 +228,19 @@ func (dc *databaseCache) getDatabaseDescByID(
// operations.
func (dc *databaseCache) getDatabaseID(
ctx context.Context,
+ settings *cluster.Settings,
txnRunner func(context.Context, func(context.Context, *client.Txn) error) error,
name string,
required bool,
) (sqlbase.ID, error) {
- dbID, err := dc.getCachedDatabaseID(name)
+ dbID, err := dc.getCachedDatabaseID(name, settings)
if err != nil {
return dbID, err
}
if dbID == sqlbase.InvalidID {
if err := txnRunner(ctx, func(ctx context.Context, txn *client.Txn) error {
var err error
- dbID, err = getDatabaseID(ctx, txn, name, required)
+ dbID, err = getDatabaseID(ctx, txn, settings, name, required)
return err
}); err != nil {
return sqlbase.InvalidID, err
@@ -249,7 +254,9 @@ func (dc *databaseCache) getDatabaseID(
// from the cache. This method never goes to the store to resolve
// the name to id mapping. Returns InvalidID if the name to id mapping or
// the database descriptor are not in the cache.
-func (dc *databaseCache) getCachedDatabaseID(name string) (sqlbase.ID, error) {
+func (dc *databaseCache) getCachedDatabaseID(
+ name string, settings *cluster.Settings,
+) (sqlbase.ID, error) {
if id := dc.getID(name); id != sqlbase.InvalidID {
return id, nil
}
@@ -258,7 +265,7 @@ func (dc *databaseCache) getCachedDatabaseID(name string) (sqlbase.ID, error) {
return sqlbase.SystemDB.ID, nil
}
- nameKey := sqlbase.NewDatabaseKey(name)
+ nameKey := sqlbase.NewDatabaseKey(name, settings)
nameVal := dc.systemConfig.GetValue(nameKey.Key())
if nameVal == nil {
return sqlbase.InvalidID, nil
@@ -270,7 +277,10 @@ func (dc *databaseCache) getCachedDatabaseID(name string) (sqlbase.ID, error) {
// renameDatabase implements the DatabaseDescEditor interface.
func (p *planner) renameDatabase(
- ctx context.Context, oldDesc *sqlbase.DatabaseDescriptor, newName string,
+ ctx context.Context,
+ settings *cluster.Settings,
+ oldDesc *sqlbase.DatabaseDescriptor,
+ newName string,
) error {
oldName := oldDesc.Name
oldDesc.SetName(newName)
@@ -278,8 +288,8 @@ func (p *planner) renameDatabase(
return err
}
- oldKey := sqlbase.NewDatabaseKey(oldName).Key()
- newKey := sqlbase.NewDatabaseKey(newName).Key()
+ oldKey := sqlbase.NewDatabaseKey(oldName, settings).Key()
+ newKey := sqlbase.NewDatabaseKey(newName, settings).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 0d7b0a94c74a..90bd1e5d859b 100644
--- a/pkg/sql/descriptor.go
+++ b/pkg/sql/descriptor.go
@@ -60,7 +60,7 @@ 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)
+ plainKey := sqlbase.NewDatabaseKey(desc.Name, p.ExecCfg().Settings)
idKey := plainKey.Key()
if exists, err := descExists(ctx, p.txn, idKey); err == nil && exists {
@@ -78,7 +78,16 @@ 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
+ }
+
+ // 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) {
@@ -232,9 +241,11 @@ func GetAllDescriptors(ctx context.Context, txn *client.Txn) ([]sqlbase.Descript
// GetAllDatabaseDescriptorIDs looks up and returns all available database
// descriptor IDs.
-func GetAllDatabaseDescriptorIDs(ctx context.Context, txn *client.Txn) ([]sqlbase.ID, error) {
+func GetAllDatabaseDescriptorIDs(
+ ctx context.Context, txn *client.Txn, settings *cluster.Settings,
+) ([]sqlbase.ID, error) {
log.Eventf(ctx, "fetching all database descriptor IDs")
- nameKey := sqlbase.NewDatabaseKey("" /* name */).Key()
+ nameKey := sqlbase.NewDatabaseKey("" /* name */, settings).Key()
kvs, err := txn.Scan(ctx, nameKey, nameKey.PrefixEnd(), 0 /*maxRows */)
if err != nil {
return nil, err
diff --git a/pkg/sql/distsql/server.go b/pkg/sql/distsql/server.go
index 85f1898b8fa5..325a816b816c 100644
--- a/pkg/sql/distsql/server.go
+++ b/pkg/sql/distsql/server.go
@@ -246,7 +246,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.TemporarySchema),
SequenceState: sessiondata.NewSequenceState(),
DataConversion: sessiondata.DataConversionConfig{
Location: location,
diff --git a/pkg/sql/drop_database.go b/pkg/sql/drop_database.go
index 8be0190d8839..e5879c69a231 100644
--- a/pkg/sql/drop_database.go
+++ b/pkg/sql/drop_database.go
@@ -62,7 +62,7 @@ func (p *planner) DropDatabase(ctx context.Context, n *tree.DropDatabase) (planN
return nil, err
}
- tbNames, err := GetObjectNames(ctx, p.txn, p, dbDesc, tree.PublicSchema, true /*explicitPrefix*/)
+ tbNames, err := GetObjectNames(ctx, p.txn, p.ExecCfg().Settings, p, dbDesc, tree.PublicSchema, true /*explicitPrefix*/)
if err != nil {
return nil, err
}
@@ -159,7 +159,7 @@ func (n *dropDatabaseNode) startExec(params runParams) error {
tbNameStrings = append(tbNameStrings, toDel.tn.FQString())
}
- _ /* zoneKey */, nameKey, descKey := getKeysForDatabaseDescriptor(n.dbDesc)
+ _ /* zoneKey */, nameKey, descKey := getKeysForDatabaseDescriptor(n.dbDesc, params.ExecCfg().Settings)
b := &client.Batch{}
if p.ExtendedEvalContext().Tracing.KVTracingEnabled() {
diff --git a/pkg/sql/drop_test.go b/pkg/sql/drop_test.go
index f86c2d6da264..740ac908aa80 100644
--- a/pkg/sql/drop_test.go
+++ b/pkg/sql/drop_test.go
@@ -133,7 +133,7 @@ INSERT INTO t.kv VALUES ('c', 'e'), ('a', 'c'), ('b', 'd');
t.Fatal(err)
}
- dbNameKey := sqlbase.NewDatabaseKey("t").Key()
+ dbNameKey := sqlbase.NewDatabaseKey("t", nil /* settings */).Key()
r, err := kvDB.Get(ctx, dbNameKey)
if err != nil {
t.Fatal(err)
@@ -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", nil /* settings */).Key()
gr, err := kvDB.Get(ctx, tbNameKey)
if err != nil {
t.Fatal(err)
@@ -253,7 +253,7 @@ CREATE DATABASE t;
t.Fatal(err)
}
- dKey := sqlbase.NewDatabaseKey("t")
+ dKey := sqlbase.NewDatabaseKey("t", nil /* settings */)
r, err := kvDB.Get(ctx, dKey.Key())
if err != nil {
t.Fatal(err)
@@ -307,7 +307,7 @@ INSERT INTO t.kv2 VALUES ('c', 'd'), ('a', 'b'), ('e', 'a');
t.Fatal(err)
}
- dKey := sqlbase.NewDatabaseKey("t")
+ dKey := sqlbase.NewDatabaseKey("t", nil /* settings */)
r, err := kvDB.Get(ctx, dKey.Key())
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", nil /* settings */)
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", nil /* settings */)
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", nil /* settings */).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, nil /* settings */).Key()
gr, err := kvDB.Get(ctx, nameKey)
if err != nil {
t.Fatal(err)
diff --git a/pkg/sql/exec_util.go b/pkg/sql/exec_util.go
index 5a6d54ea6e4d..3745b3050c71 100644
--- a/pkg/sql/exec_util.go
+++ b/pkg/sql/exec_util.go
@@ -1756,6 +1756,10 @@ func (m *sessionDataMutator) SetDatabase(dbName string) {
m.data.Database = dbName
}
+func (m *sessionDataMutator) SetTemporarySchemaName(scName string) {
+ m.data.SearchPath.UpdateTemporarySchemaName(scName)
+}
+
func (m *sessionDataMutator) SetDefaultIntSize(size int) {
m.data.DefaultIntSize = size
}
@@ -1804,8 +1808,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.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..fc6574c0554f 100644
--- a/pkg/sql/execinfrapb/api.go
+++ b/pkg/sql/execinfrapb/api.go
@@ -49,6 +49,7 @@ func MakeEvalContext(evalCtx *tree.EvalContext) EvalContext {
TxnTimestampNanos: evalCtx.TxnTimestamp.UnixNano(),
Location: evalCtx.GetLocation().String(),
Database: evalCtx.SessionData.Database,
+ TemporarySchema: evalCtx.SessionData.SearchPath.GetTemporarySchema(),
User: evalCtx.SessionData.User,
ApplicationName: evalCtx.SessionData.ApplicationName,
BytesEncodeFormat: be,
diff --git a/pkg/sql/execinfrapb/api.pb.go b/pkg/sql/execinfrapb/api.pb.go
index e12e10bbc7fe..ce5e29bb84f2 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_9c53d57cdfada992, []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_9c53d57cdfada992, []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_9c53d57cdfada992, []int{1}
}
func (m *FlowSpec) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -168,6 +168,7 @@ type EvalContext struct {
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"`
+ TemporarySchema string `protobuf:"bytes,13,opt,name=temporarySchema" json:"temporarySchema"`
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"`
@@ -180,7 +181,7 @@ 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_9c53d57cdfada992, []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_9c53d57cdfada992, []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_9c53d57cdfada992, []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_9c53d57cdfada992, []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_9c53d57cdfada992, []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_9c53d57cdfada992, []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_9c53d57cdfada992, []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.TemporarySchema)))
+ i += copy(dAtA[i:], m.TemporarySchema)
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.TemporarySchema)
+ 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 TemporarySchema", 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.TemporarySchema = string(dAtA[iNdEx:postIndex])
+ iNdEx = postIndex
default:
iNdEx = preIndex
skippy, err := skipApi(dAtA[iNdEx:])
@@ -2597,82 +2633,83 @@ var (
ErrIntOverflowApi = fmt.Errorf("proto: integer overflow")
)
-func init() { proto.RegisterFile("sql/execinfrapb/api.proto", fileDescriptor_api_cc1a98810c407fa3) }
+func init() { proto.RegisterFile("sql/execinfrapb/api.proto", fileDescriptor_api_9c53d57cdfada992) }
-var fileDescriptor_api_cc1a98810c407fa3 = []byte{
- // 1180 bytes of a gzipped FileDescriptorProto
+var fileDescriptor_api_9c53d57cdfada992 = []byte{
+ // 1197 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,
+ 0x13, 0x16, 0x25, 0xd9, 0x92, 0x46, 0x96, 0x23, 0xef, 0x2f, 0xf8, 0x81, 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,
+ 0x4b, 0xa6, 0x41, 0xd0, 0x43, 0x89, 0x15, 0x39, 0x96, 0x88, 0x90, 0x5c, 0x8a, 0xbb, 0xf2, 0x9f,
+ 0x3e, 0x41, 0x8f, 0x79, 0x84, 0x1e, 0xfa, 0x22, 0x05, 0x7a, 0xf0, 0x31, 0x87, 0x1e, 0x82, 0x1e,
+ 0xdc, 0xd6, 0x7e, 0x8b, 0x9e, 0x8a, 0x5d, 0x92, 0x36, 0x2d, 0xd7, 0xae, 0x7b, 0xe3, 0xce, 0xf7,
+ 0x7d, 0xb3, 0xbb, 0x33, 0xb3, 0x33, 0x84, 0xb7, 0xf8, 0xcc, 0xeb, 0xe3, 0x01, 0xda, 0x6e, 0xb0,
+ 0x1b, 0xd1, 0x70, 0xdc, 0xa7, 0xa1, 0xdb, 0x0b, 0x23, 0x26, 0x18, 0xd1, 0x6d, 0x66, 0xbf, 0x8c,
+ 0x18, 0xb5, 0xa7, 0x3d, 0x3e, 0xf3, 0x7a, 0x8e, 0xcb, 0x05, 0x9f, 0x79, 0xd1, 0x3c, 0x68, 0xdc,
+ 0x9e, 0xb0, 0x09, 0x53, 0xa4, 0xbe, 0xfc, 0x8a, 0xf9, 0x8d, 0xd6, 0x84, 0xb1, 0x89, 0x87, 0x7d,
+ 0xb5, 0x1a, 0xcf, 0x77, 0xfb, 0xc2, 0xf5, 0x91, 0x0b, 0xea, 0x87, 0x09, 0x81, 0x28, 0x67, 0xe1,
+ 0xb8, 0xef, 0x50, 0x41, 0x13, 0x5b, 0x63, 0x71, 0xff, 0x0c, 0xd6, 0x5e, 0xc4, 0xc2, 0x88, 0xd9,
+ 0xc8, 0x39, 0x8b, 0x78, 0xcc, 0xe8, 0xfc, 0x92, 0x87, 0xba, 0x89, 0x62, 0x1e, 0x3e, 0xf2, 0xd8,
+ 0xbe, 0x81, 0xb3, 0x39, 0x72, 0x41, 0x3e, 0x87, 0xe2, 0xae, 0xc7, 0xf6, 0xf5, 0x42, 0x5b, 0xeb,
+ 0x56, 0xd7, 0x3b, 0xbd, 0xab, 0xae, 0xd1, 0x93, 0x22, 0x33, 0x44, 0x7b, 0x50, 0x3c, 0x3a, 0x6e,
+ 0xe5, 0x0c, 0xa5, 0x22, 0xf7, 0xa0, 0xb4, 0x87, 0x11, 0x77, 0x59, 0xa0, 0x2f, 0xb5, 0xb5, 0x6e,
+ 0x6d, 0xf0, 0x7f, 0x09, 0xfe, 0x75, 0xdc, 0x5a, 0x1d, 0xba, 0x5c, 0x98, 0x5f, 0x7f, 0xf5, 0x3c,
+ 0x46, 0x8d, 0x94, 0x46, 0xb6, 0xa1, 0x8a, 0x7b, 0xd4, 0xdb, 0x60, 0x81, 0xc0, 0x03, 0xa1, 0x2f,
+ 0xab, 0x6d, 0xdf, 0xbd, 0x7a, 0xdb, 0xd1, 0x39, 0x39, 0xd9, 0x39, 0xab, 0x27, 0x23, 0x58, 0x15,
+ 0x07, 0x81, 0x65, 0x33, 0x16, 0x39, 0x96, 0x8f, 0x82, 0xea, 0x25, 0xe5, 0xb1, 0x95, 0xf1, 0x98,
+ 0x04, 0xb2, 0xf7, 0xec, 0x20, 0xd8, 0x90, 0xbc, 0x6d, 0x14, 0xd4, 0x58, 0x11, 0x99, 0x15, 0x69,
+ 0x42, 0xe9, 0x59, 0x44, 0x6d, 0x7c, 0xf2, 0x5c, 0x2f, 0xb7, 0xb5, 0x6e, 0x39, 0xd9, 0x2a, 0x35,
+ 0x6e, 0x15, 0xcb, 0x5a, 0x3d, 0xbf, 0x55, 0x2c, 0xe7, 0xeb, 0x85, 0xce, 0xa9, 0x06, 0xe5, 0x34,
+ 0x18, 0xe4, 0x23, 0x28, 0xc9, 0x40, 0x58, 0xae, 0xa3, 0x6b, 0x6d, 0xad, 0xbb, 0x32, 0xd0, 0xa5,
+ 0xf0, 0xb7, 0xe3, 0xd6, 0xb2, 0xa4, 0x6c, 0x0e, 0x4f, 0xce, 0xbe, 0x8c, 0x65, 0x49, 0xdc, 0x74,
+ 0xc8, 0x36, 0xc0, 0x79, 0x6a, 0xf4, 0x7c, 0xbb, 0xd0, 0xad, 0xae, 0xbf, 0x7f, 0x75, 0x00, 0x76,
+ 0x52, 0x6e, 0x26, 0xf8, 0x19, 0x07, 0xe4, 0x1b, 0x28, 0x4d, 0xa8, 0xc0, 0x7d, 0x7a, 0xa8, 0x72,
+ 0xb8, 0x34, 0xf8, 0x2c, 0x49, 0xc1, 0xfd, 0x89, 0x2b, 0xa6, 0xf3, 0x71, 0xcf, 0x66, 0x7e, 0xff,
+ 0xcc, 0xbb, 0x33, 0x3e, 0xff, 0xee, 0x87, 0x2f, 0x27, 0xfd, 0x34, 0x38, 0x4f, 0x99, 0x83, 0x9b,
+ 0x43, 0x23, 0xf5, 0xd5, 0x39, 0x2d, 0x42, 0x35, 0x13, 0x7b, 0xf2, 0x00, 0x08, 0x17, 0xbe, 0x78,
+ 0x96, 0x56, 0xe9, 0x53, 0x1a, 0x30, 0xae, 0xee, 0x5c, 0x48, 0x0e, 0xf5, 0x0f, 0x38, 0x59, 0x87,
+ 0x35, 0x71, 0x10, 0x2c, 0x88, 0xf2, 0x19, 0xd1, 0x65, 0x98, 0xb4, 0xa1, 0xec, 0x31, 0x9b, 0x0a,
+ 0x59, 0x54, 0xc5, 0xb6, 0xd6, 0xad, 0x24, 0xd4, 0x33, 0xab, 0x64, 0xc8, 0xc2, 0x1f, 0x53, 0x8e,
+ 0xaa, 0xec, 0xce, 0x18, 0xa9, 0x95, 0x34, 0x01, 0x38, 0xd2, 0xc8, 0x9e, 0xee, 0x50, 0x31, 0xd5,
+ 0x97, 0xdb, 0x85, 0x6e, 0xc5, 0xc8, 0x58, 0x88, 0x0e, 0xc5, 0x39, 0xc7, 0x48, 0x15, 0x4b, 0xaa,
+ 0x56, 0x16, 0xb2, 0x05, 0x15, 0x8e, 0x33, 0x8b, 0x0b, 0x2a, 0x50, 0xd5, 0xc2, 0xb5, 0xc9, 0x31,
+ 0xe5, 0x2b, 0x0a, 0x6c, 0x34, 0x25, 0x3d, 0x3d, 0x05, 0xc7, 0x99, 0x5a, 0x93, 0x3e, 0xd4, 0x69,
+ 0x18, 0x7a, 0x6e, 0x7c, 0x6c, 0x2b, 0xa0, 0x3e, 0xea, 0x95, 0xcc, 0x8e, 0xb7, 0x32, 0xe8, 0x53,
+ 0xea, 0x23, 0xa1, 0xf0, 0xbf, 0xf1, 0xa1, 0x40, 0x6e, 0x61, 0x60, 0x33, 0x07, 0xad, 0x5d, 0x16,
+ 0xf9, 0x54, 0xe8, 0xd0, 0xd6, 0xba, 0xab, 0xeb, 0x77, 0xaf, 0x3e, 0xc6, 0x40, 0x8a, 0x46, 0x4a,
+ 0xf3, 0x48, 0x49, 0xd2, 0xe8, 0x8e, 0x17, 0x01, 0xb2, 0x0e, 0x04, 0x0f, 0x44, 0x44, 0xad, 0x5d,
+ 0x8f, 0x51, 0x61, 0x39, 0xee, 0xc4, 0x15, 0x5c, 0xaf, 0xaa, 0xca, 0x89, 0x45, 0x75, 0x85, 0x3f,
+ 0x92, 0xf0, 0x50, 0xa1, 0xa4, 0x03, 0x95, 0x3d, 0xb4, 0x05, 0x8b, 0xdc, 0xef, 0x51, 0x5f, 0xc9,
+ 0x50, 0xcf, 0xcd, 0xa4, 0x07, 0xb7, 0x04, 0xfa, 0x21, 0x8b, 0x68, 0x74, 0x68, 0xda, 0x53, 0xf4,
+ 0xa9, 0x5e, 0xcb, 0x5e, 0x75, 0x01, 0xdc, 0x2a, 0x96, 0x0b, 0xf5, 0x62, 0xe7, 0x57, 0x0d, 0x6a,
+ 0x17, 0x62, 0x48, 0xbe, 0x80, 0x22, 0xc7, 0x99, 0xac, 0x2c, 0xf9, 0x2e, 0xee, 0xde, 0x30, 0xf4,
+ 0x72, 0x65, 0x28, 0x21, 0xb9, 0x07, 0xb7, 0x3d, 0xca, 0x85, 0x25, 0xb3, 0xe8, 0x06, 0x76, 0x84,
+ 0x3e, 0x06, 0x02, 0x1d, 0x55, 0x75, 0x35, 0x83, 0x48, 0xcc, 0xc4, 0xd9, 0xe6, 0x39, 0xd2, 0xd8,
+ 0x81, 0x82, 0x89, 0x33, 0xf2, 0x0e, 0x2c, 0x2b, 0x4d, 0xfc, 0x92, 0x6b, 0x83, 0x9a, 0x3c, 0xf8,
+ 0xc9, 0x71, 0x6b, 0x49, 0xd2, 0x87, 0xc6, 0x12, 0xc7, 0xd9, 0xa6, 0x43, 0xde, 0x06, 0xf0, 0xa8,
+ 0x40, 0x2e, 0xac, 0x3d, 0xea, 0x5d, 0x28, 0xe5, 0x4a, 0x6c, 0x7f, 0x4e, 0xbd, 0xce, 0x97, 0xb0,
+ 0x6a, 0xba, 0x7e, 0xe8, 0xa1, 0x81, 0x3c, 0x64, 0x01, 0x47, 0xf2, 0x31, 0x2c, 0x61, 0x14, 0xb1,
+ 0x48, 0xf9, 0xbe, 0xd8, 0x9e, 0x16, 0x1b, 0x9e, 0xa4, 0x19, 0x31, 0xbb, 0xf3, 0x43, 0x1e, 0x56,
+ 0x37, 0x58, 0xc0, 0xe7, 0x3e, 0x46, 0xa6, 0x3b, 0x09, 0xa8, 0x47, 0x9e, 0x40, 0xcd, 0x89, 0xa8,
+ 0x1b, 0x58, 0x51, 0xdc, 0xc1, 0x13, 0x8f, 0xef, 0x5d, 0xed, 0x71, 0x28, 0xe9, 0x49, 0xbf, 0x37,
+ 0x56, 0x9c, 0xcc, 0x8a, 0xbc, 0x00, 0xc2, 0xe5, 0x44, 0xb0, 0x54, 0x13, 0x4b, 0x3d, 0xe6, 0x95,
+ 0xc7, 0x3b, 0xd7, 0xc5, 0xfe, 0xe2, 0x14, 0x31, 0xea, 0x7c, 0x71, 0xae, 0x6c, 0x42, 0x65, 0x4a,
+ 0x03, 0x87, 0x4f, 0xe9, 0x4b, 0x4c, 0x86, 0xcb, 0x35, 0xc9, 0x4c, 0xef, 0xf8, 0x38, 0x95, 0x18,
+ 0xe7, 0xea, 0x4f, 0x8b, 0x47, 0x3f, 0xb6, 0xb4, 0xce, 0x2a, 0xac, 0x64, 0x2f, 0xd2, 0xf9, 0x29,
+ 0x0f, 0x6b, 0x97, 0x64, 0xe4, 0x3e, 0x10, 0x3b, 0x31, 0x5a, 0xdc, 0x9e, 0xa2, 0x33, 0xf7, 0x30,
+ 0x4e, 0x68, 0xda, 0xd3, 0xd7, 0x52, 0xdc, 0x4c, 0x61, 0xf2, 0x1d, 0x34, 0x2e, 0x89, 0x2c, 0x07,
+ 0xa9, 0xe3, 0xb9, 0x01, 0x26, 0xd1, 0x68, 0xf4, 0xe2, 0x81, 0xdd, 0x4b, 0x07, 0x76, 0xef, 0xac,
+ 0x6d, 0x0d, 0x8a, 0xaf, 0x7e, 0x6f, 0x69, 0x86, 0xbe, 0xe8, 0x78, 0x98, 0x78, 0xc8, 0x4e, 0xc9,
+ 0xc2, 0xcd, 0xa6, 0xe4, 0x63, 0xb8, 0xed, 0xbb, 0x81, 0x45, 0x6d, 0x1b, 0x43, 0x81, 0x8e, 0x95,
+ 0xca, 0x8b, 0xd7, 0xca, 0x89, 0xef, 0x06, 0x0f, 0x13, 0x49, 0x62, 0xbb, 0xf3, 0x00, 0xd6, 0x2e,
+ 0x75, 0x07, 0x52, 0x82, 0xc2, 0xe3, 0xd1, 0x8b, 0x7a, 0x8e, 0x00, 0x2c, 0x8f, 0xcc, 0x8d, 0x87,
+ 0x3b, 0xa3, 0xba, 0x26, 0xbf, 0x07, 0x0f, 0xcd, 0xd1, 0x27, 0x0f, 0xea, 0xf9, 0xf5, 0x9f, 0xf3,
+ 0x50, 0x4a, 0x9c, 0x93, 0x29, 0x54, 0x8d, 0x79, 0x60, 0x1e, 0x06, 0xb6, 0xcc, 0x2f, 0xe9, 0xfe,
+ 0x7b, 0x16, 0xe3, 0x4a, 0x6d, 0x7c, 0x70, 0xed, 0x50, 0x73, 0xe6, 0x36, 0x46, 0xdb, 0xc8, 0x39,
+ 0x9d, 0x60, 0x27, 0xd7, 0xd5, 0xee, 0x69, 0xc4, 0x86, 0xca, 0x59, 0x65, 0x91, 0xff, 0x50, 0x7e,
+ 0x8d, 0x6b, 0xce, 0x74, 0xf1, 0x1d, 0x76, 0x72, 0x64, 0x02, 0xa0, 0xa6, 0xb7, 0x88, 0x90, 0xfa,
+ 0xe4, 0xe6, 0x67, 0x6c, 0xdc, 0xf8, 0xe2, 0xf1, 0x6d, 0x06, 0x1f, 0x1e, 0xfd, 0xd9, 0xcc, 0x1d,
+ 0x9d, 0x34, 0xb5, 0xd7, 0x27, 0x4d, 0xed, 0xcd, 0x49, 0x53, 0xfb, 0xe3, 0xa4, 0xa9, 0xbd, 0x3a,
+ 0x6d, 0xe6, 0x5e, 0x9f, 0x36, 0x73, 0x6f, 0x4e, 0x9b, 0xb9, 0x6f, 0xab, 0x99, 0xdf, 0xb5, 0xbf,
+ 0x03, 0x00, 0x00, 0xff, 0xff, 0xa3, 0x95, 0xa6, 0xef, 0x5c, 0x0a, 0x00, 0x00,
}
diff --git a/pkg/sql/execinfrapb/api.proto b/pkg/sql/execinfrapb/api.proto
index 689ad47fe8d1..f3a06e3b243a 100644
--- a/pkg/sql/execinfrapb/api.proto
+++ b/pkg/sql/execinfrapb/api.proto
@@ -68,6 +68,7 @@ message EvalContext {
optional string location = 4 [(gogoproto.nullable) = false];
optional string database = 5 [(gogoproto.nullable) = false];
repeated string searchPath = 6;
+ optional string temporarySchema = 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/information_schema.go b/pkg/sql/information_schema.go
index 93ef4eea6326..7a86184e8b33 100644
--- a/pkg/sql/information_schema.go
+++ b/pkg/sql/information_schema.go
@@ -1432,7 +1432,7 @@ func forEachDatabaseDesc(
fn func(*sqlbase.DatabaseDescriptor) error,
) error {
var dbDescs []*sqlbase.DatabaseDescriptor
- dbDescs, err := p.Tables().getAllDatabaseDescriptors(ctx, p.txn)
+ dbDescs, err := p.Tables().getAllDatabaseDescriptors(ctx, p.txn, p.ExecCfg().Settings)
if err != nil {
return err
}
diff --git a/pkg/sql/lease.go b/pkg/sql/lease.go
index c730fc840ec7..a4651c6c8aeb 100644
--- a/pkg/sql/lease.go
+++ b/pkg/sql/lease.go
@@ -1571,7 +1571,7 @@ 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()
+ key := sqlbase.NewPublicTableKey(dbID, tableName, m.settings).Key()
id := sqlbase.InvalidID
if err := m.db.Txn(ctx, func(ctx context.Context, txn *client.Txn) error {
txn.SetFixedTimestamp(ctx, timestamp)
diff --git a/pkg/sql/logical_schema_accessors.go b/pkg/sql/logical_schema_accessors.go
index 65d70f3a6d19..4be50cc5ba2d 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,19 +34,22 @@ 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.
func (l *LogicalSchemaAccessor) GetObjectNames(
ctx context.Context,
txn *client.Txn,
+ settings *cluster.Settings,
dbDesc *DatabaseDescriptor,
scName string,
flags tree.DatabaseListFlags,
@@ -63,12 +67,16 @@ func (l *LogicalSchemaAccessor) GetObjectNames(
}
// Fallthrough.
- return l.SchemaAccessor.GetObjectNames(ctx, txn, dbDesc, scName, flags)
+ return l.SchemaAccessor.GetObjectNames(ctx, txn, settings, dbDesc, scName, flags)
}
// 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 +98,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 6d570308251a..7dc6620b13b9 100644
--- a/pkg/sql/logictest/testdata/logic_test/crdb_internal
+++ b/pkg/sql/logictest/testdata/logic_test/crdb_internal
@@ -83,7 +83,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.
@@ -289,7 +289,7 @@ select crdb_internal.set_vmodule('')
query T
select regexp_replace(crdb_internal.node_executable_version()::string, '(-\d+)?$', '');
----
-19.1
+20.1
query ITTT colnames
select node_id, component, field, regexp_replace(regexp_replace(value, '^\d+$', ''), e':\\d+', ':') as value from crdb_internal.node_runtime_info
@@ -442,7 +442,7 @@ select * from crdb_internal.gossip_alerts
query T
select regexp_replace(crdb_internal.node_executable_version()::string, '(-\d+)?$', '');
----
-19.1
+20.1
user root
diff --git a/pkg/sql/logictest/testdata/logic_test/grant_table b/pkg/sql/logictest/testdata/logic_test/grant_table
index 53f6a554e97e..d1e48df90c67 100644
--- a/pkg/sql/logictest/testdata/logic_test/grant_table
+++ b/pkg/sql/logictest/testdata/logic_test/grant_table
@@ -161,23 +161,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
@@ -357,6 +344,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
@@ -411,6 +415,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 6007f31ed4c0..2899c375673e 100644
--- a/pkg/sql/logictest/testdata/logic_test/information_schema
+++ b/pkg/sql/logictest/testdata/logic_test/information_schema
@@ -599,7 +599,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
@@ -618,6 +618,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
@@ -679,6 +680,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
@@ -764,6 +766,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
@@ -777,43 +782,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
@@ -920,9 +928,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
@@ -1463,6 +1475,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
@@ -1691,10 +1707,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
@@ -1874,6 +1890,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 e49bd51045f1..f2d64f980e1a 100644
--- a/pkg/sql/logictest/testdata/logic_test/pg_catalog
+++ b/pkg/sql/logictest/testdata/logic_test/pg_catalog
@@ -639,6 +639,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 0 0 0 0 NULL NULL
3752917847 27 2 true true false true false true false false true false 1 2 0 0 0 0 0 0 NULL NULL
3966258450 14 1 true true false true false true false false true false 1 3903121477 0 0 NULL NULL
+4012654114 30 3 true true false true false true false false true false 1 2 3 0 0 3903121477 0 0 0 0 0 0 NULL NULL
4225994721 13 2 true true false true false true false false true false 1 7 0 0 0 0 0 0 NULL NULL
# From #26504
@@ -690,6 +691,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 518096ef93c9..8cfe4a71e657 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
diff --git a/pkg/sql/logictest/testdata/logic_test/sequences b/pkg/sql/logictest/testdata/logic_test/sequences
index 6575f948991d..a6c9146a6cf1 100644
--- a/pkg/sql/logictest/testdata/logic_test/sequences
+++ b/pkg/sql/logictest/testdata/logic_test/sequences
@@ -1,3 +1,9 @@
+# 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 .
+# The only tests affected by this are the tests involving KV Trace.
+# TODO(arul): Should those tests be put in a separate file like `system`?
+
# see also files `drop_sequence`, `alter_sequence`, `rename_sequence`
# USING THE `lastval` FUNCTION
@@ -884,6 +890,10 @@ BEGIN;
query T
SELECT message FROM [SHOW KV TRACE FOR SESSION]
----
+querying next range at /NamespaceTable/30/1/0/0/"test"/4/1
+r26: sending batch 1 Get to (n1,s1):1
+querying next range at /NamespaceTable/30/1/52/0
+r26: sending batch 1 Get to (n1,s1):1
rows affected: 1
statement ok
diff --git a/pkg/sql/logictest/testdata/logic_test/show_source b/pkg/sql/logictest/testdata/logic_test/show_source
index 6d185c1a55e9..922bd617bdf3 100644
--- a/pkg/sql/logictest/testdata/logic_test/show_source
+++ b/pkg/sql/logictest/testdata/logic_test/show_source
@@ -161,7 +161,7 @@ query T colnames,rowsort
SELECT * FROM [SHOW TABLES FROM system]
----
table_name
-namespace
+namespace_deprecated
descriptor
users
zones
@@ -180,12 +180,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 ·
@@ -204,6 +205,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..2577056d6b65
--- /dev/null
+++ b/pkg/sql/logictest/testdata/logic_test/system_namespace_deprecated
@@ -0,0 +1,40 @@
+# LogicTest: local-mixed-19.1-19.2
+# This tests the local-mixed-19.1-19.2 configuration because system.namespace
+# changed in 20.1
+
+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 8611c3637859..5017cf183b5f 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 5edb9a8b6803..a6d465e92243 100644
--- a/pkg/sql/opt/exec/execbuilder/testdata/autocommit
+++ b/pkg/sql/opt/exec/execbuilder/testdata/autocommit
@@ -33,7 +33,10 @@ 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 Get to (n1,s1):1
+dist sender send r26: sending batch 1 Get to (n1,s1):1
+dist sender send r26: sending batch 1 Get to (n1,s1):1
+dist sender send r26: sending batch 1 CPut, 1 EndTxn to (n1,s1):1
# Multi-row insert should auto-commit.
statement ok
@@ -52,7 +55,9 @@ 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 1 Get to (n1,s1):1
+dist sender send r26: sending batch 1 Get 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
@@ -74,7 +79,9 @@ 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 1 Get to (n1,s1):1
+dist sender send r26: sending batch 1 Get to (n1,s1):1
+dist sender send r26: sending batch 2 CPut to (n1,s1):1
statement ok
ROLLBACK
@@ -96,7 +103,9 @@ 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 1 Get to (n1,s1):1
+dist sender send r26: sending batch 1 Get 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.
statement ok
@@ -116,8 +125,10 @@ 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 1 Get to (n1,s1):1
+dist sender send r26: sending batch 1 Get 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.
@@ -138,8 +149,10 @@ 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 1 Get to (n1,s1):1
+dist sender send r26: sending batch 1 Get 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.
@@ -172,7 +185,9 @@ 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 Get to (n1,s1):1
+dist sender send r26: sending batch 1 Get to (n1,s1):1
+dist sender send r26: sending batch 1 Put, 1 EndTxn to (n1,s1):1
# Multi-row upsert should auto-commit.
statement ok
@@ -191,7 +206,9 @@ 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 1 Get to (n1,s1):1
+dist sender send r26: sending batch 1 Get 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
@@ -213,7 +230,9 @@ 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 1 Get to (n1,s1):1
+dist sender send r26: sending batch 1 Get to (n1,s1):1
+dist sender send r26: sending batch 2 Put to (n1,s1):1
statement ok
ROLLBACK
@@ -235,7 +254,9 @@ 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 1 Get to (n1,s1):1
+dist sender send r26: sending batch 1 Get 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.
statement ok
@@ -255,8 +276,10 @@ 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 1 Get to (n1,s1):1
+dist sender send r26: sending batch 1 Get 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.
@@ -277,8 +300,10 @@ 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 1 Get to (n1,s1):1
+dist sender send r26: sending batch 1 Get 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.
@@ -311,8 +336,10 @@ 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 Get to (n1,s1):1
+dist sender send r26: sending batch 1 Get 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
@@ -334,8 +361,10 @@ 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 Get to (n1,s1):1
+dist sender send r26: sending batch 1 Get 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
@@ -357,8 +386,10 @@ 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 Get to (n1,s1):1
+dist sender send r26: sending batch 1 Get 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.
statement ok
@@ -378,9 +409,11 @@ 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 Get to (n1,s1):1
+dist sender send r26: sending batch 1 Get 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.
@@ -401,9 +434,11 @@ 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 Get to (n1,s1):1
+dist sender send r26: sending batch 1 Get 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.
@@ -436,7 +471,9 @@ 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 Get to (n1,s1):1
+dist sender send r26: sending batch 1 Get to (n1,s1):1
+dist sender send r26: sending batch 1 DelRng, 1 EndTxn to (n1,s1):1
# Multi-row delete should auto-commit.
statement ok
@@ -455,7 +492,9 @@ 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 Get to (n1,s1):1
+dist sender send r26: sending batch 1 Get 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
@@ -477,7 +516,9 @@ 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 Get to (n1,s1):1
+dist sender send r26: sending batch 1 Get to (n1,s1):1
+dist sender send r26: sending batch 1 DelRng to (n1,s1):1
statement ok
ROLLBACK
@@ -499,8 +540,10 @@ 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 Get to (n1,s1):1
+dist sender send r26: sending batch 1 Get 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.
statement ok
@@ -520,9 +563,11 @@ 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 Get to (n1,s1):1
+dist sender send r26: sending batch 1 Get 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.
@@ -543,9 +588,11 @@ 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 Get to (n1,s1):1
+dist sender send r26: sending batch 1 Get 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);
@@ -592,9 +639,11 @@ 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 1 Get to (n1,s1):1
+dist sender send r26: sending batch 1 Get 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
statement ok
SET TRACING=ON;
@@ -613,10 +662,12 @@ 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 Get to (n1,s1):1
+dist sender send r26: sending batch 1 Get 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
statement ok
SET TRACING=ON;
@@ -635,10 +686,12 @@ 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 Get to (n1,s1):1
+dist sender send r26: sending batch 1 Get 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
@@ -663,9 +716,13 @@ 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 1 Get to (n1,s1):1
+dist sender send r26: sending batch 1 Get to (n1,s1):1
+dist sender send r26: sending batch 1 Get to (n1,s1):1
+dist sender send r26: sending batch 1 Get 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
statement ok
SET TRACING=ON;
@@ -685,6 +742,10 @@ 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 1 Get to (n1,s1):1
+dist sender send r26: sending batch 1 Get to (n1,s1):1
+dist sender send r26: sending batch 1 Get to (n1,s1):1
+dist sender send r26: sending batch 1 Get 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..5b23b86bc363 100644
--- a/pkg/sql/opt/exec/execbuilder/testdata/delete_range
+++ b/pkg/sql/opt/exec/execbuilder/testdata/delete_range
@@ -15,11 +15,14 @@ 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
+dist sender send r26: sending batch 1 Get to (n1,s1):1
+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.
@@ -34,5 +37,7 @@ 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
+dist sender send r26: sending batch 1 Get to (n1,s1):1
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 983872ae7660..572679064219 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 > 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 > 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 2f3b3fec7c08..fbc7cb6ff085 100644
--- a/pkg/sql/opt/exec/execbuilder/testdata/upsert
+++ b/pkg/sql/opt/exec/execbuilder/testdata/upsert
@@ -244,7 +244,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
@@ -258,7 +258,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
@@ -275,7 +275,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/util.go b/pkg/sql/opt/optbuilder/util.go
index b9a0926cbf12..655c91235166 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"
@@ -432,8 +433,8 @@ func (b *Builder) resolveSchemaForCreate(name *tree.TableName) (cat.Schema, cat.
panic(err)
}
- // Only allow creation of objects in the public schema.
- if resName.Schema() != tree.PublicSchema {
+ // Only allow creation of objects in the public schema or Temporary Schema.
+ if resName.Schema() != tree.PublicSchema && resName.Schema() != sessiondata.PgTempSchemaName {
panic(pgerror.Newf(pgcode.InvalidName,
"schema cannot be modified: %q", tree.ErrString(&resName)))
}
diff --git a/pkg/sql/opt_catalog.go b/pkg/sql/opt_catalog.go
index 12f8a1040a2c..5de44e5f9e1d 100644
--- a/pkg/sql/opt_catalog.go
+++ b/pkg/sql/opt_catalog.go
@@ -23,6 +23,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/stats"
"github.com/cockroachdb/cockroach/pkg/util"
@@ -104,7 +105,10 @@ 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.ExecCfg().Settings,
+ os.planner,
+ os.desc,
os.name.Schema(),
true, /* explicitPrefix */
)
@@ -128,6 +132,24 @@ func (oc *optCatalog) ResolveSchema(
// more general error.
oc.tn.TableName = ""
oc.tn.TableNamePrefix = *name
+
+ // Scoping a table under pg_temp is equivalent to creating a temporary table
+ // through explicit syntax. `pg_temp` is merely an alias, so even if the
+ // temporary schema does not exist, it should be created as part of
+ // this transaction (expected PG behavior).
+ // Instead of searching if the temporary schema exists, we search for the
+ // public schema, which is guaranteed to exist in every database.
+ // Scoping under pg_temp is the only case where a table can be scoped under a
+ // schema that does not exist. Thus this code will not need to be extended
+ // even if/when CRDB supports user defined schemas.
+ // TODO(whomever): Once it is possible to drop schemas, it will no longer be
+ // safe to search for public (as it may not exist). The above invariant will
+ // need to be replaced then.
+ isTempScoped := false
+ if oc.tn.Schema() == sessiondata.PgTempSchemaName {
+ isTempScoped = true
+ oc.tn.SchemaName = tree.PublicSchemaName
+ }
found, desc, err := oc.tn.ResolveTarget(
ctx,
oc.planner,
@@ -147,6 +169,9 @@ func (oc *optCatalog) ResolveSchema(
pgcode.InvalidSchemaName, "target database or schema does not exist",
)
}
+ if isTempScoped {
+ oc.tn.SchemaName = sessiondata.PgTempSchemaName
+ }
return &optSchema{
planner: oc.planner,
desc: desc.(*DatabaseDescriptor),
diff --git a/pkg/sql/pgwire/pgwire_test.go b/pkg/sql/pgwire/pgwire_test.go
index f8783f0962ac..cb0bfb842c43 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..c1cbc5c3dcc3 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"
@@ -45,7 +47,11 @@ var _ SchemaAccessor = UncachedPhysicalAccessor{}
// GetDatabaseDesc implements the SchemaAccessor interface.
func (a UncachedPhysicalAccessor) GetDatabaseDesc(
- ctx context.Context, txn *client.Txn, name string, flags tree.DatabaseLookupFlags,
+ ctx context.Context,
+ txn *client.Txn,
+ settings *cluster.Settings,
+ name string,
+ flags tree.DatabaseLookupFlags,
) (desc *DatabaseDescriptor, err error) {
if name == sqlbase.SystemDB.Name {
// We can't return a direct reference to SystemDB, because the
@@ -54,7 +60,7 @@ func (a UncachedPhysicalAccessor) GetDatabaseDesc(
return &sysDB, nil
}
- descID, err := getDescriptorID(ctx, txn, sqlbase.NewDatabaseKey(name))
+ descID, err := getDescriptorID(ctx, txn, sqlbase.NewDatabaseKey(name, settings))
if err != nil {
return nil, err
}
@@ -74,20 +80,35 @@ 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.
func (a UncachedPhysicalAccessor) GetObjectNames(
ctx context.Context,
txn *client.Txn,
+ settings *cluster.Settings,
dbDesc *DatabaseDescriptor,
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,7 +117,7 @@ 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, "", settings).Key()
sr, err := txn.Scan(ctx, prefix, prefix.PrefixEnd(), 0)
if err != nil {
return nil, err
@@ -119,30 +140,42 @@ 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)
+ dbID, err := getDatabaseID(ctx, txn, settings, name.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
}
+ // Try to use the system name resolution bypass. Avoids a hotspot by explicitly
+ // checking for public schema.
+ var schemaID sqlbase.ID
+ if name.Schema() == tree.PublicSchema {
+ schemaID = keys.PublicSchemaID
+ } else {
+ var ok bool
+ 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(dbID, name.Table(), settings)
if descID == sqlbase.InvalidID {
- descID, err = getDescriptorID(ctx, txn, sqlbase.NewTableKey(dbID, name.Table()))
+ descID, err = getDescriptorID(ctx, txn, sqlbase.NewTableKey(dbID, schemaID, name.Table(), settings))
if err != nil {
return nil, err
}
@@ -173,7 +206,12 @@ 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, the first check is not required.
+ if desc.Name == name.Table() ||
+ (name.Table() == sqlbase.NamespaceTable.Name && name.Catalog() == sqlbase.SystemDB.Name) {
if flags.RequireMutable {
return sqlbase.NewMutableExistingTableDescriptor(*desc), nil
}
@@ -194,7 +232,11 @@ var _ SchemaAccessor = &CachedPhysicalAccessor{}
// GetDatabaseDesc implements the SchemaAccessor interface.
func (a *CachedPhysicalAccessor) GetDatabaseDesc(
- ctx context.Context, txn *client.Txn, name string, flags tree.DatabaseLookupFlags,
+ ctx context.Context,
+ txn *client.Txn,
+ settings *cluster.Settings,
+ name string,
+ flags tree.DatabaseLookupFlags,
) (desc *DatabaseDescriptor, err error) {
isSystemDB := name == sqlbase.SystemDB.Name
if !(flags.AvoidCached || isSystemDB || testDisableTableLeases) {
@@ -215,27 +257,49 @@ 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,
+ return a.tc.databaseCache.getDatabaseDesc(ctx, settings,
a.tc.leaseMgr.db.Txn, name, flags.Required)
}
// We avoided the cache. Go lower.
- return a.SchemaAccessor.GetDatabaseDesc(ctx, txn, name, flags)
+ return a.SchemaAccessor.GetDatabaseDesc(ctx, txn, settings, name, flags)
}
// 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.
+ var obj ObjectDescriptor
+ var err error
+ 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
+ } else {
+ return obj.(*sqlbase.ImmutableTableDescriptor), err
+ }
+ }
if flags.RequireMutable {
- table, err := a.tc.getMutableTableDescriptor(ctx, txn, name, flags)
+ table, err := a.tc.getMutableTableDescriptor(ctx, txn, settings, name, flags)
if table == nil {
// return nil interface.
return nil, err
}
return table, err
}
- table, err := a.tc.getTableVersion(ctx, txn, name, flags)
+ table, err := a.tc.getTableVersion(ctx, txn, settings, name, flags)
if table == nil {
// return nil interface.
return nil, err
diff --git a/pkg/sql/planner.go b/pkg/sql/planner.go
index 4791b6ca648c..a22859cbef15 100644
--- a/pkg/sql/planner.go
+++ b/pkg/sql/planner.go
@@ -383,6 +383,16 @@ func (p *planner) User() string {
return p.SessionData().User
}
+func (p *planner) TemporarySchema() 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..9b85d293404f 100644
--- a/pkg/sql/rename_database.go
+++ b/pkg/sql/rename_database.go
@@ -77,7 +77,7 @@ func (n *renameDatabaseNode) startExec(params runParams) error {
// DDL statements bypass the cache.
lookupFlags.AvoidCached = true
tbNames, err := phyAccessor.GetObjectNames(
- ctx, p.txn, dbDesc, tree.PublicSchema, tree.DatabaseListFlags{
+ ctx, p.txn, p.ExecCfg().Settings, dbDesc, tree.PublicSchema, tree.DatabaseListFlags{
CommonLookupFlags: lookupFlags,
ExplicitPrefix: true,
})
@@ -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
}
@@ -117,7 +117,7 @@ func (n *renameDatabaseNode) startExec(params runParams) error {
}
}
- return p.renameDatabase(ctx, dbDesc, n.newName)
+ return p.renameDatabase(ctx, p.ExecCfg().Settings, dbDesc, n.newName)
}
func (n *renameDatabaseNode) Next(runParams) (bool, error) { return false, nil }
diff --git a/pkg/sql/rename_table.go b/pkg/sql/rename_table.go
index 7825f478f1a5..00a551d2504d 100644
--- a/pkg/sql/rename_table.go
+++ b/pkg/sql/rename_table.go
@@ -106,7 +106,7 @@ 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.NewPublicTableKey(targetDbDesc.ID, newTn.Table(), p.ExecCfg().Settings).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..ba3e1743c4ff 100644
--- a/pkg/sql/resolver.go
+++ b/pkg/sql/resolver.go
@@ -15,6 +15,7 @@ import (
"fmt"
"github.com/cockroachdb/cockroach/pkg/internal/client"
+ "github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql/opt/cat"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
@@ -54,7 +55,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,
+ res, err = p.LogicalSchemaAccessor().GetDatabaseDesc(ctx, p.txn, p.ExecCfg().Settings, dbName,
p.CommonLookupFlags(required))
})
return res, err
@@ -64,12 +65,13 @@ func (p *planner) ResolveUncachedDatabaseByName(
func GetObjectNames(
ctx context.Context,
txn *client.Txn,
+ settings *cluster.Settings,
sc SchemaResolver,
dbDesc *DatabaseDescriptor,
scName string,
explicitPrefix bool,
) (res TableNames, err error) {
- return sc.LogicalSchemaAccessor().GetObjectNames(ctx, txn, dbDesc, scName,
+ return sc.LogicalSchemaAccessor().GetObjectNames(ctx, txn, settings, dbDesc, scName,
tree.DatabaseListFlags{
CommonLookupFlags: sc.CommonLookupFlags(true /*required*/),
ExplicitPrefix: explicitPrefix,
@@ -267,11 +269,15 @@ func (p *planner) LookupSchema(
ctx context.Context, dbName, scName string,
) (found bool, scMeta tree.SchemaMeta, err error) {
sc := p.LogicalSchemaAccessor()
- dbDesc, err := sc.GetDatabaseDesc(ctx, p.txn, dbName, p.CommonLookupFlags(false /*required*/))
+ dbDesc, err := sc.GetDatabaseDesc(ctx, p.txn, p.ExecCfg().Settings, dbName, p.CommonLookupFlags(false /*required*/))
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 +287,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
}
@@ -373,18 +379,19 @@ func (p *planner) getQualifiedTableName(
func findTableContainingIndex(
ctx context.Context,
txn *client.Txn,
+ settings *cluster.Settings,
sc SchemaResolver,
dbName, scName string,
idxName tree.UnrestrictedName,
lookupFlags tree.CommonLookupFlags,
) (result *tree.TableName, desc *MutableTableDescriptor, err error) {
sa := sc.LogicalSchemaAccessor()
- dbDesc, err := sa.GetDatabaseDesc(ctx, txn, dbName, lookupFlags)
+ dbDesc, err := sa.GetDatabaseDesc(ctx, txn, settings, dbName, lookupFlags)
if dbDesc == nil || err != nil {
return nil, nil, err
}
- tns, err := sa.GetObjectNames(ctx, txn, dbDesc, scName,
+ tns, err := sa.GetObjectNames(ctx, txn, settings, dbDesc, scName,
tree.DatabaseListFlags{CommonLookupFlags: lookupFlags, ExplicitPrefix: true})
if err != nil {
return nil, nil, err
@@ -435,7 +442,7 @@ func expandMutableIndexName(
ctx context.Context, p *planner, index *tree.TableIndexName, requireTable bool,
) (tn *tree.TableName, desc *MutableTableDescriptor, err error) {
p.runWithOptions(resolveFlags{skipCache: true}, func() {
- tn, desc, err = expandIndexName(ctx, p.txn, p, index, requireTable)
+ tn, desc, err = expandIndexName(ctx, p.txn, p.ExecCfg().Settings, p, index, requireTable)
})
return tn, desc, err
}
@@ -443,6 +450,7 @@ func expandMutableIndexName(
func expandIndexName(
ctx context.Context,
txn *client.Txn,
+ settings *cluster.Settings,
sc SchemaResolver,
index *tree.TableIndexName,
requireTable bool,
@@ -479,7 +487,7 @@ func expandIndexName(
lookupFlags := sc.CommonLookupFlags(requireTable)
var foundTn *tree.TableName
- foundTn, desc, err = findTableContainingIndex(ctx, txn, sc, tn.Catalog(), tn.Schema(), index.Index, lookupFlags)
+ foundTn, desc, err = findTableContainingIndex(ctx, txn, settings, sc, tn.Catalog(), tn.Schema(), index.Index, lookupFlags)
if err != nil {
return nil, nil, err
}
diff --git a/pkg/sql/schema_accessors.go b/pkg/sql/schema_accessors.go
index 525e191805e1..128493441dca 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"
)
@@ -74,20 +75,20 @@ type SchemaAccessor interface {
// GetDatabaseDesc looks up a database by name and returns its
// descriptor. If the database is not found and required is true,
// an error is returned; otherwise a nil reference is returned.
- GetDatabaseDesc(ctx context.Context, txn *client.Txn, dbName string, flags tree.DatabaseLookupFlags) (*DatabaseDescriptor, error)
+ GetDatabaseDesc(ctx context.Context, txn *client.Txn, settings *cluster.Settings, 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.
// TODO(whomever): when separate schemas are supported, this
// API should be extended to use schema descriptors.
- GetObjectNames(ctx context.Context, txn *client.Txn, db *DatabaseDescriptor, scName string, flags tree.DatabaseListFlags) (TableNames, error)
+ GetObjectNames(ctx context.Context, txn *client.Txn, settings *cluster.Settings, db *DatabaseDescriptor, scName string, flags tree.DatabaseListFlags) (TableNames, error)
// GetObjectDesc looks up an object by name and returns both its
// 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..2516edc2aff9 100644
--- a/pkg/sql/schema_changer.go
+++ b/pkg/sql/schema_changer.go
@@ -835,7 +835,7 @@ func (sc *SchemaChanger) updateDropTableJob(
case jobspb.Status_ROCKSDB_COMPACTION:
runningStatus = RunningStatusCompaction
case jobspb.Status_DONE:
- return job.WithTxn(txn).Succeeded(ctx, func(ctx context.Context, txn *client.Txn) error {
+ return job.WithTxn(txn).Succeeded(ctx, func(ctx context.Context, txn *client.Txn, _ *cluster.Settings) error {
return onSuccess(ctx, txn, job)
})
default:
@@ -875,7 +875,7 @@ 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()
+ tbKey := sqlbase.NewPublicTableKey(drain.ParentID, drain.Name, sc.settings).Key()
b.Del(tbKey)
}
diff --git a/pkg/sql/scrub.go b/pkg/sql/scrub.go
index 49317d9d8217..f38aa414cba0 100644
--- a/pkg/sql/scrub.go
+++ b/pkg/sql/scrub.go
@@ -156,14 +156,14 @@ func (n *scrubNode) startScrubDatabase(ctx context.Context, p *planner, name *tr
if err != nil {
return err
}
- tbNames, err := GetObjectNames(ctx, p.txn, p, dbDesc, tree.PublicSchema, true /*explicitPrefix*/)
+ tbNames, err := GetObjectNames(ctx, p.txn, p.ExecCfg().Settings, p, dbDesc, tree.PublicSchema, true /*explicitPrefix*/)
if err != nil {
return err
}
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 4f3bb042b53c..33f1f5a85a18 100644
--- a/pkg/sql/sem/builtins/builtins.go
+++ b/pkg/sql/sem/builtins/builtins.go
@@ -2790,7 +2790,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 {
@@ -2830,7 +2830,7 @@ may increase either contention or retry errors, or both.`,
if includePgCatalog {
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..53113c84b777 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.DefaultTemporarySchema)
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..301db59529de 100644
--- a/pkg/sql/sem/tree/name_resolution.go
+++ b/pkg/sql/sem/tree/name_resolution.go
@@ -275,9 +275,13 @@ 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 := searchPath.MaybeResolveTemporarySchema(t.Schema())
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 +291,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 +331,17 @@ 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 := searchPath.MaybeResolveTemporarySchema(t.Schema())
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 +362,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 +381,18 @@ 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 := searchPath.MaybeResolveTemporarySchema(tp.Schema())
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 +412,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..23e115e647fd 100644
--- a/pkg/sql/sem/tree/name_resolution_test.go
+++ b/pkg/sql/sem/tree/name_resolution_test.go
@@ -528,7 +528,9 @@ 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.DefaultTemporarySchema)
+ }
testCases := []struct {
// Test inputs.
diff --git a/pkg/sql/sessiondata/search_path.go b/pkg/sql/sessiondata/search_path.go
index d2d6763d2e6a..f78e74f5b9e6 100644
--- a/pkg/sql/sessiondata/search_path.go
+++ b/pkg/sql/sessiondata/search_path.go
@@ -15,6 +15,13 @@ import "strings"
// PgDatabaseName is the name of the default postgres system database.
const PgDatabaseName = "postgres"
+// DefaultTemporarySchema 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 DefaultTemporarySchema = "pg_no_temp_schema"
+
// DefaultDatabaseName is the name ofthe default CockroachDB database used
// for connections without a current db set.
const DefaultDatabaseName = "defaultdb"
@@ -22,27 +29,64 @@ 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
+// MakeSearchPath returns a new 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,
+ }
+}
+
+// UpdateTemporarySchemaName sets the temporary schema name to the supplied
+// string. This should be called every time a session creates a temporary schema
+// for the first time.
+func (s *SearchPath) UpdateTemporarySchemaName(tempSchemaName string) {
+ s.TempSchemaName = tempSchemaName
+}
+
+// UpdatePaths sets the updates the paths of the SearchPath struct whithout
+// changing the session specific temporary schema.
+func (s *SearchPath) UpdatePaths(paths []string) {
+ nsp := MakeSearchPath(paths, s.TempSchemaName)
+ s.paths = nsp.paths
+ s.containsPgCatalog = nsp.containsPgCatalog
+ s.containsPgTempSchema = nsp.containsPgTempSchema
+}
+
+// MaybeResolveTemporarySchema returns the session specific temporary schema
+// if the pg_temp alias is used and one exists. It acts as a pass through for
+// all other schemas.
+func (s *SearchPath) MaybeResolveTemporarySchema(schemaName string) string {
+ // If the scName is pg_temp and the TempSchemaName has been set, pg_temp
+ // can be used as an alias for name resolution.
+ if schemaName == PgTempSchemaName && s.TempSchemaName != DefaultTemporarySchema {
+ return s.TempSchemaName
}
+ return schemaName
}
// Iter returns an iterator through the search path. We must include the
@@ -54,16 +98,21 @@ func MakeSearchPath(paths []string) SearchPath {
// will be searched before searching any of the path items."
// - 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}
+ var paths []string
+ if !s.containsPgTempSchema {
+ paths = append(paths, PgTempSchemaName)
}
- return SearchPathIter{paths: s.paths, i: -1}
+ if !s.containsPgCatalog {
+ paths = append(paths, PgCatalogName)
+ }
+
+ return SearchPathIter{paths: append(paths, s.paths...), tempSchemaName: s.TempSchemaName, i: 0}
}
-// 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 {
+ return SearchPathIter{paths: s.paths, tempSchemaName: s.TempSchemaName, i: 0}
}
// GetPathArray returns the underlying path array of this SearchPath. The
@@ -72,14 +121,23 @@ func (s SearchPath) GetPathArray() []string {
return s.paths
}
+// GetTemporarySchema returns the temporary schema specific to the current
+// session.
+func (s SearchPath) GetTemporarySchema() 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 {
+ if s.containsPgCatalog != other.containsPgCatalog || 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 {
@@ -100,19 +158,24 @@ func (s SearchPath) String() string {
// iterator, and then repeatedly call the Next method in order to iterate over
// each search path.
type SearchPathIter struct {
- paths []string
- i int
+ paths []string
+ 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++
- return PgCatalogName, true
- }
if iter.i < len(iter.paths) {
+ schemaName := iter.paths[iter.i]
iter.i++
- return iter.paths[iter.i-1], true
+ if schemaName == PgTempSchemaName && iter.tempSchemaName != DefaultTemporarySchema {
+ return iter.tempSchemaName, true
+ } else if schemaName == PgTempSchemaName && iter.tempSchemaName == DefaultTemporarySchema {
+ // If the search path contains pg_temp, but no temporary schema has been
+ // created, we can skip it.
+ return iter.Next()
+ }
+ return schemaName, true
}
return "", false
}
diff --git a/pkg/sql/sessiondata/search_path_test.go b/pkg/sql/sessiondata/search_path_test.go
index 0b2c628466db..d60bbe0138e3 100644
--- a/pkg/sql/sessiondata/search_path_test.go
+++ b/pkg/sql/sessiondata/search_path_test.go
@@ -18,62 +18,190 @@ 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`}},
+ {[]string{}, /* explicitSearchPath */
+ []string{`pg_catalog`}, /* expectedSearchPath */
+ []string{}, /* expectedSearchPathWithoutImplicitPgSchemas */
+ []string{testTempSchemaName, `pg_catalog`}, /* expectedSearchPathWhenTemporarySchemaExists */
+ []string{}, /* expectedSearchPathWithoutImplicitPgSchemasWhenTempSchemaExists */
+ },
+ {[]string{`pg_catalog`}, /* explicitSearchPath */
+ []string{`pg_catalog`}, /* expectedSearchPath */
+ []string{`pg_catalog`}, /* expectedSearchPathWithoutImplicitPgSchemas */
+ []string{testTempSchemaName, `pg_catalog`}, /* expectedSearchPathWhenTemporarySchemaExists */
+ []string{`pg_catalog`}, /* expectedSearchPathWithoutImplicitPgSchemasWhenTempSchemaExists */
+
+ },
+ {[]string{`pg_catalog`, `pg_temp`}, /* explicitSearchPath */
+ []string{`pg_catalog`}, /* expectedSearchPath */
+ []string{`pg_catalog`}, /* expectedSearchPathWithoutImplicitPgSchemas */
+ []string{`pg_catalog`, testTempSchemaName}, /* expectedSearchPathWhenTemporarySchemaExists */
+ []string{`pg_catalog`, testTempSchemaName}, /* expectedSearchPathWithoutImplicitPgSchemasWhenTempSchemaExists */
+ },
+ {[]string{`pg_temp`, `pg_catalog`}, /* explicitSearchPath */
+ []string{`pg_catalog`}, /* expectedSearchPath */
+ []string{`pg_catalog`}, /* expectedSearchPathWithoutImplicitPgSchemas */
+ []string{testTempSchemaName, `pg_catalog`}, /* expectedSearchPathWhenTemporarySchemaExists */
+ []string{testTempSchemaName, `pg_catalog`}, /* expectedSearchPathWithoutImplicitPgSchemasWhenTempSchemaExists */
+ },
+ {[]string{`foobar`, `pg_catalog`}, /* explicitSearchPath */
+ []string{`foobar`, `pg_catalog`}, /* expectedSearchPath */
+ []string{`foobar`, `pg_catalog`}, /* expectedSearchPathWithoutImplicitPgSchemas */
+ []string{testTempSchemaName, `foobar`, `pg_catalog`}, /* expectedSearchPathWhenTemporarySchemaExists */
+ []string{`foobar`, `pg_catalog`}, /* expectedSearchPathWithoutImplicitPgSchemasWhenTempSchemaExists */
+
+ },
+ {[]string{`foobar`, `pg_temp`}, /* explicitSearchPath */
+ []string{`pg_catalog`, `foobar`}, /* expectedSearchPath */
+ []string{`foobar`}, /* expectedSearchPathWithoutImplicitPgSchemas */
+ []string{`pg_catalog`, `foobar`, testTempSchemaName}, /* expectedSearchPathWhenTemporarySchemaExists */
+ []string{`foobar`, testTempSchemaName}, /* expectedSearchPathWithoutImplicitPgSchemasWhenTempSchemaExists */
+
+ },
+ {[]string{`foobar`}, /* explicitSearchPath */
+ []string{`pg_catalog`, `foobar`}, /* expectedSearchPath */
+ []string{`foobar`}, /* expectedSearchPathWithoutImplicitPgSchemas */
+ []string{testTempSchemaName, `pg_catalog`, `foobar`}, /* expectedSearchPathWhenTemporarySchemaExists */
+ []string{`foobar`}, /* expectedSearchPathWithoutImplicitPgSchemasWhenTempSchemaExists */
+ },
}
- 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, DefaultTemporarySchema)
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, DefaultTemporarySchema)
+ 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, ",")+"/temp-schema-exists", func(t *testing.T) {
+ searchPath := MakeSearchPath(tc.explicitSearchPath, testTempSchemaName)
+ 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.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-catalog", func(t *testing.T) {
- searchPath := MakeSearchPath(tc.explicitSearchPath)
+ 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.IterWithoutImplicitPGCatalog()
+ iter := searchPath.IterWithoutImplicitPGSchemas()
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.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"}, DefaultTemporarySchema)
+ a2 := MakeSearchPath([]string{"x", "y", "z"}, DefaultTemporarySchema)
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"}, DefaultTemporarySchema)
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"}, DefaultTemporarySchema)
+ c2 := MakeSearchPath([]string{"x", "y", "z", "pg_catalog"}, DefaultTemporarySchema)
assert.True(t, c1.Equals(&c2))
assert.False(t, a1.Equals(&c1))
- d := MakeSearchPath([]string{"x"})
+ d := MakeSearchPath([]string{"x"}, DefaultTemporarySchema)
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"}, DefaultTemporarySchema)
+ 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 TestUpdateSearchPath(t *testing.T) {
+ testTempSchemaName := `test_temp_schema`
+
+ sp := MakeSearchPath([]string{"x", "y", "z"}, DefaultTemporarySchema)
+ sp.UpdatePaths([]string{"x", "pg_catalog"})
+ assert.True(t, sp.GetTemporarySchema() == DefaultTemporarySchema)
+
+ sp.UpdateTemporarySchemaName(testTempSchemaName)
+ sp.UpdatePaths([]string{"pg_catalog"})
+ assert.True(t, sp.GetTemporarySchema() == testTempSchemaName)
+
+ sp.UpdatePaths([]string{"x", "pg_temp"})
+ assert.True(t, sp.GetTemporarySchema() == testTempSchemaName)
}
diff --git a/pkg/sql/set_zone_config.go b/pkg/sql/set_zone_config.go
index cc020c0ede0a..af769ffead6e 100644
--- a/pkg/sql/set_zone_config.go
+++ b/pkg/sql/set_zone_config.go
@@ -320,11 +320,13 @@ func (n *setZoneConfigNode) startExec(params runParams) error {
// resolveZone determines the ID of the target object of the zone
// specifier. This ought to succeed regardless of whether there is
// already a zone config for the target object.
- targetID, err := resolveZone(params.ctx, params.p.txn, &zs)
+ targetID, err := resolveZone(params.ctx, params.p.txn, params.p.ExecCfg().Settings, &zs)
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/show_zone_config.go b/pkg/sql/show_zone_config.go
index a3ef7eaa3f90..04aea65271e1 100644
--- a/pkg/sql/show_zone_config.go
+++ b/pkg/sql/show_zone_config.go
@@ -107,7 +107,7 @@ func getShowZoneConfigRow(
return nil, err
}
- targetID, err := resolveZone(ctx, p.txn, &zoneSpecifier)
+ targetID, err := resolveZone(ctx, p.txn, p.ExecCfg().Settings, &zoneSpecifier)
if err != nil {
return nil, err
}
diff --git a/pkg/sql/sqlbase/constants.go b/pkg/sql/sqlbase/constants.go
index be5e62296254..73dd8bc07827 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.DefaultTemporarySchema)
// 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..91645a8150e2 100644
--- a/pkg/sql/sqlbase/keys.go
+++ b/pkg/sql/sqlbase/keys.go
@@ -15,19 +15,41 @@ import (
"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/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.
+// Pass name == "" in order to generate the prefix key to use to scan over all
+// of the names for the specified parentID.
+// Pass settings == nil to construct the key for cluster versions >= 20.1 .
+func MakeNameMetadataKey(
+ parentID ID, parentSchemaID ID, name string, settings *cluster.Settings,
+) roachpb.Key {
+ if settings != nil && !settings.Version.IsActive(cluster.VersionNamespaceTableUngossip) {
+ return MakeDeprecatedNameMetadataKey(parentID, name)
+ }
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[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(NamespaceTable.Columns[2].ID))
+ 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..a3496d102fa3 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", nil /* settings */).Key()},
+ {NewPublicTableKey(1, "BAR", nil /* settings */).Key()},
+ {NewPublicTableKey(1, "foo", nil /* settings */).Key()},
+ {NewPublicTableKey(2, "foo", nil /* settings */).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..d7774d6acca1 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"
)
@@ -140,9 +141,36 @@ func (ms MetadataSchema) GetInitialValues() ([]roachpb.KeyValue, []roachpb.RKey)
// Create name metadata key.
value := roachpb.Value{}
value.SetInt(int64(desc.GetID()))
+ if parentID != keys.RootNamespaceID {
+ ret = append(ret, roachpb.KeyValue{
+ Key: NewPublicTableKey(parentID, desc.GetName(), nil /* settings */).Key(),
+ Value: value,
+ })
+ } else {
+ // Initializing the system database. The database 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(), nil /* settings */).Key(),
+ Value: value,
+ },
+ roachpb.KeyValue{
+ Key: NewPublicSchemaKey(desc.GetID()).Key(),
+ Value: publicSchemaValue,
+ })
+ }
+
+ // This function is called during bootstrapping, and the cluster settings are populated later.
+ // There is no way to ascertain what the cluster version is at this point. So, we populate both
+ // the older system.namespace (< 20.1) and the newer system.namespace (>= 20.1)
+ deprecatedValue := roachpb.Value{}
+ deprecatedValue.SetInt(int64(desc.GetID()))
ret = append(ret, roachpb.KeyValue{
- Key: NewTableKey(parentID, desc.GetName()).Key(),
- Value: value,
+ Key: NewDeprecatedTableKey(parentID, desc.GetName()).Key(),
+ Value: deprecatedValue,
})
// Create descriptor metadata key.
@@ -217,10 +245,17 @@ 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(dbID ID, tableName string, settings *cluster.Settings) ID {
if dbID != SystemDB.ID {
return InvalidID
}
+ // Pre 20.1 clusters use system.namespace to refer to
+ // system.namespace_deprecated.
+ if settings != nil &&
+ !settings.Version.IsActive(cluster.VersionNamespaceTableUngossip) &&
+ tableName == NamespaceTable.Name {
+ return DeprecatedNamespaceTable.ID
+ }
dbID, ok := systemTableIDCache[tableName]
if !ok {
return InvalidID
diff --git a/pkg/sql/sqlbase/structured.go b/pkg/sql/sqlbase/structured.go
index 62b783ba193d..0e391be6aa22 100644
--- a/pkg/sql/sqlbase/structured.go
+++ b/pkg/sql/sqlbase/structured.go
@@ -3777,17 +3777,18 @@ func (desc *ImmutableTableDescriptor) TableDesc() *TableDescriptor {
// DatabaseKey implements DescriptorKey.
type DatabaseKey struct {
- name string
+ name string
+ settings *cluster.Settings
}
// NewDatabaseKey returns a new DatabaseKey.
-func NewDatabaseKey(name string) DatabaseKey {
- return DatabaseKey{name}
+func NewDatabaseKey(name string, settings *cluster.Settings) DatabaseKey {
+ return DatabaseKey{name: name, settings: settings}
}
// Key implements DescriptorKey interface.
func (dk DatabaseKey) Key() roachpb.Key {
- return MakeNameMetadataKey(keys.RootNamespaceID, dk.name)
+ return MakeNameMetadataKey(keys.RootNamespaceID, keys.RootNamespaceID, dk.name, dk.settings)
}
// Name implements DescriptorKey interface.
@@ -3797,21 +3798,95 @@ func (dk DatabaseKey) Name() string {
// TableKey implements DescriptorKey interface.
type TableKey struct {
- parentID ID
- name string
+ parentID ID
+ parentSchemaID ID
+ name string
+ settings *cluster.Settings
+}
+
+// NewPublicTableKey returns a new TableKey.
+func NewPublicTableKey(parentID ID, name string, settings *cluster.Settings) TableKey {
+ return TableKey{parentID: parentID, parentSchemaID: keys.PublicSchemaID, name: name, settings: settings}
}
-// 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, settings *cluster.Settings) TableKey {
+ return TableKey{parentID: parentID, parentSchemaID: parentSchemaID, name: name, settings: settings}
}
// Key implements DescriptorKey interface.
func (tk TableKey) Key() roachpb.Key {
- return MakeNameMetadataKey(tk.parentID, tk.name)
+ return MakeNameMetadataKey(tk.parentID, tk.parentSchemaID, tk.name, tk.settings)
}
// 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, nil)
+}
+
+// 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/system.go b/pkg/sql/sqlbase/system.go
index dd85588ce0fd..2d52dd134ce0 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 = `
@@ -242,11 +243,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.
@@ -296,10 +298,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{
@@ -322,6 +324,38 @@ var (
ColumnIDs: []ColumnID{1, 2},
},
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},
+ },
+ NextIndexID: 2,
Privileges: NewCustomSuperuserPrivilegeDescriptor(SystemAllowedPrivileges[keys.NamespaceTableID]),
FormatVersion: InterleavedFormatVersion,
NextMutationID: 1,
@@ -1073,6 +1107,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)
@@ -1152,6 +1187,7 @@ func addSystemDatabaseToSchema(
// IsSystemConfigID returns whether this ID is for a system config object.
func IsSystemConfigID(id ID) bool {
+ // NamespaceTableID is not in the gossip range, but is still a SystemConfigID
return id > 0 && id <= keys.MaxSystemConfigDescID
}
diff --git a/pkg/sql/sqlbase/testutils.go b/pkg/sql/sqlbase/testutils.go
index 63e559f920d5..8bd3af7a4efd 100644
--- a/pkg/sql/sqlbase/testutils.go
+++ b/pkg/sql/sqlbase/testutils.go
@@ -43,7 +43,8 @@ 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)
- dKey := NewDatabaseKey(database)
+ // testutil, so we pass settings as nil for both database and table name keys.
+ dKey := NewDatabaseKey(database, nil /* settings */)
ctx := context.TODO()
gr, err := kvDB.Get(ctx, dKey.Key())
if err != nil {
@@ -54,7 +55,7 @@ func GetTableDescriptor(kvDB *client.DB, database string, table string) *TableDe
}
dbDescID := ID(gr.ValueInt())
- tKey := NewTableKey(dbDescID, table)
+ tKey := NewPublicTableKey(dbDescID, table, nil /* settings */)
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 2cb7261e2f78..53f18675dc2d 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"
@@ -150,7 +151,11 @@ type dbCacheSubscriber interface {
// return a nil descriptor and no error if the table does not exist.
//
func (tc *TableCollection) getMutableTableDescriptor(
- ctx context.Context, txn *client.Txn, tn *tree.TableName, flags tree.ObjectLookupFlags,
+ ctx context.Context,
+ txn *client.Txn,
+ settings *cluster.Settings,
+ tn *tree.TableName,
+ flags tree.ObjectLookupFlags,
) (*sqlbase.MutableTableDescriptor, error) {
if log.V(2) {
log.Infof(ctx, "reading mutable descriptor on table '%s'", tn)
@@ -171,7 +176,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,
+ dbID, err = tc.databaseCache.getDatabaseID(ctx, settings,
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.
@@ -187,7 +192,7 @@ func (tc *TableCollection) getMutableTableDescriptor(
}
phyAccessor := UncachedPhysicalAccessor{}
- obj, err := phyAccessor.GetObjectDesc(ctx, txn, tn, flags)
+ obj, err := phyAccessor.GetObjectDesc(ctx, txn, settings, tn, flags)
if obj == nil {
return nil, err
}
@@ -206,7 +211,11 @@ func (tc *TableCollection) getMutableTableDescriptor(
// the validity window of the table descriptor version returned.
//
func (tc *TableCollection) getTableVersion(
- ctx context.Context, txn *client.Txn, tn *tree.TableName, flags tree.ObjectLookupFlags,
+ ctx context.Context,
+ txn *client.Txn,
+ settings *cluster.Settings,
+ tn *tree.TableName,
+ flags tree.ObjectLookupFlags,
) (*sqlbase.ImmutableTableDescriptor, error) {
if log.V(2) {
log.Infof(ctx, "planner acquiring lease on table '%s'", tn)
@@ -227,7 +236,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,
+ dbID, err = tc.databaseCache.getDatabaseID(ctx, settings,
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.
@@ -263,7 +272,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, settings, tn, flags)
if obj == nil {
return nil, err
}
@@ -447,7 +456,9 @@ func (tc *TableCollection) releaseTables(ctx context.Context) {
// Wait until the database cache has been updated to properly
// reflect all dropped databases, so that future commands on the
// same gateway node observe the dropped databases.
-func (tc *TableCollection) waitForCacheToDropDatabases(ctx context.Context) {
+func (tc *TableCollection) waitForCacheToDropDatabases(
+ ctx context.Context, settings *cluster.Settings,
+) {
for _, uc := range tc.uncommittedDatabases {
if !uc.dropped {
continue
@@ -458,7 +469,7 @@ func (tc *TableCollection) waitForCacheToDropDatabases(ctx context.Context) {
tc.dbCacheSubscriber.waitForCacheState(
func(dc *databaseCache) bool {
// Resolve the database name from the database cache.
- dbID, err := dc.getCachedDatabaseID(uc.name)
+ dbID, err := dc.getCachedDatabaseID(uc.name, settings)
if err != nil || dbID == sqlbase.InvalidID {
// dbID can still be 0 if required is false and
// the database is not found. Swallowing error here
@@ -643,10 +654,10 @@ func (tc *TableCollection) getAllDescriptors(
// validity before scanning system.namespace and looking up the descriptors
// in the database cache, if necessary.
func (tc *TableCollection) getAllDatabaseDescriptors(
- ctx context.Context, txn *client.Txn,
+ ctx context.Context, txn *client.Txn, settings *cluster.Settings,
) ([]*sqlbase.DatabaseDescriptor, error) {
if tc.allDatabaseDescriptors == nil {
- dbDescIDs, err := GetAllDatabaseDescriptorIDs(ctx, txn)
+ dbDescIDs, err := GetAllDatabaseDescriptorIDs(ctx, txn, settings)
if err != 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..9665a6769781 100644
--- a/pkg/sql/tests/system_table_test.go
+++ b/pkg/sql/tests/system_table_test.go
@@ -12,6 +12,7 @@ package tests_test
import (
"context"
+ "fmt"
"strings"
"testing"
@@ -29,8 +30,12 @@ import (
func TestInitialKeys(t *testing.T) {
defer leaktest.AfterTest(t)()
- const keysPerDesc = 2
- const nonDescKeys = 9
+ // keysPerDesc before 20.1 was 2. This changes to 3, because during the initial
+ // value construction, we populate both the deprecated system.namespace table
+ // and the new system.namespace table. This is done because during bootstrap,
+ // cluster version isn't known.
+ const keysPerDesc = 3
+ const nonDescKeys = 10
ms := sqlbase.MakeMetadataSchema(config.DefaultZoneConfigRef(), config.DefaultSystemZoneConfigRef())
kv, _ /* splits */ := ms.GetInitialValues()
@@ -130,6 +135,8 @@ func TestSystemTableLiterals(t *testing.T) {
t.Fatalf("test: %+v, err: %v", test, err)
}
+ fmt.Println(gen)
+
if !proto.Equal(&test.pkg, &gen) {
diff := strings.Join(pretty.Diff(&test.pkg, &gen), "\n")
t.Errorf("%s table descriptor generated from CREATE TABLE statement does not match "+
diff --git a/pkg/sql/truncate.go b/pkg/sql/truncate.go
index 0f8fa49b1488..3fd1f7ed0ec1 100644
--- a/pkg/sql/truncate.go
+++ b/pkg/sql/truncate.go
@@ -208,7 +208,7 @@ 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.NewPublicTableKey(tableDesc.ParentID, tableDesc.GetName(), p.ExecCfg().Settings).Key()
b := &client.Batch{}
// Use CPut because we want to remove a specific name -> id map.
if traceKV {
@@ -267,7 +267,7 @@ func (p *planner) truncateTable(
newTableDesc.Mutations = nil
newTableDesc.GCMutations = nil
newTableDesc.ModificationTime = p.txn.CommitTimestamp()
- key := sqlbase.NewTableKey(newTableDesc.ParentID, newTableDesc.Name).Key()
+ key := sqlbase.NewPublicTableKey(newTableDesc.ParentID, newTableDesc.Name, p.ExecCfg().Settings).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 1d34421fd97e..c0d3e22251b8 100644
--- a/pkg/sql/vars.go
+++ b/pkg/sql/vars.go
@@ -184,8 +184,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, evalCtx.Settings,
+ dbName, tree.DatabaseLookupFlags{Required: true}); err != nil {
return "", err
}
}
@@ -615,7 +615,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 6a37b2e0878d..007e7e65f3e9 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, p.ExecCfg().Settings,
+ 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..48808f59205b 100644
--- a/pkg/sql/zone_config.go
+++ b/pkg/sql/zone_config.go
@@ -18,6 +18,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/internal/client"
"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/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
@@ -258,12 +259,19 @@ func (p *planner) resolveTableForZone(
// specifier points to a table, index or partition, the table part
// must be properly normalized already. It is the caller's
// responsibility to do this using e.g .resolveTableForZone().
-func resolveZone(ctx context.Context, txn *client.Txn, zs *tree.ZoneSpecifier) (sqlbase.ID, error) {
+func resolveZone(
+ ctx context.Context, txn *client.Txn, settings *cluster.Settings, zs *tree.ZoneSpecifier,
+) (sqlbase.ID, error) {
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())
+ var key sqlbase.DescriptorKey
+ if parentID == keys.RootNamespaceID {
+ key = sqlbase.NewDatabaseKey(name, settings)
+ } else {
+ key = sqlbase.NewPublicTableKey(sqlbase.ID(parentID), name, settings)
+ }
+ kv, err := txn.Get(ctx, key.Key())
if err != nil {
return 0, err
}
diff --git a/pkg/sqlmigrations/migrations.go b/pkg/sqlmigrations/migrations.go
index 2796b3712a9b..20eb29059afd 100644
--- a/pkg/sqlmigrations/migrations.go
+++ b/pkg/sqlmigrations/migrations.go
@@ -260,7 +260,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
}
@@ -577,7 +580,10 @@ 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)
+ // No access to the cluster version here, so we populate both the newer
+ // and older (deprecated) system.namespaces table.
+ b.CPut(sqlbase.NewPublicTableKey(desc.GetParentID(), desc.GetName(), nil /* settings */).Key(), desc.GetID(), nil)
+ b.CPut(sqlbase.NewDeprecatedTableKey(desc.GetParentID(), desc.GetName()).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 7a40900278fc..96f6bb4f5c22 100644
--- a/pkg/sqlmigrations/migrations_test.go
+++ b/pkg/sqlmigrations/migrations_test.go
@@ -462,7 +462,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, nil /* settings */).Key()
descKey := sqlbase.MakeDescMetadataKey(table.ID)
descVal := sqlbase.WrapDescriptor(&table)
diff --git a/pkg/storage/client_replica_test.go b/pkg/storage/client_replica_test.go
index e56a5bb5346d..91d45001af83 100644
--- a/pkg/storage/client_replica_test.go
+++ b/pkg/storage/client_replica_test.go
@@ -1706,6 +1706,7 @@ func TestSystemZoneConfigs(t *testing.T) {
log.Info(ctx, "TestSystemZoneConfig: test cluster started")
expectedSystemRanges, err := tc.Servers[0].ExpectedInitialRangeCount()
+ log.Infof(ctx, "The number of system ranges is %d\n", expectedSystemRanges)
if err != nil {
t.Fatal(err)
}
@@ -1714,8 +1715,6 @@ func TestSystemZoneConfigs(t *testing.T) {
systemNumReplicas := int(*config.DefaultSystemZoneConfig().NumReplicas)
userNumReplicas := int(*config.DefaultZoneConfig().NumReplicas)
expectedReplicas := expectedSystemRanges*systemNumReplicas + expectedUserRanges*userNumReplicas
- log.Infof(ctx, "TestSystemZoneConfig: expecting %d system ranges and %d user ranges",
- expectedSystemRanges, expectedUserRanges)
log.Infof(ctx, "TestSystemZoneConfig: expected (%dx%d) + (%dx%d) = %d replicas total",
expectedSystemRanges, systemNumReplicas, expectedUserRanges, userNumReplicas, expectedReplicas)