Skip to content

Commit

Permalink
catalog: refactor system schema, streamline descs.Collection access
Browse files Browse the repository at this point in the history
This commit refactors how the system schema and the privileges for these
tables are defined:
- system table names are all hard-coded in `catconstants`;
- system table descriptor definitions are much more concise;
- system table privilege definitions as well as privilege validation
  and repair logic are moved from `descpb` to `catprivilege`;
- system table privileges are defined by name instead of by ID.

This refactor made it possible to clean up the descriptors collection
collection logic somewhat:
1. uncommitted descriptors are moved out of kvDescriptors and into their
   own layer,
2. all tables can now be leased except for those in a small deny-list,
3. kvDescriptors read code paths (by name and by ID) are more unified,
4. system database namespace lookups in kvDescriptors.getByName go
   through a cache,
5. descs.Collection read code paths are more unified as well.
6. descriptor validation at transaction commit time leverages the
   descs.Collection as a catalog.BatchDescGetter.

Notably, as alluded in (2), instead of the existing allow-list of
leasable system descriptors we now have `UnleasableSystemDescriptors`
defined in `systemschema`, a deny-list comprised of:
- the system database (1),
- the descriptor table (3),
- the lease table (11),
- the rangelog table (13),
- the namespace table (30).

All these changes contribute to reducing the number of round-trips to
the storage layer.

Release note: None
  • Loading branch information
