Skip to content

Commit

Permalink
Merge #97620
Browse files Browse the repository at this point in the history
97620: multitenant: enumerate tenant capability names r=arulajmani a=ecwall

Fixes #97619

Change tenant capability names from string constants to enums to increase
type safety.

Release note: None

Co-authored-by: Evan Wall <[email protected]>
  • Loading branch information
craig[bot] and ecwall committed Feb 24, 2023
2 parents 0a60831 + 111298e commit 769eac6
Show file tree
Hide file tree
Showing 11 changed files with 118 additions and 24 deletions.
1 change: 1 addition & 0 deletions build/bazelutil/check.sh
Original file line number Diff line number Diff line change
Expand Up @@ -10,6 +10,7 @@ CONFIGS="-c grep.column=false -c grep.lineNumber=false -c grep.fullName=false"
GIT_GREP="git $CONFIGS grep"

EXISTING_GO_GENERATE_COMMENTS="
pkg/multitenant/tenantcapabilities/tenantcapabilitiespb/capabilities.go://go:generate stringer -type=TenantCapabilityName -linecomment
pkg/roachprod/vm/aws/config.go://go:generate go-bindata -mode 0600 -modtime 1400000000 -pkg aws -o embedded.go config.json old.json
pkg/roachprod/vm/aws/config.go://go:generate gofmt -s -w embedded.go
pkg/roachprod/vm/aws/config.go://go:generate goimports -w embedded.go
Expand Down
1 change: 1 addition & 0 deletions pkg/gen/stringer.bzl
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@ STRINGER_SRCS = [
"//pkg/kv/kvpb:method_string.go",
"//pkg/kv/kvserver/closedts/sidetransport:cantclosereason_string.go",
"//pkg/kv/kvserver:refreshraftreason_string.go",
"//pkg/multitenant/tenantcapabilities/tenantcapabilitiespb:tenantcapabilityname_string.go",
"//pkg/sql/catalog/catalogkeys:commenttype_string.go",
"//pkg/sql/catalog/catpb:privilegedescversion_string.go",
"//pkg/sql/catalog/descpb:formatversion_string.go",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2,6 +2,7 @@ load("//build/bazelutil/unused_checker:unused.bzl", "get_x_data")
load("@rules_proto//proto:defs.bzl", "proto_library")
load("@io_bazel_rules_go//go:def.bzl", "go_library")
load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library")
load("//build:STRINGER.bzl", "stringer")

proto_library(
name = "tenantcapabilitiespb_proto",
Expand All @@ -22,10 +23,21 @@ go_proto_library(

go_library(
name = "tenantcapabilitiespb",
srcs = ["capabilities.go"],
srcs = [
"capabilities.go",
":tenantcapabilityname-stringer", # keep
],
embed = [":tenantcapabilitiespb_go_proto"],
importpath = "github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities/tenantcapabilitiespb",
visibility = ["//visibility:public"],
deps = ["@com_github_cockroachdb_errors//:errors"],
)

stringer(
name = "tenantcapabilityname-stringer",
src = "capabilities.go",
additional_args = ["--linecomment"],
typ = "TenantCapabilityName",
)

get_x_data(name = "get_x_data")
Original file line number Diff line number Diff line change
Expand Up @@ -10,19 +10,56 @@

package tenantcapabilitiespb

import "github.com/cockroachdb/errors"

// TenantCapabilityName is a pseudo-enum of valid capability names.
type TenantCapabilityName int32

// valueOffset sets the iota offset to make sure the 0 value is not a valid
// enum value.
const valueOffset = 1

// IsSet returns true if the capability name has a non-zero value.
func (t TenantCapabilityName) IsSet() bool {
return t >= valueOffset
}

var stringToTenantCapabilityName = func() map[string]TenantCapabilityName {
numCapabilities := len(_TenantCapabilityName_index) - 1
m := make(map[string]TenantCapabilityName, numCapabilities)
for i := 0; i < numCapabilities; i++ {
startIndex := _TenantCapabilityName_index[i]
endIndex := _TenantCapabilityName_index[i+1]
s := _TenantCapabilityName_name[startIndex:endIndex]
m[s] = TenantCapabilityName(i + valueOffset)
}
return m
}()

// TenantCapabilityNameFromString converts a string to a TenantCapabilityName
// or returns an error if no conversion is possible.
func TenantCapabilityNameFromString(s string) (TenantCapabilityName, error) {
tenantCapabilityName, ok := stringToTenantCapabilityName[s]
if !ok {
return 0, errors.Newf("unknown capability: %q", s)
}
return tenantCapabilityName, nil
}

//go:generate stringer -type=TenantCapabilityName -linecomment
const (
// CanAdminSplit if set to true, grants the tenant the ability to
// successfully perform `AdminSplit` requests.
CanAdminSplit = "can_admin_split"
CanAdminSplit TenantCapabilityName = iota + valueOffset // can_admin_split
// CanViewNodeInfo if set to true, grants the tenant the ability
// retrieve node-level observability data at endpoints such as `_status/nodes`
// and in the DB Console overview page.
CanViewNodeInfo = "can_view_node_info"
CanViewNodeInfo // can_view_node_info
// CanViewTSDBMetrics if set to true, grants the tenant the ability to
// make arbitrary queries of the TSDB of the entire cluster. Currently,
// we do not store per-tenant metrics so this will surface system metrics
// to the tenant.
// TODO(davidh): Revise this once tenant-scoped metrics are implemented in
// https://github.com/cockroachdb/cockroach/issues/96438
CanViewTSDBMetrics = "can_view_tsdb_metrics"
CanViewTSDBMetrics // can_view_tsdb_metrics
)

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

16 changes: 8 additions & 8 deletions pkg/sql/multitenant_admin_function_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -58,8 +58,8 @@ type testClusterCfg struct {
numNodes int
setupClusterSetting *settings.BoolSetting
queryClusterSetting *settings.BoolSetting
setupCapability string
queryCapability string
setupCapability tenantcapabilitiespb.TenantCapabilityName
queryCapability tenantcapabilitiespb.TenantCapabilityName
}

func createTestClusterArgs(numReplicas, numVoters int32) base.TestClusterArgs {
Expand Down Expand Up @@ -242,9 +242,9 @@ type testCase struct {
queryClusterSetting *settings.BoolSetting
// Used for tests that have a capability prereq
// (eq SPLIT AT is required for UNSPLIT AT).
setupCapability string
setupCapability tenantcapabilitiespb.TenantCapabilityName
// Capability required for secondary tenant query.
queryCapability string
queryCapability tenantcapabilitiespb.TenantCapabilityName
}

func (tc testCase) runTest(
Expand Down Expand Up @@ -308,12 +308,12 @@ func (tc testCase) runTest(
var waitForTenantCapabilitiesFns []func()
setCapabilities := func(
tenantID roachpb.TenantID,
capabilities ...string,
capabilities ...tenantcapabilitiespb.TenantCapabilityName,
) {
// Filter out empty capabilities.
var caps []string
var caps []tenantcapabilitiespb.TenantCapabilityName
for _, capability := range capabilities {
if capability != "" {
if capability.IsSet() {
caps = append(caps, capability)
}
}
Expand All @@ -324,7 +324,7 @@ func (tc testCase) runTest(
if i > 0 {
builder.WriteString(", ")
}
builder.WriteString(capability)
builder.WriteString(capability.String())
}
query := fmt.Sprintf("ALTER TENANT [$1] GRANT CAPABILITY %s", builder.String())
_, err := systemDB.ExecContext(ctx, query, tenantID.ToUint64())
Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/show_tenant.go
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,7 @@ type tenantValues struct {
}

type showTenantNodeCapability struct {
name string
name tenantcapabilitiespb.TenantCapabilityName
value string
}

Expand Down Expand Up @@ -266,7 +266,7 @@ func (n *showTenantNode) Values() tree.Datums {
if n.withCapabilities {
capability := n.capability
result = append(result,
tree.NewDString(capability.name),
tree.NewDString(capability.name.String()),
tree.NewDString(capability.value),
)
}
Expand Down
27 changes: 20 additions & 7 deletions pkg/sql/tenant_capability.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,7 @@ import (
"github.com/cockroachdb/errors"
)

var capabilityTypes = map[string]*types.T{
var capabilityTypes = map[tenantcapabilitiespb.TenantCapabilityName]*types.T{
tenantcapabilitiespb.CanAdminSplit: types.Bool,
tenantcapabilitiespb.CanViewNodeInfo: types.Bool,
tenantcapabilitiespb.CanViewTSDBMetrics: types.Bool,
Expand Down Expand Up @@ -55,7 +55,10 @@ func (p *planner) AlterTenantCapability(

exprs := make([]tree.TypedExpr, len(n.Capabilities))
for i, capability := range n.Capabilities {
capabilityName := capability.Name
capabilityName, err := tenantcapabilitiespb.TenantCapabilityNameFromString(capability.Name)
if err != nil {
return nil, err
}
desiredType, ok := capabilityTypes[capabilityName]
if !ok {
return nil, pgerror.Newf(pgcode.Syntax, "unknown capability: %q", capabilityName)
Expand All @@ -75,7 +78,14 @@ func (p *planner) AlterTenantCapability(
if capabilityValue != nil {
var dummyHelper tree.IndexedVarHelper
typedValue, err := p.analyzeExpr(
ctx, capabilityValue, nil, dummyHelper, desiredType, true /* requireType */, fmt.Sprintf("%s %s", alterTenantCapabilityOp, capability.Name))
ctx,
capabilityValue,
nil, /* source */
dummyHelper,
desiredType,
true, /* requireType */
fmt.Sprintf("%s %s", alterTenantCapabilityOp, capabilityName),
)
if err != nil {
return nil, err
}
Expand Down Expand Up @@ -117,7 +127,10 @@ func (n *alterTenantCapabilityNode) startExec(params runParams) error {

dst := &tenantInfo.Capabilities
for i, capability := range n.n.Capabilities {
capabilityName := capability.Name
capabilityName, err := tenantcapabilitiespb.TenantCapabilityNameFromString(capability.Name)
if err != nil {
return err
}
typedExpr := n.typedExprs[i]
switch capabilityName {
case tenantcapabilitiespb.CanAdminSplit:
Expand All @@ -126,7 +139,7 @@ func (n *alterTenantCapabilityNode) startExec(params runParams) error {
} else {
b := true
if typedExpr != nil {
b, err = paramparse.DatumAsBool(ctx, p.EvalContext(), capabilityName, typedExpr)
b, err = paramparse.DatumAsBool(ctx, p.EvalContext(), capabilityName.String(), typedExpr)
if err != nil {
return err
}
Expand All @@ -140,7 +153,7 @@ func (n *alterTenantCapabilityNode) startExec(params runParams) error {
} else {
b := true
if typedExpr != nil {
b, err = paramparse.DatumAsBool(ctx, p.EvalContext(), capabilityName, typedExpr)
b, err = paramparse.DatumAsBool(ctx, p.EvalContext(), capabilityName.String(), typedExpr)
if err != nil {
return err
}
Expand All @@ -154,7 +167,7 @@ func (n *alterTenantCapabilityNode) startExec(params runParams) error {
} else {
b := true
if typedExpr != nil {
b, err = paramparse.DatumAsBool(ctx, p.EvalContext(), capabilityName, typedExpr)
b, err = paramparse.DatumAsBool(ctx, p.EvalContext(), capabilityName.String(), typedExpr)
if err != nil {
return err
}
Expand Down
1 change: 1 addition & 0 deletions pkg/testutils/serverutils/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@ go_library(
"//pkg/keys",
"//pkg/kv",
"//pkg/kv/kvserver/liveness/livenesspb",
"//pkg/multitenant/tenantcapabilities/tenantcapabilitiespb",
"//pkg/roachpb",
"//pkg/rpc",
"//pkg/security",
Expand Down
3 changes: 2 additions & 1 deletion pkg/testutils/serverutils/test_cluster_shim.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ import (
"testing"

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities/tenantcapabilitiespb"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
Expand Down Expand Up @@ -235,7 +236,7 @@ type TestClusterInterface interface {
// tenant capabilities for the specified tenant ID.
// Only boolean capabilities are currently supported as we wait for the
// specified capabilities to have a "true" value.
WaitForTenantCapabilities(*testing.T, roachpb.TenantID, ...string)
WaitForTenantCapabilities(*testing.T, roachpb.TenantID, ...tenantcapabilitiespb.TenantCapabilityName)
}

// SplitPoint describes a split point that is passed to SplitTable.
Expand Down
6 changes: 4 additions & 2 deletions pkg/testutils/testcluster/testcluster.go
Original file line number Diff line number Diff line change
Expand Up @@ -1778,7 +1778,9 @@ func (tc *TestCluster) SplitTable(

// WaitForTenantCapabilities implements TestClusterInterface.
func (tc *TestCluster) WaitForTenantCapabilities(
t *testing.T, tenID roachpb.TenantID, capabilityNames ...string,
t *testing.T,
tenID roachpb.TenantID,
capabilityNames ...tenantcapabilitiespb.TenantCapabilityName,
) {
for i, ts := range tc.Servers {
testutils.SucceedsSoon(t, func() error {
Expand All @@ -1787,7 +1789,7 @@ func (tc *TestCluster) WaitForTenantCapabilities(
}

if len(capabilityNames) > 0 {
missingCapabilityError := func(capabilityName string) error {
missingCapabilityError := func(capabilityName tenantcapabilitiespb.TenantCapabilityName) error {
return errors.Newf("server=%d tenant %s does not have capability %q", i, tenID, capabilityName)
}
capabilities, found := ts.Server.TenantCapabilitiesReader().GetCapabilities(tenID)
Expand Down

0 comments on commit 769eac6

Please sign in to comment.