diff --git a/pkg/cmd/roachtest/tests/cluster_to_cluster.go b/pkg/cmd/roachtest/tests/cluster_to_cluster.go index c9b65940d4eb..a8defac46536 100644 --- a/pkg/cmd/roachtest/tests/cluster_to_cluster.go +++ b/pkg/cmd/roachtest/tests/cluster_to_cluster.go @@ -575,7 +575,7 @@ func (rd *replicationDriver) setupC2C( destSQL := sqlutils.MakeSQLRunner(destDB) srcClusterSettings(t, srcSQL) - destClusterSettings(t, destSQL, rd.rs.additionalDuration) + destClusterSettings(t, destSQL, rd.rng, rd.rs.additionalDuration) overrideSrcAndDestTenantTTL(t, srcSQL, destSQL, rd.rs.overrideTenantTTL) @@ -1869,7 +1869,9 @@ func srcClusterSettings(t test.Test, db *sqlutils.SQLRunner) { ) } -func destClusterSettings(t test.Test, db *sqlutils.SQLRunner, additionalDuration time.Duration) { +func destClusterSettings( + t test.Test, db *sqlutils.SQLRunner, rng *rand.Rand, additionalDuration time.Duration, +) { db.ExecMultiple(t, `SET CLUSTER SETTING kv.rangefeed.enabled = true;`, `SET CLUSTER SETTING kv.lease.reject_on_leader_unknown.enabled = true;`, @@ -1881,6 +1883,10 @@ func destClusterSettings(t test.Test, db *sqlutils.SQLRunner, additionalDuration db.Exec(t, fmt.Sprintf(`SET CLUSTER SETTING stream_replication.replan_flow_frequency = '%s'`, replanFrequency)) } + + if rng.Intn(2) == 0 { + db.Exec(t, `SET CLUSTER SETTING physical_cluster_replication.reader_system_table_id_offset = 100000`) + } } func overrideSrcAndDestTenantTTL( diff --git a/pkg/config/system_test.go b/pkg/config/system_test.go index 892b8d1b99f0..f0953ee3e44c 100644 --- a/pkg/config/system_test.go +++ b/pkg/config/system_test.go @@ -203,7 +203,7 @@ func TestGetLargestID(t *testing.T) { // Real SQL layout. func() testCase { - ms := bootstrap.MakeMetadataSchema(keys.SystemSQLCodec, zonepb.DefaultZoneConfigRef(), zonepb.DefaultSystemZoneConfigRef()) + ms := bootstrap.MakeMetadataSchema(keys.SystemSQLCodec, zonepb.DefaultZoneConfigRef(), zonepb.DefaultSystemZoneConfigRef(), bootstrap.NoOffset) descIDs := ms.DescriptorIDs() maxDescID := config.ObjectID(descIDs[len(descIDs)-1]) kvs, _ /* splits */ := ms.GetInitialValues() @@ -321,7 +321,7 @@ func TestComputeSplitKeySystemRanges(t *testing.T) { cfg := config.NewSystemConfig(zonepb.DefaultZoneConfigRef()) kvs, _ /* splits */ := bootstrap.MakeMetadataSchema( - keys.SystemSQLCodec, cfg.DefaultZoneConfig, zonepb.DefaultSystemZoneConfigRef(), + keys.SystemSQLCodec, cfg.DefaultZoneConfig, zonepb.DefaultSystemZoneConfigRef(), bootstrap.NoOffset, ).GetInitialValues() cfg.SystemConfigEntries = config.SystemConfigEntries{ Values: kvs, @@ -353,7 +353,7 @@ func TestComputeSplitKeyTableIDs(t *testing.T) { minKey := roachpb.RKey(keys.TimeseriesPrefix.PrefixEnd()) schema := bootstrap.MakeMetadataSchema( - keys.SystemSQLCodec, zonepb.DefaultZoneConfigRef(), zonepb.DefaultSystemZoneConfigRef(), + keys.SystemSQLCodec, zonepb.DefaultZoneConfigRef(), zonepb.DefaultSystemZoneConfigRef(), bootstrap.NoOffset, ) // Real system tables only. baseSql, _ /* splits */ := schema.GetInitialValues() @@ -460,7 +460,7 @@ func TestComputeSplitKeyTenantBoundaries(t *testing.T) { minTenID, maxTenID := roachpb.MinTenantID.ToUint64(), roachpb.MaxTenantID.ToUint64() schema := bootstrap.MakeMetadataSchema( - keys.SystemSQLCodec, zonepb.DefaultZoneConfigRef(), zonepb.DefaultSystemZoneConfigRef(), + keys.SystemSQLCodec, zonepb.DefaultZoneConfigRef(), zonepb.DefaultSystemZoneConfigRef(), bootstrap.NoOffset, ) minKey := tkey(bootstrap.TestingUserDescID(0)) @@ -599,7 +599,7 @@ func TestGetZoneConfigForKey(t *testing.T) { cfg := config.NewSystemConfig(zonepb.DefaultZoneConfigRef()) kvs, _ /* splits */ := bootstrap.MakeMetadataSchema( - keys.SystemSQLCodec, cfg.DefaultZoneConfig, zonepb.DefaultSystemZoneConfigRef(), + keys.SystemSQLCodec, cfg.DefaultZoneConfig, zonepb.DefaultSystemZoneConfigRef(), bootstrap.NoOffset, ).GetInitialValues() cfg.SystemConfigEntries = config.SystemConfigEntries{ Values: kvs, diff --git a/pkg/crosscluster/physical/standby_read_ts_poller_job_test.go b/pkg/crosscluster/physical/standby_read_ts_poller_job_test.go index 3a723b086a3d..d2daf85b50b2 100644 --- a/pkg/crosscluster/physical/standby_read_ts_poller_job_test.go +++ b/pkg/crosscluster/physical/standby_read_ts_poller_job_test.go @@ -37,6 +37,11 @@ func TestStandbyReadTSPollerJob(t *testing.T) { c, cleanup := replicationtestutils.CreateTenantStreamingClusters(ctx, t, args) defer cleanup() + c.SrcTenantSQL.Exec(t, `CREATE TABLE foo (i INT PRIMARY KEY)`) + c.SrcTenantSQL.Exec(t, `CREATE TABLE bar (i INT PRIMARY KEY)`) + + offset, offsetChecksInReaderTenant := maybeOffsetReaderTenantSystemTables(t, c) + producerJobID, ingestionJobID := c.StartStreamReplication(ctx) jobutils.WaitForJobToRun(c.T, c.SrcSysSQL, jobspb.JobID(producerJobID)) @@ -68,6 +73,11 @@ INSERT INTO a VALUES (1); waitForPollerJobToStartDest(t, c, ingestionJobID) observeValueInReaderTenant(t, c.ReaderTenantSQL) + var idWithOffsetCount int + c.ReaderTenantSQL.QueryRow(t, fmt.Sprintf("SELECT count(*) FROM system.namespace where id = %d", 50+offset)).Scan(&idWithOffsetCount) + require.Equal(t, 1, idWithOffsetCount, "expected to find namespace entry for table a with offset applied") + offsetChecksInReaderTenant(c.ReaderTenantSQL) + // Failback and setup stanby reader tenant on the og source. { c.Cutover(ctx, producerJobID, ingestionJobID, srcTime.GoTime(), false) @@ -102,7 +112,111 @@ INSERT INTO a VALUES (1); var numTables int srcReaderSQL.QueryRow(t, `SELECT count(*) FROM [SHOW TABLES]`).Scan(&numTables) observeValueInReaderTenant(t, srcReaderSQL) + offsetChecksInReaderTenant(srcReaderSQL) + } +} + +func maybeOffsetReaderTenantSystemTables( + t *testing.T, c *replicationtestutils.TenantStreamingClusters, +) (int, func(sql *sqlutils.SQLRunner)) { + if c.Rng.Intn(2) == 0 { + return 0, func(sql *sqlutils.SQLRunner) {} + } + offset := 100000 + c.DestSysSQL.Exec(t, fmt.Sprintf(`SET CLUSTER SETTING physical_cluster_replication.reader_system_table_id_offset = %d`, offset)) + // Set on source to ensure failback works well too. + c.SrcSysSQL.Exec(t, fmt.Sprintf(`SET CLUSTER SETTING physical_cluster_replication.reader_system_table_id_offset = %d`, offset)) + + // swap a system table ID and a user table ID to simulate a cluster that has interleaving user and system table ids. + scaryTableIDRemapFunc := ` + CREATE OR REPLACE FUNCTION renumber_desc(oldID INT, newID INT) RETURNS BOOL AS +$$ +BEGIN +-- Rewrite the ID within the descriptor +SELECT crdb_internal.unsafe_upsert_descriptor( + newid, + crdb_internal.json_to_pb( + 'cockroach.sql.sqlbase.Descriptor', + d + ), + true + ) + FROM ( + SELECT id, + json_set( + json_set( + crdb_internal.pb_to_json( + 'cockroach.sql.sqlbase.Descriptor', + descriptor, + false + ), + ARRAY['table', 'id'], + newid::STRING::JSONB + ), + ARRAY['table', 'modificationTime'], + json_build_object( + 'wallTime', + ( + ( + extract('epoch', now()) + * 1000000 + )::INT8 + * 1000 + )::STRING + ) + ) AS d + FROM system.descriptor + WHERE id IN (oldid,) + ); +-- Update the namespace entry and delete the old descriptor. + SELECT crdb_internal.unsafe_upsert_namespace_entry("parentID", "parentSchemaID", name, newID, true) FROM (SELECT "parentID", "parentSchemaID", name, id FROM system.namespace where id =oldID) UNION ALL + SELECT crdb_internal.unsafe_delete_descriptor(oldID, true); + + RETURN true; + +END +$$ LANGUAGE PLpgSQL;` + + c.SrcTenantSQL.Exec(t, scaryTableIDRemapFunc) + var txnInsightsID, privilegesID int + c.SrcTenantSQL.QueryRow(t, `SELECT id FROM system.namespace WHERE name = 'transaction_execution_insights'`).Scan(&txnInsightsID) + c.SrcTenantSQL.QueryRow(t, `SELECT id FROM system.namespace WHERE name = 'privileges'`).Scan(&privilegesID) + require.NotEqual(t, 0, txnInsightsID) + require.NotEqual(t, 0, privilegesID) + + // renumber these two priv tables to be out of the way + txnInsightIDRemapedID := txnInsightsID + 1000 + privilegesIDRemapedID := privilegesID + 1000 + c.SrcTenantSQL.Exec(t, `SELECT renumber_desc($1, $2)`, txnInsightsID, txnInsightIDRemapedID) + c.SrcTenantSQL.Exec(t, `SELECT renumber_desc($1, $2)`, privilegesID, privilegesIDRemapedID) + + // create two user tables on the source and interleave them with system table ids + var fooID, barID int + c.SrcTenantSQL.QueryRow(t, `SELECT id FROM system.namespace WHERE name = 'foo'`).Scan(&fooID) + c.SrcTenantSQL.QueryRow(t, `SELECT id FROM system.namespace WHERE name = 'bar'`).Scan(&barID) + require.NotEqual(t, 0, fooID) + require.NotEqual(t, 0, barID) + + c.SrcTenantSQL.Exec(t, `SELECT renumber_desc($1, $2)`, fooID, txnInsightsID) + c.SrcTenantSQL.Exec(t, `SELECT renumber_desc($1, $2)`, barID, privilegesID) + + // Drop the function, to avoid hitting 152978 + c.SrcTenantSQL.Exec(t, `DROP FUNCTION renumber_desc`) + + offsetChecksInReaderTenant := func(sql *sqlutils.SQLRunner) { + // Check that txn execution insights table is not at the same id as source as it's not replicated. + sql.QueryRow(t, `SELECT id FROM system.namespace WHERE name = 'transaction_execution_insights'`).Scan(&txnInsightsID) + require.NotEqual(t, txnInsightIDRemapedID, txnInsightsID) + + // On 25.3, the privs table is not replicated so the ids should differ. + sql.QueryRow(t, `SELECT id FROM system.namespace WHERE name = 'privileges'`).Scan(&privilegesID) + require.NotEqual(t, privilegesIDRemapedID, privilegesID) + var count int + sql.QueryRow(t, `SELECT count(*) FROM system.namespace WHERE name = 'privileges'`).Scan(&count) + require.Equal(t, 1, count) } + + return offset, offsetChecksInReaderTenant } func observeValueInReaderTenant(t *testing.T, readerSQL *sqlutils.SQLRunner) { @@ -113,8 +227,8 @@ func observeValueInReaderTenant(t *testing.T, readerSQL *sqlutils.SQLRunner) { var numTables int readerSQL.QueryRow(t, `SELECT count(*) FROM [SHOW TABLES]`).Scan(&numTables) - if numTables != 1 { - return errors.Errorf("expected 1 table to be present in reader tenant, but got %d instead", numTables) + if numTables != 3 { + return errors.Errorf("expected 3 tables to be present in reader tenant, but got %d instead", numTables) } var actualQueryResult int @@ -175,6 +289,9 @@ func TestReaderTenantCutover(t *testing.T) { c, cleanup := replicationtestutils.CreateTenantStreamingClusters(ctx, t, args) defer cleanup() + c.SrcTenantSQL.Exec(t, `CREATE TABLE foo (i INT PRIMARY KEY)`) + c.SrcTenantSQL.Exec(t, `CREATE TABLE bar (i INT PRIMARY KEY)`) + producerJobID, ingestionJobID := c.StartStreamReplication(ctx) jobutils.WaitForJobToRun(c.T, c.SrcSysSQL, jobspb.JobID(producerJobID)) diff --git a/pkg/crosscluster/physical/stream_ingestion_planning.go b/pkg/crosscluster/physical/stream_ingestion_planning.go index 2edd5db949fe..718f8443158b 100644 --- a/pkg/crosscluster/physical/stream_ingestion_planning.go +++ b/pkg/crosscluster/physical/stream_ingestion_planning.go @@ -7,6 +7,7 @@ package physical import ( "context" + "math" "github.com/cockroachdb/cockroach/pkg/ccl/utilccl" "github.com/cockroachdb/cockroach/pkg/crosscluster/streamclient" @@ -17,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/repstream/streampb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server/telemetry" + "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" "github.com/cockroachdb/cockroach/pkg/sql/exprutil" @@ -34,6 +36,16 @@ import ( // replicated data will be retained. const defaultRetentionTTLSeconds = int32(4 * 60 * 60) +var readerTenantSystemTableIDOffset = settings.RegisterIntSetting( + settings.ApplicationLevel, + "physical_cluster_replication.reader_system_table_id_offset", + "the offset added to dynamically allocated system table IDs in the reader tenant", + 0, + // Max offset is 1000 less than MaxUint32 to leave room 1000 dynamically + // allocated system table ids. Hope that never happens. + settings.NonNegativeIntWithMaximum(math.MaxUint32-1000), +) + func streamIngestionJobDescription( p sql.PlanHookState, source streamclient.ConfigUri, @@ -331,7 +343,8 @@ func createReaderTenant( } readerInfo.ID = readerID.ToUint64() - _, err = sql.BootstrapTenant(ctx, p.ExecCfg(), p.Txn(), readerInfo, readerZcfg) + systemTableIDOffset := readerTenantSystemTableIDOffset.Get(&p.ExecCfg().Settings.SV) + _, err = sql.BootstrapTenant(ctx, p.ExecCfg(), p.Txn(), readerInfo, readerZcfg, uint32(systemTableIDOffset)) if err != nil { return readerID, err } diff --git a/pkg/kv/kvserver/store_test.go b/pkg/kv/kvserver/store_test.go index 78236bf01c84..9783a10aa574 100644 --- a/pkg/kv/kvserver/store_test.go +++ b/pkg/kv/kvserver/store_test.go @@ -125,7 +125,7 @@ type testStoreOpts struct { func (opts *testStoreOpts) splits() (_kvs []roachpb.KeyValue, _splits []roachpb.RKey) { kvs, splits := bootstrap.MakeMetadataSchema( - keys.SystemSQLCodec, zonepb.DefaultZoneConfigRef(), zonepb.DefaultSystemZoneConfigRef(), + keys.SystemSQLCodec, zonepb.DefaultZoneConfigRef(), zonepb.DefaultSystemZoneConfigRef(), bootstrap.NoOffset, ).GetInitialValues() if !opts.createSystemRanges { return kvs, nil diff --git a/pkg/server/node.go b/pkg/server/node.go index d605760f3ab1..6e60b5a8cc6f 100644 --- a/pkg/server/node.go +++ b/pkg/server/node.go @@ -452,10 +452,10 @@ func allocateStoreIDs( // GetBootstrapSchema returns the schema which will be used to bootstrap a new // server. -func GetBootstrapSchema( +func GetBootstrapSchemaForTest( defaultZoneConfig *zonepb.ZoneConfig, defaultSystemZoneConfig *zonepb.ZoneConfig, ) bootstrap.MetadataSchema { - return bootstrap.MakeMetadataSchema(keys.SystemSQLCodec, defaultZoneConfig, defaultSystemZoneConfig) + return bootstrap.MakeMetadataSchema(keys.SystemSQLCodec, defaultZoneConfig, defaultSystemZoneConfig, bootstrap.NoOffset) } // bootstrapCluster initializes the passed-in engines for a new cluster. diff --git a/pkg/server/node_test.go b/pkg/server/node_test.go index cf1b9ae9405d..aabd63c90848 100644 --- a/pkg/server/node_test.go +++ b/pkg/server/node_test.go @@ -105,7 +105,7 @@ func TestBootstrapCluster(t *testing.T) { } // Add the initial keys for sql. - kvs, tableSplits := GetBootstrapSchema( + kvs, tableSplits := GetBootstrapSchemaForTest( zonepb.DefaultZoneConfigRef(), zonepb.DefaultSystemZoneConfigRef(), ).GetInitialValues() for _, kv := range kvs { diff --git a/pkg/server/testserver.go b/pkg/server/testserver.go index 514bfe7f9f94..a9760038326b 100644 --- a/pkg/server/testserver.go +++ b/pkg/server/testserver.go @@ -1886,7 +1886,7 @@ func ExpectedInitialRangeCount( defaultZoneConfig *zonepb.ZoneConfig, defaultSystemZoneConfig *zonepb.ZoneConfig, ) (int, error) { - _, splits := bootstrap.MakeMetadataSchema(codec, defaultZoneConfig, defaultSystemZoneConfig).GetInitialValues() + _, splits := bootstrap.MakeMetadataSchema(codec, defaultZoneConfig, defaultSystemZoneConfig, bootstrap.NoOffset).GetInitialValues() // N splits means N+1 ranges. return len(config.StaticSplits()) + len(splits) + 1, nil } diff --git a/pkg/sql/catalog/bootstrap/BUILD.bazel b/pkg/sql/catalog/bootstrap/BUILD.bazel index a3409ba18097..3c7a703a7b96 100644 --- a/pkg/sql/catalog/bootstrap/BUILD.bazel +++ b/pkg/sql/catalog/bootstrap/BUILD.bazel @@ -58,6 +58,7 @@ go_test( "//pkg/security/securityassets", "//pkg/security/securitytest", "//pkg/server", + "//pkg/sql/catalog/descpb", "//pkg/testutils", "//pkg/testutils/datapathutils", "//pkg/testutils/serverutils", diff --git a/pkg/sql/catalog/bootstrap/bootstrap_test.go b/pkg/sql/catalog/bootstrap/bootstrap_test.go index ebdeae1e72da..55dd2088c783 100644 --- a/pkg/sql/catalog/bootstrap/bootstrap_test.go +++ b/pkg/sql/catalog/bootstrap/bootstrap_test.go @@ -12,6 +12,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/testutils/datapathutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -132,5 +133,27 @@ func makeMetadataSchema(tenantID uint64) MetadataSchema { if tenantID > 0 { codec = keys.MakeSQLCodec(roachpb.MustMakeTenantID(tenantID)) } - return MakeMetadataSchema(codec, zonepb.DefaultZoneConfigRef(), zonepb.DefaultSystemZoneConfigRef()) + return MakeMetadataSchema(codec, zonepb.DefaultZoneConfigRef(), zonepb.DefaultSystemZoneConfigRef(), NoOffset) +} + +func TestDynamicSystemTableIDOffset(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + offset := uint32(1000) + + defaultMetadata := MakeMetadataSchema(keys.SystemSQLCodec, zonepb.DefaultZoneConfigRef(), zonepb.DefaultSystemZoneConfigRef(), NoOffset) + offsetMetadata := MakeMetadataSchema(keys.SystemSQLCodec, zonepb.DefaultZoneConfigRef(), zonepb.DefaultSystemZoneConfigRef(), offset) + + require.Len(t, defaultMetadata.descs, len(offsetMetadata.descs)) + + for i := range defaultMetadata.descs { + defaultID := defaultMetadata.descs[i].GetID() + if defaultID <= keys.MaxReservedDescID { + // Reserved IDs are not offset. + require.Equal(t, defaultID, offsetMetadata.descs[i].GetID()) + } else { + require.Equal(t, defaultMetadata.descs[i].GetID()+descpb.ID(offset), offsetMetadata.descs[i].GetID()) + } + } } diff --git a/pkg/sql/catalog/bootstrap/initial_values.go b/pkg/sql/catalog/bootstrap/initial_values.go index 5391dfb9d978..1f5be1b89be4 100644 --- a/pkg/sql/catalog/bootstrap/initial_values.go +++ b/pkg/sql/catalog/bootstrap/initial_values.go @@ -23,10 +23,11 @@ import ( // InitialValuesOpts is used to get initial values for system/secondary tenants // and allows overriding initial values with ones from previous releases. type InitialValuesOpts struct { - DefaultZoneConfig *zonepb.ZoneConfig - DefaultSystemZoneConfig *zonepb.ZoneConfig - OverrideKey clusterversion.Key - Codec keys.SQLCodec + DefaultZoneConfig *zonepb.ZoneConfig + DefaultSystemZoneConfig *zonepb.ZoneConfig + OverrideKey clusterversion.Key + Codec keys.SQLCodec + DynamicSystemTableIDOffset uint32 } // GenerateInitialValues generates the initial values with which to bootstrap a @@ -77,7 +78,7 @@ var initialValuesFactoryByKey = map[clusterversion.Key]initialValuesFactoryFn{ func buildLatestInitialValues( opts InitialValuesOpts, ) (kvs []roachpb.KeyValue, splits []roachpb.RKey, _ error) { - schema := MakeMetadataSchema(opts.Codec, opts.DefaultZoneConfig, opts.DefaultSystemZoneConfig) + schema := MakeMetadataSchema(opts.Codec, opts.DefaultZoneConfig, opts.DefaultSystemZoneConfig, opts.DynamicSystemTableIDOffset) kvs, splits = schema.GetInitialValues() return kvs, splits, nil } diff --git a/pkg/sql/catalog/bootstrap/metadata.go b/pkg/sql/catalog/bootstrap/metadata.go index 530d6a7e8661..1e662a8f0976 100644 --- a/pkg/sql/catalog/bootstrap/metadata.go +++ b/pkg/sql/catalog/bootstrap/metadata.go @@ -43,16 +43,23 @@ type MetadataSchema struct { otherSplitIDs []uint32 otherKV []roachpb.KeyValue ids catalog.DescriptorIDSet + + // dynamicSystemTableIDOffset offsets the dynamically allocated IDs. So, if + // this is set to 500, the ids will be 501, 502, etc. + dynamicSystemTableIDOffset uint32 } +const NoOffset = 0 + // MakeMetadataSchema constructs a new MetadataSchema value which constructs // the "system" database. func MakeMetadataSchema( codec keys.SQLCodec, defaultZoneConfig *zonepb.ZoneConfig, defaultSystemZoneConfig *zonepb.ZoneConfig, + dynamicSystemTableIDOffset uint32, ) MetadataSchema { - ms := MetadataSchema{codec: codec} + ms := MetadataSchema{codec: codec, dynamicSystemTableIDOffset: dynamicSystemTableIDOffset} addSystemDatabaseToSchema(&ms, defaultZoneConfig, defaultSystemZoneConfig) return ms } @@ -347,7 +354,7 @@ func (ms MetadataSchema) FirstNonSystemDescriptorID() descpb.ID { } func (ms MetadataSchema) allocateID() (nextID descpb.ID) { - maxID := descpb.ID(keys.MaxReservedDescID) + maxID := descpb.ID(keys.MaxReservedDescID) + descpb.ID(ms.dynamicSystemTableIDOffset) for _, d := range ms.descs { if d.GetID() > maxID { maxID = d.GetID() @@ -620,7 +627,7 @@ func addSystemTenantEntry(target *MetadataSchema) { } func testingMinUserDescID(codec keys.SQLCodec) uint32 { - ms := MakeMetadataSchema(codec, zonepb.DefaultZoneConfigRef(), zonepb.DefaultSystemZoneConfigRef()) + ms := MakeMetadataSchema(codec, zonepb.DefaultZoneConfigRef(), zonepb.DefaultSystemZoneConfigRef(), NoOffset) return uint32(ms.FirstNonSystemDescriptorID()) } @@ -656,7 +663,7 @@ func GetAndHashInitialValuesToString(tenantID uint64) (initialValues string, has if tenantID > 0 { codec = keys.MakeSQLCodec(roachpb.MustMakeTenantID(tenantID)) } - ms := MakeMetadataSchema(codec, zonepb.DefaultZoneConfigRef(), zonepb.DefaultSystemZoneConfigRef()) + ms := MakeMetadataSchema(codec, zonepb.DefaultZoneConfigRef(), zonepb.DefaultSystemZoneConfigRef(), NoOffset) initialValues = InitialValuesToString(ms) h := sha256.Sum256([]byte(initialValues)) diff --git a/pkg/sql/catalog/internal/catkv/system_database_cache.go b/pkg/sql/catalog/internal/catkv/system_database_cache.go index 8ab421f6205f..f4a8eecdc506 100644 --- a/pkg/sql/catalog/internal/catkv/system_database_cache.go +++ b/pkg/sql/catalog/internal/catkv/system_database_cache.go @@ -50,6 +50,7 @@ func NewSystemDatabaseCache(codec keys.SQLCodec, settings *cluster.Settings) *Sy codec, zonepb.DefaultZoneConfigRef(), zonepb.DefaultSystemZoneConfigRef(), + bootstrap.NoOffset, ) var warm nstree.MutableCatalog _ = ms.ForEachCatalogDescriptor(func(desc catalog.Descriptor) error { diff --git a/pkg/sql/doctor/doctor.go b/pkg/sql/doctor/doctor.go index 1b0f1f089ab8..54a01cba9efe 100644 --- a/pkg/sql/doctor/doctor.go +++ b/pkg/sql/doctor/doctor.go @@ -271,7 +271,7 @@ func descReport(stdout io.Writer, desc catalog.Descriptor, format string, args . // timestamp. func DumpSQL(out io.Writer, descTable DescriptorTable, namespaceTable NamespaceTable) error { // Assume the target is an empty cluster with the same binary version - ms := bootstrap.MakeMetadataSchema(keys.SystemSQLCodec, zonepb.DefaultZoneConfigRef(), zonepb.DefaultSystemZoneConfigRef()) + ms := bootstrap.MakeMetadataSchema(keys.SystemSQLCodec, zonepb.DefaultZoneConfigRef(), zonepb.DefaultSystemZoneConfigRef(), bootstrap.NoOffset) minUserDescID := ms.FirstNonSystemDescriptorID() minUserCreatedDescID := minUserDescID + descpb.ID(len(catalogkeys.DefaultUserDBs))*2 // Print first transaction, which removes all predefined user descriptors. diff --git a/pkg/sql/tenant_creation.go b/pkg/sql/tenant_creation.go index 2dbf629d14cf..4ec3a7fdac0a 100644 --- a/pkg/sql/tenant_creation.go +++ b/pkg/sql/tenant_creation.go @@ -136,7 +136,7 @@ func (p *planner) createTenantInternal( return tid, nil } - return BootstrapTenant(ctx, p.execCfg, p.Txn(), info, initialTenantZoneConfig) + return BootstrapTenant(ctx, p.execCfg, p.Txn(), info, initialTenantZoneConfig, bootstrap.NoOffset) } // BootstrapTenant bootstraps the span of the newly created tenant identified in @@ -147,6 +147,7 @@ func BootstrapTenant( txn *kv.Txn, info mtinfopb.TenantInfoWithUsage, zfcg *zonepb.ZoneConfig, + dynamicSystemTableIDOffset uint32, ) (roachpb.TenantID, error) { tid := roachpb.MustMakeTenantID(info.ID) @@ -182,10 +183,11 @@ func BootstrapTenant( } initialValuesOpts := bootstrap.InitialValuesOpts{ - DefaultZoneConfig: zfcg, - DefaultSystemZoneConfig: zfcg, - OverrideKey: bootstrapVersionOverride, - Codec: codec, + DefaultZoneConfig: zfcg, + DefaultSystemZoneConfig: zfcg, + OverrideKey: bootstrapVersionOverride, + Codec: codec, + DynamicSystemTableIDOffset: dynamicSystemTableIDOffset, } kvs, splits, err := initialValuesOpts.GenerateInitialValues() if err != nil { diff --git a/pkg/sql/tests/system_table_test.go b/pkg/sql/tests/system_table_test.go index 89bf01943c92..af1f12fb9fde 100644 --- a/pkg/sql/tests/system_table_test.go +++ b/pkg/sql/tests/system_table_test.go @@ -54,7 +54,7 @@ func TestInitialKeys(t *testing.T) { nonDescKeys = 8 } - ms := bootstrap.MakeMetadataSchema(codec, zonepb.DefaultZoneConfigRef(), zonepb.DefaultSystemZoneConfigRef()) + ms := bootstrap.MakeMetadataSchema(codec, zonepb.DefaultZoneConfigRef(), zonepb.DefaultSystemZoneConfigRef(), bootstrap.NoOffset) kv, _ /* splits */ := ms.GetInitialValues() expected := nonDescKeys + keysPerDesc*ms.SystemDescriptorCount() if actual := len(kv); actual != expected { @@ -129,7 +129,7 @@ func TestInitialKeysAndSplits(t *testing.T) { } ms := bootstrap.MakeMetadataSchema( - codec, zonepb.DefaultZoneConfigRef(), zonepb.DefaultSystemZoneConfigRef(), + codec, zonepb.DefaultZoneConfigRef(), zonepb.DefaultSystemZoneConfigRef(), bootstrap.NoOffset, ) kvs, splits := ms.GetInitialValues() diff --git a/pkg/testutils/localtestcluster/local_test_cluster.go b/pkg/testutils/localtestcluster/local_test_cluster.go index e1d900d6b0ed..ca18aaa0f21d 100644 --- a/pkg/testutils/localtestcluster/local_test_cluster.go +++ b/pkg/testutils/localtestcluster/local_test_cluster.go @@ -276,7 +276,7 @@ func (ltc *LocalTestCluster) Start(t testing.TB, initFactory InitFactoryFn) { var splits []roachpb.RKey if !ltc.DontCreateSystemRanges { schema := bootstrap.MakeMetadataSchema( - keys.SystemSQLCodec, zonepb.DefaultZoneConfigRef(), zonepb.DefaultSystemZoneConfigRef(), + keys.SystemSQLCodec, zonepb.DefaultZoneConfigRef(), zonepb.DefaultSystemZoneConfigRef(), bootstrap.NoOffset, ) var tableSplits []roachpb.RKey initialValues, tableSplits = schema.GetInitialValues()