Skip to content

Commit 3648118

Browse files
committed
catalog/bootstrap: add option to offset ids during system table creation
This patch adds an option to MakeSetadataSchema to offset the dynamically allocated system table ids. So, if the caller sets this to say 1000, the vc will still have ids 0-49, but then ids 1050,1051, etc. A future patch will leverage this option during PCR reader tenant creation to ensure that replicating user table ids never collide with the reader tenant's system table ids. Informs #152909 Release note: none
1 parent 2d6b423 commit 3648118

File tree

15 files changed

+65
-30
lines changed

15 files changed

+65
-30
lines changed

pkg/config/system_test.go

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -203,7 +203,7 @@ func TestGetLargestID(t *testing.T) {
203203

204204
// Real SQL layout.
205205
func() testCase {
206-
ms := bootstrap.MakeMetadataSchema(keys.SystemSQLCodec, zonepb.DefaultZoneConfigRef(), zonepb.DefaultSystemZoneConfigRef())
206+
ms := bootstrap.MakeMetadataSchema(keys.SystemSQLCodec, zonepb.DefaultZoneConfigRef(), zonepb.DefaultSystemZoneConfigRef(), bootstrap.NoOffset)
207207
descIDs := ms.DescriptorIDs()
208208
maxDescID := config.ObjectID(descIDs[len(descIDs)-1])
209209
kvs, _ /* splits */ := ms.GetInitialValues()
@@ -321,7 +321,7 @@ func TestComputeSplitKeySystemRanges(t *testing.T) {
321321

322322
cfg := config.NewSystemConfig(zonepb.DefaultZoneConfigRef())
323323
kvs, _ /* splits */ := bootstrap.MakeMetadataSchema(
324-
keys.SystemSQLCodec, cfg.DefaultZoneConfig, zonepb.DefaultSystemZoneConfigRef(),
324+
keys.SystemSQLCodec, cfg.DefaultZoneConfig, zonepb.DefaultSystemZoneConfigRef(), bootstrap.NoOffset,
325325
).GetInitialValues()
326326
cfg.SystemConfigEntries = config.SystemConfigEntries{
327327
Values: kvs,
@@ -353,7 +353,7 @@ func TestComputeSplitKeyTableIDs(t *testing.T) {
353353
minKey := roachpb.RKey(keys.TimeseriesPrefix.PrefixEnd())
354354

355355
schema := bootstrap.MakeMetadataSchema(
356-
keys.SystemSQLCodec, zonepb.DefaultZoneConfigRef(), zonepb.DefaultSystemZoneConfigRef(),
356+
keys.SystemSQLCodec, zonepb.DefaultZoneConfigRef(), zonepb.DefaultSystemZoneConfigRef(), bootstrap.NoOffset,
357357
)
358358
// Real system tables only.
359359
baseSql, _ /* splits */ := schema.GetInitialValues()
@@ -460,7 +460,7 @@ func TestComputeSplitKeyTenantBoundaries(t *testing.T) {
460460
minTenID, maxTenID := roachpb.MinTenantID.ToUint64(), roachpb.MaxTenantID.ToUint64()
461461

462462
schema := bootstrap.MakeMetadataSchema(
463-
keys.SystemSQLCodec, zonepb.DefaultZoneConfigRef(), zonepb.DefaultSystemZoneConfigRef(),
463+
keys.SystemSQLCodec, zonepb.DefaultZoneConfigRef(), zonepb.DefaultSystemZoneConfigRef(), bootstrap.NoOffset,
464464
)
465465
minKey := tkey(bootstrap.TestingUserDescID(0))
466466

@@ -599,7 +599,7 @@ func TestGetZoneConfigForKey(t *testing.T) {
599599
cfg := config.NewSystemConfig(zonepb.DefaultZoneConfigRef())
600600

601601
kvs, _ /* splits */ := bootstrap.MakeMetadataSchema(
602-
keys.SystemSQLCodec, cfg.DefaultZoneConfig, zonepb.DefaultSystemZoneConfigRef(),
602+
keys.SystemSQLCodec, cfg.DefaultZoneConfig, zonepb.DefaultSystemZoneConfigRef(), bootstrap.NoOffset,
603603
).GetInitialValues()
604604
cfg.SystemConfigEntries = config.SystemConfigEntries{
605605
Values: kvs,

pkg/crosscluster/physical/stream_ingestion_planning.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -331,7 +331,7 @@ func createReaderTenant(
331331
}
332332

333333
readerInfo.ID = readerID.ToUint64()
334-
_, err = sql.BootstrapTenant(ctx, p.ExecCfg(), p.Txn(), readerInfo, readerZcfg)
334+
_, err = sql.BootstrapTenant(ctx, p.ExecCfg(), p.Txn(), readerInfo, readerZcfg, 0)
335335
if err != nil {
336336
return readerID, err
337337
}

pkg/kv/kvserver/store_test.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -126,7 +126,7 @@ type testStoreOpts struct {
126126

127127
func (opts *testStoreOpts) splits() (_kvs []roachpb.KeyValue, _splits []roachpb.RKey) {
128128
kvs, splits := bootstrap.MakeMetadataSchema(
129-
keys.SystemSQLCodec, zonepb.DefaultZoneConfigRef(), zonepb.DefaultSystemZoneConfigRef(),
129+
keys.SystemSQLCodec, zonepb.DefaultZoneConfigRef(), zonepb.DefaultSystemZoneConfigRef(), bootstrap.NoOffset,
130130
).GetInitialValues()
131131
if !opts.createSystemRanges {
132132
return kvs, nil

pkg/server/node.go

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -444,10 +444,10 @@ func allocateStoreIDs(
444444

445445
// GetBootstrapSchema returns the schema which will be used to bootstrap a new
446446
// server.
447-
func GetBootstrapSchema(
447+
func GetBootstrapSchemaForTest(
448448
defaultZoneConfig *zonepb.ZoneConfig, defaultSystemZoneConfig *zonepb.ZoneConfig,
449449
) bootstrap.MetadataSchema {
450-
return bootstrap.MakeMetadataSchema(keys.SystemSQLCodec, defaultZoneConfig, defaultSystemZoneConfig)
450+
return bootstrap.MakeMetadataSchema(keys.SystemSQLCodec, defaultZoneConfig, defaultSystemZoneConfig, bootstrap.NoOffset)
451451
}
452452

453453
// bootstrapCluster initializes the passed-in engines for a new cluster.

pkg/server/node_test.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -104,7 +104,7 @@ func TestBootstrapCluster(t *testing.T) {
104104
}
105105

106106
// Add the initial keys for sql.
107-
kvs, tableSplits := GetBootstrapSchema(
107+
kvs, tableSplits := GetBootstrapSchemaForTest(
108108
zonepb.DefaultZoneConfigRef(), zonepb.DefaultSystemZoneConfigRef(),
109109
).GetInitialValues()
110110
for _, kv := range kvs {

pkg/server/testserver.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1882,7 +1882,7 @@ func ExpectedInitialRangeCount(
18821882
defaultZoneConfig *zonepb.ZoneConfig,
18831883
defaultSystemZoneConfig *zonepb.ZoneConfig,
18841884
) (int, error) {
1885-
_, splits := bootstrap.MakeMetadataSchema(codec, defaultZoneConfig, defaultSystemZoneConfig).GetInitialValues()
1885+
_, splits := bootstrap.MakeMetadataSchema(codec, defaultZoneConfig, defaultSystemZoneConfig, bootstrap.NoOffset).GetInitialValues()
18861886
// N splits means N+1 ranges.
18871887
return len(config.StaticSplits()) + len(splits) + 1, nil
18881888
}

pkg/sql/catalog/bootstrap/BUILD.bazel

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -62,6 +62,7 @@ go_test(
6262
"//pkg/security/securityassets",
6363
"//pkg/security/securitytest",
6464
"//pkg/server",
65+
"//pkg/sql/catalog/descpb",
6566
"//pkg/testutils",
6667
"//pkg/testutils/datapathutils",
6768
"//pkg/testutils/serverutils",

pkg/sql/catalog/bootstrap/bootstrap_test.go

Lines changed: 24 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -12,6 +12,7 @@ import (
1212
"github.com/cockroachdb/cockroach/pkg/config/zonepb"
1313
"github.com/cockroachdb/cockroach/pkg/keys"
1414
"github.com/cockroachdb/cockroach/pkg/roachpb"
15+
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
1516
"github.com/cockroachdb/cockroach/pkg/testutils/datapathutils"
1617
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
1718
"github.com/cockroachdb/cockroach/pkg/util/log"
@@ -132,5 +133,27 @@ func makeMetadataSchema(tenantID uint64) MetadataSchema {
132133
if tenantID > 0 {
133134
codec = keys.MakeSQLCodec(roachpb.MustMakeTenantID(tenantID))
134135
}
135-
return MakeMetadataSchema(codec, zonepb.DefaultZoneConfigRef(), zonepb.DefaultSystemZoneConfigRef())
136+
return MakeMetadataSchema(codec, zonepb.DefaultZoneConfigRef(), zonepb.DefaultSystemZoneConfigRef(), NoOffset)
137+
}
138+
139+
func TestDynamicSystemTableIDOffset(t *testing.T) {
140+
defer leaktest.AfterTest(t)()
141+
defer log.Scope(t).Close(t)
142+
143+
offset := uint32(1000)
144+
145+
defaultMetadata := MakeMetadataSchema(keys.SystemSQLCodec, zonepb.DefaultZoneConfigRef(), zonepb.DefaultSystemZoneConfigRef(), NoOffset)
146+
offsetMetadata := MakeMetadataSchema(keys.SystemSQLCodec, zonepb.DefaultZoneConfigRef(), zonepb.DefaultSystemZoneConfigRef(), offset)
147+
148+
require.Len(t, defaultMetadata.descs, len(offsetMetadata.descs))
149+
150+
for i := range defaultMetadata.descs {
151+
defaultID := defaultMetadata.descs[i].GetID()
152+
if defaultID <= keys.MaxReservedDescID {
153+
// Reserved IDs are not offset.
154+
require.Equal(t, defaultID, offsetMetadata.descs[i].GetID())
155+
} else {
156+
require.Equal(t, defaultMetadata.descs[i].GetID()+descpb.ID(offset), offsetMetadata.descs[i].GetID())
157+
}
158+
}
136159
}

pkg/sql/catalog/bootstrap/initial_values.go

Lines changed: 6 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -23,10 +23,11 @@ import (
2323
// InitialValuesOpts is used to get initial values for system/secondary tenants
2424
// and allows overriding initial values with ones from previous releases.
2525
type InitialValuesOpts struct {
26-
DefaultZoneConfig *zonepb.ZoneConfig
27-
DefaultSystemZoneConfig *zonepb.ZoneConfig
28-
OverrideKey clusterversion.Key
29-
Codec keys.SQLCodec
26+
DefaultZoneConfig *zonepb.ZoneConfig
27+
DefaultSystemZoneConfig *zonepb.ZoneConfig
28+
OverrideKey clusterversion.Key
29+
Codec keys.SQLCodec
30+
DynamicSystemTableIDOffset uint32
3031
}
3132

3233
// GenerateInitialValues generates the initial values with which to bootstrap a
@@ -84,7 +85,7 @@ var initialValuesFactoryByKey = map[clusterversion.Key]initialValuesFactoryFn{
8485
func buildLatestInitialValues(
8586
opts InitialValuesOpts,
8687
) (kvs []roachpb.KeyValue, splits []roachpb.RKey, _ error) {
87-
schema := MakeMetadataSchema(opts.Codec, opts.DefaultZoneConfig, opts.DefaultSystemZoneConfig)
88+
schema := MakeMetadataSchema(opts.Codec, opts.DefaultZoneConfig, opts.DefaultSystemZoneConfig, opts.DynamicSystemTableIDOffset)
8889
kvs, splits = schema.GetInitialValues()
8990
return kvs, splits, nil
9091
}

pkg/sql/catalog/bootstrap/metadata.go

Lines changed: 11 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -43,16 +43,23 @@ type MetadataSchema struct {
4343
otherSplitIDs []uint32
4444
otherKV []roachpb.KeyValue
4545
ids catalog.DescriptorIDSet
46+
47+
// dynamicSystemTableIDOffset offsets the dynamically allocated IDs. So, if
48+
// this is set to 500, the ids will be 501, 502, etc.
49+
dynamicSystemTableIDOffset uint32
4650
}
4751

52+
const NoOffset = 0
53+
4854
// MakeMetadataSchema constructs a new MetadataSchema value which constructs
4955
// the "system" database.
5056
func MakeMetadataSchema(
5157
codec keys.SQLCodec,
5258
defaultZoneConfig *zonepb.ZoneConfig,
5359
defaultSystemZoneConfig *zonepb.ZoneConfig,
60+
dynamicSystemTableIDOffset uint32,
5461
) MetadataSchema {
55-
ms := MetadataSchema{codec: codec}
62+
ms := MetadataSchema{codec: codec, dynamicSystemTableIDOffset: dynamicSystemTableIDOffset}
5663
addSystemDatabaseToSchema(&ms, defaultZoneConfig, defaultSystemZoneConfig)
5764
return ms
5865
}
@@ -347,7 +354,7 @@ func (ms MetadataSchema) FirstNonSystemDescriptorID() descpb.ID {
347354
}
348355

349356
func (ms MetadataSchema) allocateID() (nextID descpb.ID) {
350-
maxID := descpb.ID(keys.MaxReservedDescID)
357+
maxID := descpb.ID(keys.MaxReservedDescID) + descpb.ID(ms.dynamicSystemTableIDOffset)
351358
for _, d := range ms.descs {
352359
if d.GetID() > maxID {
353360
maxID = d.GetID()
@@ -620,7 +627,7 @@ func addSystemTenantEntry(target *MetadataSchema) {
620627
}
621628

622629
func testingMinUserDescID(codec keys.SQLCodec) uint32 {
623-
ms := MakeMetadataSchema(codec, zonepb.DefaultZoneConfigRef(), zonepb.DefaultSystemZoneConfigRef())
630+
ms := MakeMetadataSchema(codec, zonepb.DefaultZoneConfigRef(), zonepb.DefaultSystemZoneConfigRef(), NoOffset)
624631
return uint32(ms.FirstNonSystemDescriptorID())
625632
}
626633

@@ -656,7 +663,7 @@ func GetAndHashInitialValuesToString(tenantID uint64) (initialValues string, has
656663
if tenantID > 0 {
657664
codec = keys.MakeSQLCodec(roachpb.MustMakeTenantID(tenantID))
658665
}
659-
ms := MakeMetadataSchema(codec, zonepb.DefaultZoneConfigRef(), zonepb.DefaultSystemZoneConfigRef())
666+
ms := MakeMetadataSchema(codec, zonepb.DefaultZoneConfigRef(), zonepb.DefaultSystemZoneConfigRef(), NoOffset)
660667

661668
initialValues = InitialValuesToString(ms)
662669
h := sha256.Sum256([]byte(initialValues))

0 commit comments

Comments
 (0)