Skip to content

Commit

Permalink
Merge #104845
Browse files Browse the repository at this point in the history
104845: keysutils: remove TestingSQLCodec r=yuzefovich a=yuzefovich

**server: use server's codec instead of TestingSQLCodec in some tests**

Release note: None

**keysutils: extend MakePrettyScannerForNamedTables to support tenants**

This commit extends `MakePrettyScannerForNamedTables` to support keys from the secondary tenants' key space, thus, removing the last usage of `TestingSQLCodec`. The way this is achieved is similar to how we overwrite the tableKeyParse function for `/Table` prefix in the dictionary - in this commit, we overwrite the tenantKeyParse function for `/Tenant` prefix.

Release note: None

Fixes: #48123.

Co-authored-by: Yahor Yuzefovich <yahor@cockroachlabs.com>
  • Loading branch information
craig[bot] and yuzefovich committed Jun 15, 2023
2 parents 290f665 + 2485688 commit 8dc422f
Show file tree
Hide file tree
Showing 12 changed files with 131 additions and 90 deletions.
2 changes: 1 addition & 1 deletion pkg/cli/flags_util.go
Original file line number Diff line number Diff line change
Expand Up @@ -168,7 +168,7 @@ func (k *mvccKey) Set(value string) error {
}
*k = mvccKey(storage.MakeMVCCMetadataKey(roachpb.Key(unquoted)))
case human:
scanner := keysutil.MakePrettyScanner(nil /* tableParser */)
scanner := keysutil.MakePrettyScanner(nil /* tableParser */, nil /* tenantParser */)
key, err := scanner.Scan(keyStr)
if err != nil {
return err
Expand Down
51 changes: 31 additions & 20 deletions pkg/keys/printer.go
Original file line number Diff line number Diff line change
Expand Up @@ -233,26 +233,37 @@ func localStoreKeyParse(input string) (remainder string, output roachpb.Key) {

const strTable = "/Table/"

func tenantKeyParse(input string) (remainder string, output roachpb.Key) {
input = mustShiftSlash(input)
slashPos := strings.Index(input, "/")
if slashPos < 0 {
slashPos = len(input)
}
remainder = input[slashPos:] // `/something/else` -> `/else`
tenantIDStr := input[:slashPos]
tenantID, err := strconv.ParseUint(tenantIDStr, 10, 64)
if err != nil {
panic(&ErrUglifyUnsupported{err})
}
output = MakeTenantPrefix(roachpb.MustMakeTenantID(tenantID))
if strings.HasPrefix(remainder, strTable) {
var indexKey roachpb.Key
remainder = remainder[len(strTable)-1:]
remainder, indexKey = tableKeyParse(remainder)
output = append(output, indexKey...)
// GetTenantKeyParseFn returns a function that parses the relevant prefix of the
// tenant data into a roachpb.Key, returning the remainder and the key
// corresponding to the consumed prefix of 'input'. It is expected that the
// '/Tenant' prefix has already been removed (i.e. the input is assumed to be of
// the form '/<tenantID>/...'). If the input is of the form
// '/<tenantID>/Table/<tableID>/...', then passed-in tableKeyParseFn function is
// invoked on the '/<tableID>/...' part.
func GetTenantKeyParseFn(
tableKeyParseFn func(string) (string, roachpb.Key),
) func(input string) (remainder string, output roachpb.Key) {
return func(input string) (remainder string, output roachpb.Key) {
input = mustShiftSlash(input)
slashPos := strings.Index(input, "/")
if slashPos < 0 {
slashPos = len(input)
}
remainder = input[slashPos:] // `/something/else` -> `/else`
tenantIDStr := input[:slashPos]
tenantID, err := strconv.ParseUint(tenantIDStr, 10, 64)
if err != nil {
panic(&ErrUglifyUnsupported{err})
}
output = MakeTenantPrefix(roachpb.MustMakeTenantID(tenantID))
if strings.HasPrefix(remainder, strTable) {
var indexKey roachpb.Key
remainder = remainder[len(strTable)-1:]
remainder, indexKey = tableKeyParseFn(remainder)
output = append(output, indexKey...)
}
return remainder, output
}
return remainder, output
}

func tableKeyParse(input string) (remainder string, output roachpb.Key) {
Expand Down Expand Up @@ -830,7 +841,7 @@ func init() {
{Name: "", prefix: nil, ppFunc: decodeKeyPrint, PSFunc: tableKeyParse, sfFunc: formatTableKey},
}},
{Name: "/Tenant", start: TenantTableDataMin, end: TenantTableDataMax, Entries: []DictEntry{
{Name: "", prefix: nil, ppFunc: tenantKeyPrint, PSFunc: tenantKeyParse, sfFunc: formatTenantKey},
{Name: "", prefix: nil, ppFunc: tenantKeyPrint, PSFunc: GetTenantKeyParseFn(tableKeyParse), sfFunc: formatTenantKey},
}},
}
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/keys/printer_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -483,7 +483,7 @@ exp: %s
t.Errorf("%d: from string expected %s, got %s", i, exp, test.key.String())
}

scanner := keysutil.MakePrettyScanner(nil /* tableParser */)
scanner := keysutil.MakePrettyScanner(nil /* tableParser */, nil /* tenantParser */)
parsed, err := scanner.Scan(keyInfo)
if err != nil {
if !errors.HasType(err, (*keys.ErrUglifyUnsupported)(nil)) {
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/loqrecovery/loqrecoverypb/recovery.go
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,7 @@ func (r *RecoveryKey) UnmarshalYAML(fn func(interface{}) error) error {
if err := fn(&pretty); err != nil {
return err
}
scanner := keysutil.MakePrettyScanner(nil /* tableParser */)
scanner := keysutil.MakePrettyScanner(nil /* tableParser */, nil /* tenantParser */)
key, err := scanner.Scan(pretty)
if err != nil {
return errors.Wrapf(err, "failed to parse key %s", pretty)
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/loqrecovery/recovery_env_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -504,7 +504,7 @@ func raftLogFromPendingDescriptorUpdate(
}

func parsePrettyKey(t *testing.T, pretty string) roachpb.RKey {
scanner := keysutil.MakePrettyScanner(nil /* tableParser */)
scanner := keysutil.MakePrettyScanner(nil /* tableParser */, nil /* tenantParser */)
key, err := scanner.Scan(pretty)
if err != nil {
t.Fatalf("failed to parse key %s: %v", pretty, err)
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/reports/constraint_stats_report_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -985,7 +985,7 @@ func compileTestCase(tc baseReportTestCase) (compiledTestCase, error) {
allStores = append(allStores, sds...)
}

keyScanner := keysutils.MakePrettyScannerForNamedTables(tableToID, idxToID)
keyScanner := keysutils.MakePrettyScannerForNamedTables(roachpb.SystemTenantID, tableToID, idxToID)
ranges, err := processSplits(keyScanner, tc.splits, allStores)
if err != nil {
return compiledTestCase{}, err
Expand Down
3 changes: 2 additions & 1 deletion pkg/kv/kvserver/reports/reporter_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -565,7 +565,8 @@ func TestZoneChecker(t *testing.T) {
splits[i].key = ranges[i].split
}
keyScanner := keysutils.MakePrettyScannerForNamedTables(
map[string]int{"t1": t1ID} /* tableNameToID */, nil /* idxNameToID */)
roachpb.SystemTenantID, map[string]int{"t1": t1ID} /* tableNameToID */, nil, /* idxNameToID */
)
rngs, err := processSplits(keyScanner, splits, nil /* stores */)
require.NoError(t, err)

Expand Down
1 change: 0 additions & 1 deletion pkg/server/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -521,7 +521,6 @@ go_test(
"//pkg/testutils",
"//pkg/testutils/datapathutils",
"//pkg/testutils/diagutils",
"//pkg/testutils/keysutils",
"//pkg/testutils/serverutils",
"//pkg/testutils/skip",
"//pkg/testutils/sqlutils",
Expand Down
7 changes: 3 additions & 4 deletions pkg/server/decommission_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/security/username"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/keysutils"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/testutils/skip"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
Expand Down Expand Up @@ -129,8 +128,8 @@ func TestDecommissionPreCheckEvaluation(t *testing.T) {
require.NoError(t, err)
tblBID, err := firstSvr.admin.queryTableID(ctx, username.RootUserName(), "test", "tblB")
require.NoError(t, err)
startKeyTblA := keysutils.TestingSQLCodec.TablePrefix(uint32(tblAID))
startKeyTblB := keysutils.TestingSQLCodec.TablePrefix(uint32(tblBID))
startKeyTblA := firstSvr.Codec().TablePrefix(uint32(tblAID))
startKeyTblB := firstSvr.Codec().TablePrefix(uint32(tblBID))

// Split off ranges for tblA and tblB.
_, rDescA, err := firstSvr.SplitRange(startKeyTblA)
Expand Down Expand Up @@ -234,7 +233,7 @@ func TestDecommissionPreCheckOddToEven(t *testing.T) {
runQueries(alterQueries...)
tblAID, err := firstSvr.admin.queryTableID(ctx, username.RootUserName(), "test", "tblA")
require.NoError(t, err)
startKeyTblA := keysutils.TestingSQLCodec.TablePrefix(uint32(tblAID))
startKeyTblA := firstSvr.Codec().TablePrefix(uint32(tblAID))

// Split off range for tblA.
_, rDescA, err := firstSvr.SplitRange(startKeyTblA)
Expand Down
36 changes: 21 additions & 15 deletions pkg/testutils/keysutils/pretty_scanner.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,18 +22,25 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/keysutil"
)

// MakePrettyScannerForNamedTables create a PrettyScanner that, beside what the
// MakePrettyScannerForNamedTables creates a PrettyScanner that, beside what the
// PrettyScanner is generally able to decode, can also decode keys of the form
// "/<table name>/<index name>/1/2/3/..." using supplied maps from names to ids.
//
// TODO(nvanbenschoten): support tenant SQL keys.
// If the passed-in tenantID is not of the system tenant, then it handles keys
// of the form "/Tenant/<tenantID>/Table/<table name>/<index name>/1/2/3/...".
func MakePrettyScannerForNamedTables(
tableNameToID map[string]int, idxNameToID map[string]int,
tenantID roachpb.TenantID, tableNameToID map[string]int, idxNameToID map[string]int,
) keysutil.PrettyScanner {
return keysutil.MakePrettyScanner(func(input string) (string, roachpb.Key) {
remainder, k := parseTableKeysAsAscendingInts(input, tableNameToID, idxNameToID)
return remainder, k
})
var tableParser, tenantParser keys.KeyParserFunc
if tenantID == roachpb.SystemTenantID {
tableParser = func(input string) (string, roachpb.Key) {
return parseTableKeysAsAscendingInts(input, tableNameToID, idxNameToID)
}
} else {
tenantParser = keys.GetTenantKeyParseFn(func(input string) (string, roachpb.Key) {
return parseTableKeysAsAscendingInts(input, tableNameToID, idxNameToID)
})
}
return keysutil.MakePrettyScanner(tableParser, tenantParser)
}

// parseTableKeysAsAscendingInts takes a pretty-printed key segment like
Expand All @@ -49,7 +56,8 @@ func MakePrettyScannerForNamedTables(
//
// Notice that the input is not expected to have the "/Table" prefix (which is
// generated by pretty-printing keys). That prefix is assumed to have been
// consumed before this function is invoked (by the PrettyScanner).
// consumed before this function is invoked (by the PrettyScanner or
// GetTenantKeyParseFn).
//
// The "/..." part is returned as the remainder (the first return value).
func parseTableKeysAsAscendingInts(
Expand All @@ -67,7 +75,10 @@ func parseTableKeysAsAscendingInts(
if !ok {
panic(fmt.Sprintf("unknown table: %s", tableName))
}
output := TestingSQLCodec.TablePrefix(uint32(tableID))
// We assume that the tenant prefix (if there was any) was already removed
// and included into the output by the caller, so we use the system codec
// here.
output := keys.SystemSQLCodec.TablePrefix(uint32(tableID))
if remainder == "" {
return "", output
}
Expand Down Expand Up @@ -167,8 +178,3 @@ func parseAscendingIntIndexKey(input string) (string, roachpb.Key) {
}
return remainder, key
}

// TestingSQLCodec is a SQL key codec. It is equivalent to keys.SystemSQLCodec, but
// should be used when it is unclear which tenant should be referenced by the
// surrounding context.
var TestingSQLCodec = keys.MakeSQLCodec(roachpb.SystemTenantID)
72 changes: 39 additions & 33 deletions pkg/testutils/keysutils/pretty_scanner_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,25 +23,25 @@ import (
func TestPrettyScanner(t *testing.T) {
tests := []struct {
prettyKey string
expKey func() roachpb.Key
expKey func(roachpb.TenantID) roachpb.Key
expRemainder string
}{
{
prettyKey: "/Table/t1",
expKey: func() roachpb.Key {
return keys.SystemSQLCodec.TablePrefix(50)
expKey: func(tenantID roachpb.TenantID) roachpb.Key {
return keys.MakeSQLCodec(tenantID).TablePrefix(50)
},
},
{
prettyKey: "/Table/t1/pk",
expKey: func() roachpb.Key {
return keys.SystemSQLCodec.IndexPrefix(50, 1)
expKey: func(tenantID roachpb.TenantID) roachpb.Key {
return keys.MakeSQLCodec(tenantID).IndexPrefix(50, 1)
},
},
{
prettyKey: "/Table/t1/pk/1/2/3",
expKey: func() roachpb.Key {
k := keys.SystemSQLCodec.IndexPrefix(50, 1)
expKey: func(tenantID roachpb.TenantID) roachpb.Key {
k := keys.MakeSQLCodec(tenantID).IndexPrefix(50, 1)
k = encoding.EncodeVarintAscending(k, 1)
k = encoding.EncodeVarintAscending(k, 2)
k = encoding.EncodeVarintAscending(k, 3)
Expand All @@ -55,8 +55,8 @@ func TestPrettyScanner(t *testing.T) {
},
{
prettyKey: "/Table/t1/idx1/1/2/3",
expKey: func() roachpb.Key {
k := keys.SystemSQLCodec.IndexPrefix(50, 5)
expKey: func(tenantID roachpb.TenantID) roachpb.Key {
k := keys.MakeSQLCodec(tenantID).IndexPrefix(50, 5)
k = encoding.EncodeVarintAscending(k, 1)
k = encoding.EncodeVarintAscending(k, 2)
k = encoding.EncodeVarintAscending(k, 3)
Expand All @@ -67,32 +67,38 @@ func TestPrettyScanner(t *testing.T) {

tableToID := map[string]int{"t1": 50}
idxToID := map[string]int{"t1.idx1": 5}
scanner := MakePrettyScannerForNamedTables(tableToID, idxToID)
for _, test := range tests {
t.Run(test.prettyKey, func(t *testing.T) {
k, err := scanner.Scan(test.prettyKey)
if err != nil {
if test.expRemainder != "" {
if testutils.IsError(err, fmt.Sprintf("can't parse\"%s\"", test.expRemainder)) {
t.Fatalf("expected remainder: %s, got err: %s", test.expRemainder, err)
for _, tenantID := range []roachpb.TenantID{roachpb.SystemTenantID, roachpb.MustMakeTenantID(42)} {
scanner := MakePrettyScannerForNamedTables(tenantID, tableToID, idxToID)
for _, test := range tests {
prettyKey := test.prettyKey
if tenantID != roachpb.SystemTenantID {
prettyKey = fmt.Sprintf("/Tenant/%s%s", tenantID, prettyKey)
}
t.Run(prettyKey, func(t *testing.T) {
k, err := scanner.Scan(prettyKey)
if err != nil {
if test.expRemainder != "" {
if testutils.IsError(err, fmt.Sprintf("can't parse\"%s\"", test.expRemainder)) {
t.Fatalf("expected remainder: %s, got err: %s", test.expRemainder, err)
}
} else {
t.Fatal(err)
}
} else {
t.Fatal(err)
}
}
if test.expRemainder != "" && err == nil {
t.Fatalf("expected a remainder but got none: %s", test.expRemainder)
}
if test.expKey == nil {
if k != nil {
t.Fatalf("unexpected key returned: %s", k)
if test.expRemainder != "" && err == nil {
t.Fatalf("expected a remainder but got none: %s", test.expRemainder)
}
return
}
expKey := test.expKey()
if !k.Equal(expKey) {
t.Fatalf("expected: %+v, got %+v", []byte(expKey), []byte(k))
}
})
if test.expKey == nil {
if k != nil {
t.Fatalf("unexpected key returned: %s", k)
}
return
}
expKey := test.expKey(tenantID)
if !k.Equal(expKey) {
t.Fatalf("expected: %+v, got %+v", []byte(expKey), []byte(k))
}
})
}
}
}
Loading

0 comments on commit 8dc422f

Please sign in to comment.