Skip to content

Commit

Permalink
roachpb: modify SpanConfigEntry and span config RPCs to work on targets
Browse files Browse the repository at this point in the history
This patch modifies roachpb.SpanConfigEntry to tie a
roachpb.SpanConfigTarget (instead of roachpb.Span)  with a
roachpb.SpanConfig.

A roachpb.SpanConfigTarget is a union proto over roachpb.Span and
roachpb.SystemSpanConfigTarget. Given a roachpb.SpanConfigEntry,
we can use the roachpb.SpanConfigTarget to disambiguate between
regular span configurations and system span configurations.

We use the modified roachpb.SpanConfigEntry in all our RPCs. We also
replace usages of roachpb.Spans with roachpb.SpanConfigTargets in the
RPCs as well.

Release note: None
  • Loading branch information
arulajmani committed Feb 10, 2022
1 parent d10188f commit ebec94f
Show file tree
Hide file tree
Showing 11 changed files with 376 additions and 140 deletions.
18 changes: 3 additions & 15 deletions pkg/ccl/kvccl/kvtenantccl/connector.go
Original file line number Diff line number Diff line change
Expand Up @@ -461,12 +461,8 @@ func (c *Connector) GetSpanConfigRecords(
ctx context.Context, targets []spanconfig.Target,
) (records []spanconfig.Record, _ error) {
if err := c.withClient(ctx, func(ctx context.Context, c *client) error {
spans := make([]roachpb.Span, 0, len(targets))
for _, target := range targets {
spans = append(spans, *target.GetSpan())
}
resp, err := c.GetSpanConfigs(ctx, &roachpb.GetSpanConfigsRequest{
Spans: spans,
Targets: spanconfig.TargetsToTargetProtos(targets),
})
if err != nil {
return err
Expand All @@ -485,18 +481,10 @@ func (c *Connector) GetSpanConfigRecords(
func (c *Connector) UpdateSpanConfigRecords(
ctx context.Context, toDelete []spanconfig.Target, toUpsert []spanconfig.Record,
) error {
spansToDelete := make([]roachpb.Span, 0, len(toDelete))
for _, toDel := range toDelete {
spansToDelete = append(spansToDelete, roachpb.Span(toDel))
}

entriesToUpsert := spanconfig.RecordsToSpanConfigEntries(toUpsert)

return c.withClient(ctx, func(ctx context.Context, c *client) error {

_, err := c.UpdateSpanConfigs(ctx, &roachpb.UpdateSpanConfigsRequest{
ToDelete: spansToDelete,
ToUpsert: entriesToUpsert,
ToDelete: spanconfig.TargetsToTargetProtos(toDelete),
ToUpsert: spanconfig.RecordsToSpanConfigEntries(toUpsert),
})
return err
})
Expand Down
10 changes: 0 additions & 10 deletions pkg/roachpb/span_config.go
Original file line number Diff line number Diff line change
Expand Up @@ -96,16 +96,6 @@ func (c ConstraintsConjunction) String() string {
return sb.String()
}

// Equal compares two span config entries.
func (s *SpanConfigEntry) Equal(o SpanConfigEntry) bool {
return s.Span.Equal(o.Span) && s.Config.Equal(o.Config)
}

// Empty returns true if the span config entry is empty.
func (s *SpanConfigEntry) Empty() bool {
return s.Equal(SpanConfigEntry{})
}

// TestingDefaultSpanConfig exports the default span config for testing purposes.
func TestingDefaultSpanConfig() SpanConfig {
return SpanConfig{
Expand Down
99 changes: 72 additions & 27 deletions pkg/roachpb/span_config.proto
Original file line number Diff line number Diff line change
Expand Up @@ -166,56 +166,101 @@ message SpanConfig {
bool rangefeed_enabled = 10;
}

// SystemSpanConfigTarget specifies the target of system span configurations.
message SystemSpanConfigTarget {
option (gogoproto.equal) = true;

// SourceTenantID is the ID of the tenant that specified the system span
// configuration.
TenantID SourceTenantID = 1 [(gogoproto.nullable) = false];

// TargetTenantID is the ID of the tenant that the associated system span
// configuration applies to.
//
// If the host tenant is both the source and the target then the system span
// configuration applies over all ranges in the system (including those
// belonging to secondary tenants).
//
// Secondary tenants are only allowed to target themselves. The host tenant
// may use this field to target a specific secondary tenant.
TenantID TargetTenantID = 2 [(gogoproto.nullable) = false];
}

// SpanConfigTarget specifies the target of an associated span configuration.
message SpanConfigTarget {
oneof union {
// Span is a keyspan that a span config is said to apply over.
Span span = 1;
// SystemSpanConfigTarget specifies the target of a system span
// configuration.
SystemSpanConfigTarget system_span_config_target = 2;
}
}

// SpanConfigEntry ties a span to its corresponding config.
message SpanConfigEntry {
// Span is the keyspan the config is said to apply over.
Span span = 1 [(gogoproto.nullable) = false];
reserved 1;

// Config is the set of attributes that apply over the corresponding keyspan.
// Target specifies the target (keyspan(s)) the config applies over.
SpanConfigTarget target = 3 [(gogoproto.nullable) = false];

// Config is the set of attributes that apply over the corresponding target.
SpanConfig config = 2 [(gogoproto.nullable) = false];
};

// GetSpanConfigsRequest is used to fetch the span configurations over the
// specified keyspans.
// GetSpanConfigsRequest is used to fetch the span configurations and system
// span configurations.
message GetSpanConfigsRequest {
// Spans to request the configurations for. The spans listed here are not
// allowed to overlap with one another.
repeated Span spans = 1 [(gogoproto.nullable) = false];
reserved 1;

// Targets to request configurations for. The targets listed here are not
// allowed to be duplicated/overlap with one another.
repeated SpanConfigTarget targets = 2 [(gogoproto.nullable) = false];
};

// GetSpanConfigsResponse lists out the span configurations that overlap with
// the requested spans.
// GetSpanConfigsResponse lists out the span configurations and system span
// configurations that have been requested.
message GetSpanConfigsResponse {
// SpanConfigEntries capture the span configurations over the requested spans.
// The results for each Span in the matching GetSpanConfigsRequest are
// flattened out into a single slice, and follow the same ordering. It's
// possible for there to be no configurations for a given span; there'll
// simply be no entries for it.
repeated SpanConfigEntry span_config_entries = 1 [(gogoproto.nullable) = false];
//
// Any system span configurations set by the tenant are also returned if
// requested.
repeated SpanConfigEntry span_config_entries = 2 [(gogoproto.nullable) = false];

reserved 1;
};

// UpdateSpanConfigsRequest is used to update the span configurations over the
// given spans.
// UpdateSpanConfigsRequest is used to update the span configurations and system
// span configurations over the given spans.
//
// This is a "targeted" API: the targets being deleted are expected to have been
// present exactly as specified. The same is true for targets being upserted with
// new configs. If targets aren't present, an error is returned.
//
// This is a "targeted" API: the spans being deleted are expected to have been
// present with the same bounds (same start/end key); the same is true for spans
// being upserted with new configs. If bounds are mismatched, an error is
// returned. If spans are being added, they're expected to not overlap with any
// Adding configurations that target a span are expected to not overlap with any
// existing spans. When divvying up an existing span into multiple others,
// callers are expected to delete the old and upsert the new ones. This can
// happen as part of the same request; we delete the spans marked for deletion
// before upserting whatever was requested.
// happen as part of the same request, as we delete targets marked for deletion
// before upserting what was requested.
//
// Spans are not allowed to overlap with other spans in the same list but can
// across lists. This is necessary to support the delete+upsert semantics
// described above.
// Targets are not allowed to overlap with other targets in the same list. They
// are, however, allowed to overlap across lists. This is necessary to support
// the delete+upsert semantics described above.
// Targets of configurations being added must not overlap with any existing
// targets.
message UpdateSpanConfigsRequest {
// ToDelete captures the spans we want to delete configs for.
repeated Span to_delete = 1 [(gogoproto.nullable) = false];
reserved 1, 2;

// ToDelete captures the targets we want to delete configs for.
repeated SpanConfigTarget to_delete = 3 [(gogoproto.nullable) = false];

// ToUpsert captures the spans we want to upsert and the configs we want to
// upsert with.
repeated SpanConfigEntry to_upsert = 2 [(gogoproto.nullable) = false];
// ToUpsert captures the targets we want to upsert and the configs we want
// to upsert with.
repeated SpanConfigEntry to_upsert = 4 [(gogoproto.nullable) = false];
};

message UpdateSpanConfigsResponse { };
Expand Down
1 change: 1 addition & 0 deletions pkg/rpc/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -98,6 +98,7 @@ go_test(
"//pkg/security",
"//pkg/security/securitytest",
"//pkg/settings/cluster",
"//pkg/spanconfig",
"//pkg/testutils",
"//pkg/testutils/skip",
"//pkg/util",
Expand Down
71 changes: 50 additions & 21 deletions pkg/rpc/auth_tenant.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ import (

"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/errors"
"github.com/cockroachdb/logtags"
"google.golang.org/grpc"
)
Expand Down Expand Up @@ -269,14 +270,9 @@ func (a tenantAuthorizer) authTenantSettings(
func (a tenantAuthorizer) authGetSpanConfigs(
tenID roachpb.TenantID, args *roachpb.GetSpanConfigsRequest,
) error {
tenSpan := tenantPrefix(tenID)
for _, sp := range args.Spans {
rSpan, err := keys.SpanAddr(sp)
if err != nil {
return authError(err.Error())
}
if !tenSpan.ContainsKeyRange(rSpan.Key, rSpan.EndKey) {
return authErrorf("requested key span %s not fully contained in tenant keyspace %s", rSpan, tenSpan)
for _, target := range args.Targets {
if err := validateSpanConfigTarget(tenID, target); err != nil {
return err
}
}
return nil
Expand All @@ -287,8 +283,45 @@ func (a tenantAuthorizer) authGetSpanConfigs(
func (a tenantAuthorizer) authUpdateSpanConfigs(
tenID roachpb.TenantID, args *roachpb.UpdateSpanConfigsRequest,
) error {
tenSpan := tenantPrefix(tenID)
validate := func(sp roachpb.Span) error {
for _, entry := range args.ToUpsert {
if err := validateSpanConfigTarget(tenID, entry.Target); err != nil {
return err
}
}
for _, target := range args.ToDelete {
if err := validateSpanConfigTarget(tenID, target); err != nil {
return err
}
}

return nil
}

// validateSpanConfigTarget validates that the tenant is authorized to interact
// with the supplied span config target. In particular, span targets must be
// wholly contained within the tenant keyspace and system span config targets
// must be well-formed.
func validateSpanConfigTarget(
tenID roachpb.TenantID, spanConfigTarget roachpb.SpanConfigTarget,
) error {
validateSystemTarget := func(target roachpb.SystemSpanConfigTarget) error {
if tenID == roachpb.SystemTenantID {
// Nothing to validate, the system tenant is allowed to set system span
// configurations over secondary tenants.
return nil
}

if target.SourceTenantID != tenID || target.SourceTenantID != target.TargetTenantID {
return authErrorf(
"secondary tenants cannot interact with system span configurations of other tenants",
)
}

return nil
}

validateSpan := func(sp roachpb.Span) error {
tenSpan := tenantPrefix(tenID)
rSpan, err := keys.SpanAddr(sp)
if err != nil {
return authError(err.Error())
Expand All @@ -299,18 +332,14 @@ func (a tenantAuthorizer) authUpdateSpanConfigs(
return nil
}

for _, entry := range args.ToUpsert {
if err := validate(entry.Span); err != nil {
return err
}
}
for _, span := range args.ToDelete {
if err := validate(span); err != nil {
return err
}
switch spanConfigTarget.Union.(type) {
case *roachpb.SpanConfigTarget_Span:
return validateSpan(*spanConfigTarget.GetSpan())
case *roachpb.SpanConfigTarget_SystemSpanConfigTarget:
return validateSystemTarget(*spanConfigTarget.GetSystemSpanConfigTarget())
default:
return errors.AssertionFailedf("unknown span config target type")
}

return nil
}

func contextWithTenant(ctx context.Context, tenID roachpb.TenantID) context.Context {
Expand Down
Loading

0 comments on commit ebec94f

Please sign in to comment.