Skip to content

Commit

Permalink
fixes to the migration code
Browse files Browse the repository at this point in the history
  • Loading branch information
cthumuluru-crdb committed Nov 14, 2024
1 parent 3cac1fc commit 8c66d1d
Showing 1 changed file with 73 additions and 0 deletions.
73 changes: 73 additions & 0 deletions pkg/sql/catalog/bootstrap/initial_values.go
Original file line number Diff line number Diff line change
Expand Up @@ -7,6 +7,7 @@ package bootstrap

import (
"bytes"
"context"
"crypto/sha256"
_ "embed"
"encoding/hex"
Expand All @@ -16,7 +17,11 @@ import (
"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/config/zonepb"
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/roachpb"

"github.com/cockroachdb/cockroach/pkg/sql/sem/catconstants"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/errors"
)

Expand Down Expand Up @@ -87,6 +92,74 @@ func buildLatestInitialValues(
return kvs, splits, nil
}

func CopySystemTableKVs(
ctx context.Context, txn *kv.Txn, targetCodec keys.SQLCodec, descsIDMap map[string]uint32,
) ([]roachpb.KeyValue, error) {
sourceCodec := keys.SystemSQLCodec
tables := []catconstants.SystemTableName{
catconstants.ZonesTableName,
catconstants.TenantsTableName,
catconstants.TenantSettingsTableName,
}

batch := txn.NewBatch()
for _, table := range tables {
descID, ok := descsIDMap[string(table)]
if !ok {
log.Ops.Errorf(ctx, "descID not found for : %s", table)
return nil, errors.Errorf("descID not found for : %s", table)
}
span := sourceCodec.TableSpan(descID)
batch.Scan(span.Key, span.EndKey)
}

if err := txn.Run(ctx, batch); err != nil {
return nil, err
}

if len(batch.Results) != len(tables) {
return nil, errors.AssertionFailedf(
"unexpected batch result count, expected: %d, found: %d",
len(tables),
len(batch.Results))
}

ret := make([]roachpb.KeyValue, 0)
for i, result := range batch.Results {
if err := result.Err; err != nil {
return nil, err
}
rows := result.Rows
log.Ops.Infof(ctx, "rows count: %d, table id: %d", len(rows), descsIDMap[string(tables[i])])

tablePrefix := targetCodec.TablePrefix(descsIDMap[string(tables[i])])
kvs := make([]roachpb.KeyValue, 0, len(rows))
for _, row := range rows {
key, err := keys.StripTablePrefix(row.Key)
if err != nil {
return nil, err
}

// tablePrefix is a slice and reusing it will modify the underlying array in undesirable
// ways. Build a newKey to avoid that.
// TODO(chandrat) is it possible to avoid this garbage?
newKey := make([]byte, len(tablePrefix)+len(key))
newKey = append(newKey, tablePrefix...)
newKey = append(newKey, key...)

v := roachpb.KeyValue{
Key: newKey,
Value: *row.Value,
}
v.Value.ClearChecksum()
kvs = append(kvs, v)
}
ret = append(ret, kvs...)
}

return ret, nil
}

// hardCodedInitialValues defines an initialValuesFactoryFn using
// hard-coded values.
type hardCodedInitialValues struct {
Expand Down

0 comments on commit 8c66d1d

Please sign in to comment.