Skip to content

Commit

Permalink
sql: copy system.tenants separately to control the copied names
Browse files Browse the repository at this point in the history
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.
  • Loading branch information
shubhamdhama committed Nov 14, 2024
1 parent b0a3235 commit b1fe111
Showing 1 changed file with 105 additions and 0 deletions.
105 changes: 105 additions & 0 deletions pkg/sql/catalog/bootstrap/initial_values.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
)

Expand Down Expand Up @@ -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,
Expand Down Expand Up @@ -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 {
Expand Down

0 comments on commit b1fe111

Please sign in to comment.