Marius Posta committed Aug 19, 2021
1 parent b16c86d commit a321822
Show file tree
Hide file tree
Showing 69 changed files with 3,409 additions and 3,002 deletions.
1 change: 1 addition & 0 deletions pkg/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -185,6 +185,7 @@ ALL_TESTS = [
"//pkg/sql/catalog/catalogkeys:catalogkeys_test",
"//pkg/sql/catalog/catalogkv:catalogkv_test",
"//pkg/sql/catalog/catformat:catformat_test",
"//pkg/sql/catalog/catprivilege:catprivilege_test",
"//pkg/sql/catalog/colinfo:colinfo_test",
"//pkg/sql/catalog/dbdesc:dbdesc_test",
"//pkg/sql/catalog/descpb:descpb_test",
Expand Down
116 changes: 58 additions & 58 deletions pkg/bench/rttanalysis/testdata/benchmark_expectations
Original file line number Diff line number Diff line change
@@ -1,57 +1,57 @@
exp,benchmark
19,AlterRole/alter_role_with_1_option
20,AlterRole/alter_role_with_2_options
25,AlterRole/alter_role_with_3_options
17,AlterTableAddCheckConstraint/alter_table_add_1_check_constraint
17,AlterTableAddCheckConstraint/alter_table_add_2_check_constraints
17,AlterTableAddCheckConstraint/alter_table_add_3_check_constraints
17,AlterTableAddColumn/alter_table_add_1_column
17,AlterTableAddColumn/alter_table_add_2_columns
17,AlterTableAddColumn/alter_table_add_3_columns
22,AlterTableAddForeignKey/alter_table_add_1_foreign_key
27,AlterTableAddForeignKey/alter_table_add_2_foreign_keys
32,AlterTableAddForeignKey/alter_table_add_3_foreign_keys
22,AlterTableAddForeignKey/alter_table_add_foreign_key_with_3_columns
21,AlterTableConfigureZone/alter_table_configure_zone_5_replicas
21,AlterTableConfigureZone/alter_table_configure_zone_7_replicas_
21,AlterTableConfigureZone/alter_table_configure_zone_ranges
21-22,AlterTableDropColumn/alter_table_drop_1_column
25,AlterTableDropColumn/alter_table_drop_2_columns
29,AlterTableDropColumn/alter_table_drop_3_columns
18,AlterTableDropConstraint/alter_table_drop_1_check_constraint
19,AlterTableDropConstraint/alter_table_drop_2_check_constraints
20,AlterTableDropConstraint/alter_table_drop_3_check_constraints
15,AlterRole/alter_role_with_1_option
16,AlterRole/alter_role_with_2_options
21,AlterRole/alter_role_with_3_options
15,AlterTableAddCheckConstraint/alter_table_add_1_check_constraint
15,AlterTableAddCheckConstraint/alter_table_add_2_check_constraints
15,AlterTableAddCheckConstraint/alter_table_add_3_check_constraints
15,AlterTableAddColumn/alter_table_add_1_column
15,AlterTableAddColumn/alter_table_add_2_columns
15,AlterTableAddColumn/alter_table_add_3_columns
20,AlterTableAddForeignKey/alter_table_add_1_foreign_key
25,AlterTableAddForeignKey/alter_table_add_2_foreign_keys
30,AlterTableAddForeignKey/alter_table_add_3_foreign_keys
20,AlterTableAddForeignKey/alter_table_add_foreign_key_with_3_columns
20,AlterTableConfigureZone/alter_table_configure_zone_5_replicas
20,AlterTableConfigureZone/alter_table_configure_zone_7_replicas_
20,AlterTableConfigureZone/alter_table_configure_zone_ranges
18,AlterTableDropColumn/alter_table_drop_1_column
20,AlterTableDropColumn/alter_table_drop_2_columns
22,AlterTableDropColumn/alter_table_drop_3_columns
16,AlterTableDropConstraint/alter_table_drop_1_check_constraint
17,AlterTableDropConstraint/alter_table_drop_2_check_constraints
18,AlterTableDropConstraint/alter_table_drop_3_check_constraints
12-13,AlterTableSplit/alter_table_split_at_1_value
18-19,AlterTableSplit/alter_table_split_at_2_values
24-25,AlterTableSplit/alter_table_split_at_3_values
8,AlterTableUnsplit/alter_table_unsplit_at_1_value
10,AlterTableUnsplit/alter_table_unsplit_at_2_values
12,AlterTableUnsplit/alter_table_unsplit_at_3_values
21,CreateRole/create_role_with_1_option
23,CreateRole/create_role_with_2_options
24,CreateRole/create_role_with_3_options
22,CreateRole/create_role_with_no_options
22,DropDatabase/drop_database_0_tables
31,DropDatabase/drop_database_1_table
40,DropDatabase/drop_database_2_tables
49,DropDatabase/drop_database_3_tables
29,DropRole/drop_1_role
36,DropRole/drop_2_roles
43,DropRole/drop_3_roles
19,DropSequence/drop_1_sequence
26,DropSequence/drop_2_sequences
33,DropSequence/drop_3_sequences
22,DropTable/drop_1_table
32,DropTable/drop_2_tables
42,DropTable/drop_3_tables
22,DropView/drop_1_view
30,DropView/drop_2_views
38,DropView/drop_3_views
18,Grant/grant_all_on_1_table
20,Grant/grant_all_on_2_tables
22,Grant/grant_all_on_3_tables
21,GrantRole/grant_1_role
24,GrantRole/grant_2_roles
17,CreateRole/create_role_with_1_option
19,CreateRole/create_role_with_2_options
20,CreateRole/create_role_with_3_options
18,CreateRole/create_role_with_no_options
19,DropDatabase/drop_database_0_tables
26,DropDatabase/drop_database_1_table
33,DropDatabase/drop_database_2_tables
40,DropDatabase/drop_database_3_tables
25,DropRole/drop_1_role
32,DropRole/drop_2_roles
39,DropRole/drop_3_roles
17,DropSequence/drop_1_sequence
24,DropSequence/drop_2_sequences
31,DropSequence/drop_3_sequences
19,DropTable/drop_1_table
27,DropTable/drop_2_tables
35,DropTable/drop_3_tables
20,DropView/drop_1_view
28,DropView/drop_2_views
36,DropView/drop_3_views
16,Grant/grant_all_on_1_table
18,Grant/grant_all_on_2_tables
20,Grant/grant_all_on_3_tables
17,GrantRole/grant_1_role
20,GrantRole/grant_2_roles
2,ORMQueries/activerecord_type_introspection_query
2,ORMQueries/django_table_introspection_1_table
2,ORMQueries/django_table_introspection_4_tables
Expand All @@ -64,19 +64,19 @@ exp,benchmark
2,ORMQueries/pg_class
2,ORMQueries/pg_namespace
2,ORMQueries/pg_type
18,Revoke/revoke_all_on_1_table
20,Revoke/revoke_all_on_2_tables
22,Revoke/revoke_all_on_3_tables
20,RevokeRole/revoke_1_role
22,RevokeRole/revoke_2_roles
16,Revoke/revoke_all_on_1_table
18,Revoke/revoke_all_on_2_tables
20,Revoke/revoke_all_on_3_tables
16,RevokeRole/revoke_1_role
18,RevokeRole/revoke_2_roles
1,SystemDatabaseQueries/select_system.users_with_empty_database_Name
1,SystemDatabaseQueries/select_system.users_with_schema_Name
2,SystemDatabaseQueries/select_system.users_without_schema_Name
26,Truncate/truncate_1_column_0_rows
26,Truncate/truncate_1_column_1_row
26,Truncate/truncate_1_column_2_rows
26,Truncate/truncate_2_column_0_rows
26,Truncate/truncate_2_column_1_rows
26,Truncate/truncate_2_column_2_rows
23,Truncate/truncate_1_column_0_rows
23,Truncate/truncate_1_column_1_row
23,Truncate/truncate_1_column_2_rows
23,Truncate/truncate_2_column_0_rows
23,Truncate/truncate_2_column_1_rows
23,Truncate/truncate_2_column_2_rows
1,VirtualTableQueries/select_crdb_internal.invalid_objects_with_1_fk
1,VirtualTableQueries/select_crdb_internal.tables_with_1_fk
1 change: 1 addition & 0 deletions pkg/ccl/backupccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -59,6 +59,7 @@ go_library(
"//pkg/sql/catalog/catalogkeys",
"//pkg/sql/catalog/catalogkv",
"//pkg/sql/catalog/catconstants",
"//pkg/sql/catalog/catprivilege",
"//pkg/sql/catalog/colinfo",
"//pkg/sql/catalog/dbdesc",
"//pkg/sql/catalog/descpb",
Expand Down
3 changes: 2 additions & 1 deletion pkg/ccl/backupccl/backup_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -58,6 +58,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc"
"github.com/cockroachdb/cockroach/pkg/sql/execinfra"
"github.com/cockroachdb/cockroach/pkg/sql/randgen"
Expand Down Expand Up @@ -6698,7 +6699,7 @@ func TestRestoreErrorPropagates(t *testing.T) {
defer dirCleanupFn()
params := base.TestClusterArgs{}
params.ServerArgs.ExternalIODir = dir
jobsTableKey := keys.SystemSQLCodec.TablePrefix(keys.JobsTableID)
jobsTableKey := keys.SystemSQLCodec.TablePrefix(uint32(systemschema.JobsTable.GetID()))
var shouldFail, failures int64
params.ServerArgs.Knobs.Store = &kvserver.StoreTestingKnobs{
TestingRequestFilter: func(ctx context.Context, ba roachpb.BatchRequest) *roachpb.Error {
Expand Down
5 changes: 3 additions & 2 deletions pkg/ccl/backupccl/restore_job.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catprivilege"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/dbdesc"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descs"
Expand Down Expand Up @@ -328,7 +329,7 @@ func WriteDescriptors(
}
}
privilegeDesc := desc.GetPrivileges()
descpb.MaybeFixUsagePrivForTablesAndDBs(&privilegeDesc)
catprivilege.MaybeFixUsagePrivForTablesAndDBs(&privilegeDesc)
wroteDBs[desc.GetID()] = desc
if err := descsCol.WriteDescToBatch(
ctx, false /* kvTrace */, desc.(catalog.MutableDescriptor), b,
Expand Down Expand Up @@ -376,7 +377,7 @@ func WriteDescriptors(
}
}
privilegeDesc := table.GetPrivileges()
descpb.MaybeFixUsagePrivForTablesAndDBs(&privilegeDesc)
catprivilege.MaybeFixUsagePrivForTablesAndDBs(&privilegeDesc)
// If the table descriptor is being written to a multi-region database and
// the table does not have a locality config setup, set one up here. The
// table's locality config will be set to the default locality - REGIONAL
Expand Down
36 changes: 18 additions & 18 deletions pkg/ccl/benchccl/rttanalysisccl/testdata/benchmark_expectations
Original file line number Diff line number Diff line change
@@ -1,19 +1,19 @@
exp,benchmark
25,AlterPrimaryRegion/alter_empty_database_alter_primary_region
25,AlterPrimaryRegion/alter_empty_database_set_initial_primary_region
25,AlterPrimaryRegion/alter_populated_database_alter_primary_region
26,AlterPrimaryRegion/alter_populated_database_set_initial_primary_region
22,AlterRegions/alter_empty_database_add_region
23,AlterRegions/alter_empty_database_drop_region
22,AlterRegions/alter_populated_database_add_region
23,AlterRegions/alter_populated_database_drop_region
25,AlterSurvivalGoals/alter_empty_database_from_region_to_zone
25,AlterSurvivalGoals/alter_empty_database_from_zone_to_region
65,AlterSurvivalGoals/alter_populated_database_from_region_to_zone
65,AlterSurvivalGoals/alter_populated_database_from_zone_to_region
24,AlterTableLocality/alter_from_global_to_rbr
28,AlterTableLocality/alter_from_global_to_regional_by_table
22,AlterTableLocality/alter_from_rbr_to_global
22,AlterTableLocality/alter_from_rbr_to_regional_by_table
28,AlterTableLocality/alter_from_regional_by_table_to_global
24,AlterTableLocality/alter_from_regional_by_table_to_rbr
21,AlterPrimaryRegion/alter_empty_database_alter_primary_region
22,AlterPrimaryRegion/alter_empty_database_set_initial_primary_region
21,AlterPrimaryRegion/alter_populated_database_alter_primary_region
23,AlterPrimaryRegion/alter_populated_database_set_initial_primary_region
20,AlterRegions/alter_empty_database_add_region
21,AlterRegions/alter_empty_database_drop_region
20,AlterRegions/alter_populated_database_add_region
21,AlterRegions/alter_populated_database_drop_region
21,AlterSurvivalGoals/alter_empty_database_from_region_to_zone
21,AlterSurvivalGoals/alter_empty_database_from_zone_to_region
41,AlterSurvivalGoals/alter_populated_database_from_region_to_zone
41,AlterSurvivalGoals/alter_populated_database_from_zone_to_region
22,AlterTableLocality/alter_from_global_to_rbr
24,AlterTableLocality/alter_from_global_to_regional_by_table
20,AlterTableLocality/alter_from_rbr_to_global
20,AlterTableLocality/alter_from_rbr_to_regional_by_table
24,AlterTableLocality/alter_from_regional_by_table_to_global
22,AlterTableLocality/alter_from_regional_by_table_to_rbr
1 change: 1 addition & 0 deletions pkg/config/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -43,6 +43,7 @@ go_test(
"//pkg/sql/catalog/bootstrap",
"//pkg/sql/catalog/catalogkeys",
"//pkg/sql/catalog/descpb",
"//pkg/sql/catalog/systemschema",
"//pkg/sql/catalog/tabledesc",
"//pkg/testutils",
"//pkg/util/encoding",
Expand Down
14 changes: 8 additions & 6 deletions pkg/config/system_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/catalog/bootstrap"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/util/encoding"
Expand Down Expand Up @@ -65,7 +66,7 @@ func sqlKV(tableID uint32, indexID, descID uint64) roachpb.KeyValue {
return kv(k, nil)
}

func descriptor(descID uint64) roachpb.KeyValue {
func descriptor(descID uint32) roachpb.KeyValue {
id := descpb.ID(descID)
k := catalogkeys.MakeDescMetadataKey(keys.SystemSQLCodec, id)
v := tabledesc.NewBuilder(&descpb.TableDescriptor{ID: id}).BuildImmutable()
Expand Down Expand Up @@ -350,7 +351,6 @@ func TestComputeSplitKeyTableIDs(t *testing.T) {
defer leaktest.AfterTest(t)()

const (
start = keys.MinUserDescID
reservedStart = keys.MaxSystemConfigDescID + 1
)

Expand All @@ -367,13 +367,14 @@ func TestComputeSplitKeyTableIDs(t *testing.T) {
baseSql, _ /* splits */ := schema.GetInitialValues()
// Real system tables plus some user stuff.
kvs, _ /* splits */ := schema.GetInitialValues()
start := uint32(keys.MinUserDescID)
userSQL := append(kvs, descriptor(start), descriptor(start+1), descriptor(start+5))
// Real system tables and partitioned user tables.
var subzoneSQL = make([]roachpb.KeyValue, len(userSQL))
copy(subzoneSQL, userSQL)
subzoneSQL = append(subzoneSQL,
zoneConfig(start+1, subzone("a", ""), subzone("c", "e")),
zoneConfig(start+5, subzone("b", ""), subzone("c", "d"), subzone("d", "")))
zoneConfig(config.SystemTenantObjectID(start+1), subzone("a", ""), subzone("c", "e")),
zoneConfig(config.SystemTenantObjectID(start+5), subzone("b", ""), subzone("c", "d"), subzone("d", "")))

sort.Sort(roachpb.KeyValueByKey(userSQL))
sort.Sort(roachpb.KeyValueByKey(subzoneSQL))
Expand Down Expand Up @@ -463,12 +464,13 @@ func TestComputeSplitKeyTenantBoundaries(t *testing.T) {
// in the secondary tenant keyspace rather than within the system ranges and
// system config span that come earlier in the keyspace. Those splits are
// tested separately above.
minKey := tkey(keys.MinUserDescID)
minTenID, maxTenID := roachpb.MinTenantID.ToUint64(), roachpb.MaxTenantID.ToUint64()

schema := bootstrap.MakeMetadataSchema(
keys.SystemSQLCodec, zonepb.DefaultZoneConfigRef(), zonepb.DefaultSystemZoneConfigRef(),
)
minKey := tkey(keys.MinUserDescID)

// Real system tenant only.
baseSql, _ /* splits */ := schema.GetInitialValues()
// Real system tenant plus some secondary tenants.
Expand Down Expand Up @@ -572,7 +574,7 @@ func TestGetZoneConfigForKey(t *testing.T) {

// Non-gossiped system tables should refer to themselves.
{tkey(keys.LeaseTableID), keys.LeaseTableID},
{tkey(keys.JobsTableID), keys.JobsTableID},
{tkey(uint32(systemschema.JobsTable.GetID())), config.SystemTenantObjectID(systemschema.JobsTable.GetID())},
{tkey(keys.LocationsTableID), keys.LocationsTableID},
{tkey(keys.NamespaceTableID), keys.NamespaceTableID},

Expand Down
1 change: 1 addition & 0 deletions pkg/kv/kvserver/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -345,6 +345,7 @@ go_test(
"//pkg/sql/catalog/catalogkeys",
"//pkg/sql/catalog/dbdesc",
"//pkg/sql/catalog/descpb",
"//pkg/sql/catalog/systemschema",
"//pkg/sql/catalog/tabledesc",
"//pkg/sql/rowenc",
"//pkg/sql/sem/tree",
Expand Down
3 changes: 2 additions & 1 deletion pkg/kv/kvserver/client_split_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/catalog/bootstrap"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc"
"github.com/cockroachdb/cockroach/pkg/storage"
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
Expand Down Expand Up @@ -2563,7 +2564,7 @@ func TestUnsplittableRange(t *testing.T) {
require.NoError(t, err)

// Add a single large row to /Table/14.
tableKey := roachpb.RKey(keys.SystemSQLCodec.TablePrefix(keys.UITableID))
tableKey := roachpb.RKey(keys.SystemSQLCodec.TablePrefix(uint32(systemschema.UITable.GetID())))
row1Key := roachpb.Key(encoding.EncodeVarintAscending(append([]byte(nil), tableKey...), 1))
col1Key := keys.MakeFamilyKey(append([]byte(nil), row1Key...), 0)
valueLen := 0.9 * maxBytes
Expand Down
1 change: 1 addition & 0 deletions pkg/kv/kvserver/protectedts/ptcache/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,7 @@ go_test(
"//pkg/security",
"//pkg/security/securitytest",
"//pkg/server",
"//pkg/sql/catalog/systemschema",
"//pkg/sql/sqlutil",
"//pkg/testutils",
"//pkg/testutils/serverutils",
Expand Down
5 changes: 3 additions & 2 deletions pkg/kv/kvserver/protectedts/ptcache/cache_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts/ptpb"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts/ptstorage"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema"
"github.com/cockroachdb/cockroach/pkg/sql/sqlutil"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
Expand Down Expand Up @@ -446,8 +447,8 @@ func protect(
}

var (
metaTableSpan = tableSpan(keys.ProtectedTimestampsMetaTableID)
recordsTableSpan = tableSpan(keys.ProtectedTimestampsRecordsTableID)
metaTableSpan = tableSpan(uint32(systemschema.ProtectedTimestampsMetaTable.GetID()))
recordsTableSpan = tableSpan(uint32(systemschema.ProtectedTimestampsRecordsTable.GetID()))
)

type scanTracker struct {
Expand Down
1 change: 1 addition & 0 deletions pkg/migration/migrations/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -92,6 +92,7 @@ go_test(
"//pkg/sql/catalog/descs",
"//pkg/sql/catalog/systemschema",
"//pkg/sql/catalog/tabledesc",
"//pkg/sql/privilege",
"//pkg/sql/sem/tree",
"//pkg/sql/sqlutil",
"//pkg/sql/types",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -47,16 +47,21 @@ func TestDeleteDeprecatedNamespaceDescriptorMigration(t *testing.T) {
})
defer tc.Stopper().Stop(ctx)

// Inject deprecated namespace table descriptor and namespace entries.
// Inject deprecated namespace table descriptors and namespace entries.
err := tc.Servers[0].DB().Txn(ctx, func(ctx context.Context, txn *kv.Txn) error {
codec := keys.SystemSQLCodec
deprecated := *systemschema.NamespaceTable.TableDesc()
deprecated.ID = keys.DeprecatedNamespaceTableID
descProto := &descpb.Descriptor{Union: &descpb.Descriptor_Table{Table: &deprecated}}
deprecatedDescProto := &descpb.Descriptor{Union: &descpb.Descriptor_Table{Table: &deprecated}}
namespace2 := *systemschema.NamespaceTable.TableDesc()
namespace2.ID = keys.NamespaceTableID
namespace2.Name = `namespace2`
ns2DescProto := &descpb.Descriptor{Union: &descpb.Descriptor_Table{Table: &namespace2}}
b := txn.NewBatch()
b.Put(catalogkeys.MakeDescMetadataKey(codec, keys.DeprecatedNamespaceTableID), descProto)
b.Put(catalogkeys.MakeDescMetadataKey(codec, keys.DeprecatedNamespaceTableID), deprecatedDescProto)
namespaceKey := catalogkeys.MakePublicObjectNameKey(codec, keys.SystemDatabaseID, `namespace`)
b.Put(namespaceKey, keys.DeprecatedNamespaceTableID)
b.Put(catalogkeys.MakeDescMetadataKey(codec, keys.NamespaceTableID), ns2DescProto)
namespace2Key := catalogkeys.MakePublicObjectNameKey(codec, keys.SystemDatabaseID, `namespace2`)
b.Put(namespace2Key, keys.NamespaceTableID)
return txn.Run(ctx, b)
Expand Down
Loading

0 comments on commit a321822

Please sign in to comment.