From b1fe1110600e19a706a9b3e4e0b3530654835c57 Mon Sep 17 00:00:00 2001 From: Shubham Dhama Date: Thu, 14 Nov 2024 16:30:00 +0000 Subject: [PATCH] sql: copy `system.tenants` separately to control the copied names When we start the `TenantTwo` we want it to have `system` as its name from the very beginning. To achieve this, instead of rewriting keys we use KVWriter. `TenantTwo`'s name would become `system`, and `TenantOne`'s name would be whatever is passed to `create_tenant` command (ideally, `main`). Note, we are chaning this value for `TenantTwo`'s `tenants` table, not for the `TenantOne`'s system table. --- pkg/sql/catalog/bootstrap/initial_values.go | 105 ++++++++++++++++++++ 1 file changed, 105 insertions(+) diff --git a/pkg/sql/catalog/bootstrap/initial_values.go b/pkg/sql/catalog/bootstrap/initial_values.go index f44b900465b3..4d008d5610a3 100644 --- a/pkg/sql/catalog/bootstrap/initial_values.go +++ b/pkg/sql/catalog/bootstrap/initial_values.go @@ -18,11 +18,16 @@ import ( "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/multitenant/mtinfo" + "github.com/cockroachdb/cockroach/pkg/multitenant/mtinfopb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/isql" "github.com/cockroachdb/cockroach/pkg/sql/sem/catconstants" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/errors" ) @@ -117,6 +122,13 @@ func copySystemTableKVs( ret := make([]roachpb.KeyValue, 0) sourceCodec := keys.SystemSQLCodec + if kvs, err := systemTenantTableKVs( + ctx, kvTxn, isqlTxn, targetCodec, descMap); err != nil { + return nil, err + } else { + ret = append(ret, kvs...) + } + tables := []catconstants.SystemTableName{ catconstants.ZonesTableName, // we need to filter out data for tables we don't need catconstants.TenantSettingsTableName, @@ -174,6 +186,99 @@ func copySystemTableKVs( return ret, nil } +func systemTenantTableKVs( + ctx context.Context, + kvTxn *kv.Txn, + isqlTxn isql.Txn, + targetCodec keys.SQLCodec, + descMap map[string]catalog.Descriptor, +) ([]roachpb.KeyValue, error) { + ret := make([]roachpb.KeyValue, 0) + + tenants, err := GetTenantRecords(ctx, isqlTxn) + const expectedTenantsCount = 2 + if err != nil { + return nil, err + } else if len(tenants) < expectedTenantsCount { + return nil, errors.Errorf( + "unexpected tenants count: expected %d, found %d", + expectedTenantsCount, len(tenants)) + } + + tenantOne := tenants[0] + if tenantOne.ID != roachpb.TenantOne.ToUint64() { + return nil, errors.Errorf( + "expected first tenant with id %d", roachpb.TenantOne.ToUint64()) + } + tenantTwo := tenants[1] + if tenantTwo.ID != roachpb.TenantTwo.ToUint64() { + return nil, errors.Errorf( + "expected second tenant with id %d", roachpb.TenantTwo.ToUint64()) + } + + tenantOne.Name = tenantTwo.Name + tenantTwo.Name = catconstants.SystemTenantName + + desc := descMap[string(catconstants.TenantsTableName)] + tenantsTableWriter := MakeKVWriter(targetCodec, desc.(catalog.TableDescriptor)) + for _, tenantInfo := range tenants { + info := tenantInfo.ProtoInfo + infoBytes, err := protoutil.Marshal(&info) + if err != nil { + panic(err) + } + kvs, err := tenantsTableWriter.RecordToKeyValues( + // ID + tree.NewDInt(tree.DInt(tenantInfo.ID)), + // active -- deprecated. + tree.MakeDBool(true), + // info. + // TODO(shubham): Deal with some extra data later + tree.NewDBytes(tree.DBytes(infoBytes)), + // name. + tree.NewDString(string(tenantInfo.Name)), + // data_state. + tree.NewDInt(tree.DInt(tenantInfo.DataState)), + // service_mode. + tree.NewDInt(tree.DInt(tenantInfo.ServiceMode)), + ) + if err != nil { + panic(err) + } + + ret = append(ret, kvs...) + } + + return ret, nil +} + +// GetTenantRecords +// +// TODO(shubham): use sql.GetTenantRecordByID instead. It's copied as it is to +// avoid some dependency headache for the time being. GetTenantRecordByID +// retrieves a tenant in system.tenants. +func GetTenantRecords(ctx context.Context, txn isql.Txn) ([]*mtinfopb.TenantInfo, error) { + q := `SELECT id, info, name, data_state, service_mode FROM system.tenants WHERE data_state != $1 ORDER BY id` + var arg interface{} = mtinfopb.DataStateDrop + + rows, err := txn.QueryBufferedEx(ctx, "get-tenants-info", + txn.KV(), sessiondata.NodeUserSessionDataOverride, q, arg) + if err != nil { + return nil, err + } + + tenants := make([]*mtinfopb.TenantInfo, 0, len(rows)) + for _, row := range rows { + _, info, err := mtinfo.GetTenantInfoFromSQLRow(row) + if err != nil { + return nil, err + } + tenants = append(tenants, info) + } + + return tenants, nil +} + // hardCodedInitialValues defines an initialValuesFactoryFn using // hard-coded values. type hardCodedInitialValues struct {