Skip to content

Commit

Permalink
spanconfig: remove panics from code around targets
Browse files Browse the repository at this point in the history
Changed to return errors to the caller when dealing with protos
received over the wire. Not doing so was plain wrong before.

For all other places where we were panic-ing, changed to log.Fatal.
These pertain to spanconfig.Target types and are defensive checks to
ensure the target isn't empty -- considering we sanitize these things
and construct them ourselves, log.Fatal seems sane here.

Release note: None
  • Loading branch information
arulajmani committed Feb 11, 2022
1 parent ba6ee42 commit 2252c53
Show file tree
Hide file tree
Showing 7 changed files with 75 additions and 45 deletions.
11 changes: 7 additions & 4 deletions pkg/ccl/kvccl/kvtenantccl/connector.go
Original file line number Diff line number Diff line change
Expand Up @@ -462,13 +462,16 @@ func (c *Connector) GetSpanConfigRecords(
) (records []spanconfig.Record, _ error) {
if err := c.withClient(ctx, func(ctx context.Context, c *client) error {
resp, err := c.GetSpanConfigs(ctx, &roachpb.GetSpanConfigsRequest{
Targets: spanconfig.TargetsToTargetProtos(targets),
Targets: spanconfig.TargetsToTargetProtos(ctx, targets),
})
if err != nil {
return err
}

records = spanconfig.EntriesToRecords(resp.SpanConfigEntries)
records, err = spanconfig.EntriesToRecords(resp.SpanConfigEntries)
if err != nil {
return err
}
return nil
}); err != nil {
return nil, err
Expand All @@ -483,8 +486,8 @@ func (c *Connector) UpdateSpanConfigRecords(
) error {
return c.withClient(ctx, func(ctx context.Context, c *client) error {
_, err := c.UpdateSpanConfigs(ctx, &roachpb.UpdateSpanConfigsRequest{
ToDelete: spanconfig.TargetsToTargetProtos(toDelete),
ToUpsert: spanconfig.RecordsToSpanConfigEntries(toUpsert),
ToDelete: spanconfig.TargetsToTargetProtos(ctx, toDelete),
ToUpsert: spanconfig.RecordsToSpanConfigEntries(ctx, toUpsert),
})
return err
})
Expand Down
2 changes: 1 addition & 1 deletion pkg/rpc/auth_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -200,7 +200,7 @@ func TestTenantAuthRequest(t *testing.T) {
t, sp != nil || systemTarget != nil, "neither system target and span aren't valid args",
)
if sp != nil {
return spanconfig.MakeTargetFromSpan(*sp).SpanConfigTargetProto()
return spanconfig.MakeTargetFromSpan(*sp).SpanConfigTargetProto(context.Background())
}
return roachpb.SpanConfigTarget{
Union: &roachpb.SpanConfigTarget_SystemSpanConfigTarget{
Expand Down
21 changes: 14 additions & 7 deletions pkg/server/node.go
Original file line number Diff line number Diff line change
Expand Up @@ -1547,15 +1547,17 @@ func (emptyMetricStruct) MetricStruct() {}
func (n *Node) GetSpanConfigs(
ctx context.Context, req *roachpb.GetSpanConfigsRequest,
) (*roachpb.GetSpanConfigsResponse, error) {
records, err := n.spanConfigAccessor.GetSpanConfigRecords(
ctx, spanconfig.TargetProtosToTargets(req.Targets),
)
targets, err := spanconfig.TargetProtosToTargets(req.Targets)
if err != nil {
return nil, err
}
records, err := n.spanConfigAccessor.GetSpanConfigRecords(ctx, targets)
if err != nil {
return nil, err
}

return &roachpb.GetSpanConfigsResponse{
SpanConfigEntries: spanconfig.RecordsToSpanConfigEntries(records),
SpanConfigEntries: spanconfig.RecordsToSpanConfigEntries(ctx, records),
}, nil
}

Expand All @@ -1566,11 +1568,16 @@ func (n *Node) UpdateSpanConfigs(
// TODO(irfansharif): We want to protect ourselves from tenants creating
// outlandishly large string buffers here and OOM-ing the host cluster. Is
// the maximum protobuf message size enough of a safeguard?
err := n.spanConfigAccessor.UpdateSpanConfigRecords(
ctx, spanconfig.TargetProtosToTargets(req.ToDelete), spanconfig.EntriesToRecords(req.ToUpsert),
)
toUpsert, err := spanconfig.EntriesToRecords(req.ToUpsert)
if err != nil {
return nil, err
}
toDelete, err := spanconfig.TargetProtosToTargets(req.ToDelete)
if err != nil {
return nil, err
}
if err := n.spanConfigAccessor.UpdateSpanConfigRecords(ctx, toDelete, toUpsert); err != nil {
return nil, err
}
return &roachpb.UpdateSpanConfigsResponse{}, nil
}
1 change: 1 addition & 0 deletions pkg/spanconfig/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ go_library(
"//pkg/sql/catalog/descpb",
"//pkg/util/encoding",
"//pkg/util/hlc",
"//pkg/util/log",
"@com_github_cockroachdb_errors//:errors",
],
)
Expand Down
30 changes: 16 additions & 14 deletions pkg/spanconfig/spanconfigkvaccessor/kvaccessor.go
Original file line number Diff line number Diff line change
Expand Up @@ -79,7 +79,7 @@ func (k *KVAccessor) GetSpanConfigRecords(
return nil, err
}

getStmt, getQueryArgs := k.constructGetStmtAndArgs(targets)
getStmt, getQueryArgs := k.constructGetStmtAndArgs(ctx, targets)
it, err := k.ie.QueryIteratorEx(ctx, "get-span-cfgs", k.optionalTxn,
sessiondata.InternalExecutorOverride{User: security.RootUserName()},
getStmt, getQueryArgs...,
Expand Down Expand Up @@ -159,19 +159,19 @@ func (k *KVAccessor) updateSpanConfigRecordsWithTxn(
var deleteStmt string
var deleteQueryArgs []interface{}
if len(toDelete) > 0 {
deleteStmt, deleteQueryArgs = k.constructDeleteStmtAndArgs(toDelete)
deleteStmt, deleteQueryArgs = k.constructDeleteStmtAndArgs(ctx, toDelete)
}

var upsertStmt, validationStmt string
var upsertQueryArgs, validationQueryArgs []interface{}
if len(toUpsert) > 0 {
var err error
upsertStmt, upsertQueryArgs, err = k.constructUpsertStmtAndArgs(toUpsert)
upsertStmt, upsertQueryArgs, err = k.constructUpsertStmtAndArgs(ctx, toUpsert)
if err != nil {
return err
}

validationStmt, validationQueryArgs = k.constructValidationStmtAndArgs(toUpsert)
validationStmt, validationQueryArgs = k.constructValidationStmtAndArgs(ctx, toUpsert)
}

if len(toDelete) > 0 {
Expand Down Expand Up @@ -215,7 +215,9 @@ func (k *KVAccessor) updateSpanConfigRecordsWithTxn(

// constructGetStmtAndArgs constructs the statement and query arguments needed
// to fetch span configs for the given spans.
func (k *KVAccessor) constructGetStmtAndArgs(targets []spanconfig.Target) (string, []interface{}) {
func (k *KVAccessor) constructGetStmtAndArgs(
ctx context.Context, targets []spanconfig.Target,
) (string, []interface{}) {
// We want to fetch the overlapping span configs for each requested span in
// a single round trip and using only constrained index scans. For a single
// requested span, we effectively want to query the following:
Expand Down Expand Up @@ -261,7 +263,7 @@ func (k *KVAccessor) constructGetStmtAndArgs(targets []spanconfig.Target) (strin
}

startKeyIdx, endKeyIdx := i*2, (i*2)+1
encodedSp := target.Encode()
encodedSp := target.Encode(ctx)
queryArgs[startKeyIdx] = encodedSp.Key
queryArgs[endKeyIdx] = encodedSp.EndKey

Expand All @@ -285,7 +287,7 @@ SELECT start_key, end_key, config FROM (
// constructDeleteStmtAndArgs constructs the statement and query arguments
// needed to delete span configs for the given spans.
func (k *KVAccessor) constructDeleteStmtAndArgs(
toDelete []spanconfig.Target,
ctx context.Context, toDelete []spanconfig.Target,
) (string, []interface{}) {
// We're constructing a single delete statement to delete all requested
// spans. It's of the form:
Expand All @@ -297,7 +299,7 @@ func (k *KVAccessor) constructDeleteStmtAndArgs(
deleteQueryArgs := make([]interface{}, len(toDelete)*2)
for i, toDel := range toDelete {
startKeyIdx, endKeyIdx := i*2, (i*2)+1
encodedSp := toDel.Encode()
encodedSp := toDel.Encode(ctx)
deleteQueryArgs[startKeyIdx] = encodedSp.Key
deleteQueryArgs[endKeyIdx] = encodedSp.EndKey
values[i] = fmt.Sprintf("($%d::BYTES, $%d::BYTES)",
Expand All @@ -311,7 +313,7 @@ func (k *KVAccessor) constructDeleteStmtAndArgs(
// constructUpsertStmtAndArgs constructs the statement and query arguments
// needed to upsert the given span config entries.
func (k *KVAccessor) constructUpsertStmtAndArgs(
toUpsert []spanconfig.Record,
ctx context.Context, toUpsert []spanconfig.Record,
) (string, []interface{}, error) {
// We're constructing a single upsert statement to upsert all requested
// spans. It's of the form:
Expand All @@ -328,8 +330,8 @@ func (k *KVAccessor) constructUpsertStmtAndArgs(
}

startKeyIdx, endKeyIdx, configIdx := i*3, (i*3)+1, (i*3)+2
upsertQueryArgs[startKeyIdx] = record.Target.Encode().Key
upsertQueryArgs[endKeyIdx] = record.Target.Encode().EndKey
upsertQueryArgs[startKeyIdx] = record.Target.Encode(ctx).Key
upsertQueryArgs[endKeyIdx] = record.Target.Encode(ctx).EndKey
upsertQueryArgs[configIdx] = marshaled
upsertValues[i] = fmt.Sprintf("($%d::BYTES, $%d::BYTES, $%d::BYTES)",
startKeyIdx+1, endKeyIdx+1, configIdx+1) // prepared statement placeholders (1-indexed)
Expand All @@ -343,7 +345,7 @@ func (k *KVAccessor) constructUpsertStmtAndArgs(
// needed to validate that the spans being upserted don't violate table
// invariants (spans are non overlapping).
func (k *KVAccessor) constructValidationStmtAndArgs(
toUpsert []spanconfig.Record,
ctx context.Context, toUpsert []spanconfig.Record,
) (string, []interface{}) {
// We want to validate that upserting spans does not break the invariant
// that spans in the table are non-overlapping. We only need to validate
Expand Down Expand Up @@ -384,8 +386,8 @@ func (k *KVAccessor) constructValidationStmtAndArgs(
}

startKeyIdx, endKeyIdx := i*2, (i*2)+1
validationQueryArgs[startKeyIdx] = entry.Target.Encode().Key
validationQueryArgs[endKeyIdx] = entry.Target.Encode().EndKey
validationQueryArgs[startKeyIdx] = entry.Target.Encode(ctx).Key
validationQueryArgs[endKeyIdx] = entry.Target.Encode(ctx).EndKey

fmt.Fprintf(&validationInnerStmtBuilder, `
SELECT count(*) = 1 FROM (
Expand Down
52 changes: 34 additions & 18 deletions pkg/spanconfig/target.go
Original file line number Diff line number Diff line change
Expand Up @@ -10,7 +10,13 @@

package spanconfig

import "github.com/cockroachdb/cockroach/pkg/roachpb"
import (
"context"

"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/errors"
)

// Target specifies the target of an associated span configuration.
type Target struct {
Expand All @@ -20,14 +26,14 @@ type Target struct {
}

// MakeTarget returns a new Target.
func MakeTarget(t roachpb.SpanConfigTarget) Target {
func MakeTarget(t roachpb.SpanConfigTarget) (Target, error) {
switch t.Union.(type) {
case *roachpb.SpanConfigTarget_Span:
return MakeTargetFromSpan(*t.GetSpan())
return MakeTargetFromSpan(*t.GetSpan()), nil
// TODO(arul): Add a case here for SpanConfigTarget_SystemTarget once we've
// taught and tested the KVAccessor to work with system targets.
default:
panic("cannot handle target")
return Target{}, errors.AssertionFailedf("unknown type of system target %v", t)
}
}

Expand Down Expand Up @@ -71,15 +77,16 @@ func (t Target) GetSystemTarget() SystemTarget {

// Encode returns an encoded span suitable for persistence in
// system.span_configurations.
func (t Target) Encode() roachpb.Span {
func (t Target) Encode(ctx context.Context) roachpb.Span {
switch {
case t.IsSpanTarget():
return t.span
case t.IsSystemTarget():
return t.systemTarget.encode()
default:
panic("cannot handle any other type of target")
log.Fatalf(ctx, "unknown type of system target %v", t)
}
return roachpb.Span{}
}

// Less returns true if the receiver is considered less than the supplied
Expand Down Expand Up @@ -153,7 +160,7 @@ func (t Target) isEmpty() bool {

// SpanConfigTargetProto returns a roachpb.SpanConfigTarget equivalent to the
// receiver.
func (t Target) SpanConfigTargetProto() roachpb.SpanConfigTarget {
func (t Target) SpanConfigTargetProto(ctx context.Context) roachpb.SpanConfigTarget {
switch {
case t.IsSpanTarget():
sp := t.GetSpan()
Expand All @@ -172,8 +179,9 @@ func (t Target) SpanConfigTargetProto() roachpb.SpanConfigTarget {
},
}
default:
panic("cannot handle any other type of target")
log.Fatalf(ctx, "cannot handle any other type of target")
}
return roachpb.SpanConfigTarget{}
}

// DecodeTarget takes a raw span and decodes it into a Target given its
Expand Down Expand Up @@ -205,11 +213,11 @@ func (t Targets) Less(i, j int) bool {

// RecordsToSpanConfigEntries converts a list of records to a list
// roachpb.SpanConfigEntry protos suitable for sending over the wire.
func RecordsToSpanConfigEntries(records []Record) []roachpb.SpanConfigEntry {
func RecordsToSpanConfigEntries(ctx context.Context, records []Record) []roachpb.SpanConfigEntry {
entries := make([]roachpb.SpanConfigEntry, 0, len(records))
for _, rec := range records {
entries = append(entries, roachpb.SpanConfigEntry{
Target: rec.Target.SpanConfigTargetProto(),
Target: rec.Target.SpanConfigTargetProto(ctx),
Config: rec.Config,
})
}
Expand All @@ -218,33 +226,41 @@ func RecordsToSpanConfigEntries(records []Record) []roachpb.SpanConfigEntry {

// EntriesToRecords converts a list of roachpb.SpanConfigEntries
// (received over the wire) to a list of Records.
func EntriesToRecords(entries []roachpb.SpanConfigEntry) []Record {
func EntriesToRecords(entries []roachpb.SpanConfigEntry) ([]Record, error) {
records := make([]Record, 0, len(entries))
for _, entry := range entries {
target, err := MakeTarget(entry.Target)
if err != nil {
return nil, err
}
records = append(records, Record{
Target: MakeTarget(entry.Target),
Target: target,
Config: entry.Config,
})
}
return records
return records, nil
}

// TargetsToTargetProtos converts a list of targets to a list of
// roachpb.SpanConfigTarget protos suitable for sending over the wire.
func TargetsToTargetProtos(targets []Target) []roachpb.SpanConfigTarget {
func TargetsToTargetProtos(ctx context.Context, targets []Target) []roachpb.SpanConfigTarget {
targetProtos := make([]roachpb.SpanConfigTarget, 0, len(targets))
for _, target := range targets {
targetProtos = append(targetProtos, target.SpanConfigTargetProto())
targetProtos = append(targetProtos, target.SpanConfigTargetProto(ctx))
}
return targetProtos
}

// TargetProtosToTargets converts a list of roachpb.SpanConfigTargets
// (received over the wire) to a list of Targets.
func TargetProtosToTargets(protoTargtets []roachpb.SpanConfigTarget) []Target {
func TargetProtosToTargets(protoTargtets []roachpb.SpanConfigTarget) ([]Target, error) {
targets := make([]Target, 0, len(protoTargtets))
for _, t := range protoTargtets {
targets = append(targets, MakeTarget(t))
target, err := MakeTarget(t)
if err != nil {
return nil, err
}
targets = append(targets, target)
}
return targets
return targets, nil
}
3 changes: 2 additions & 1 deletion pkg/spanconfig/target_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@
package spanconfig

import (
"context"
"math/rand"
"sort"
"testing"
Expand Down Expand Up @@ -38,7 +39,7 @@ func TestEncodeDecodeSystemTarget(t *testing.T) {

// Next, we encode/decode a spanconfig.Target that wraps a SystemTarget.
target := MakeTargetFromSystemTarget(systemTarget)
decodedTarget := DecodeTarget(target.Encode())
decodedTarget := DecodeTarget(target.Encode(context.Background()))
require.Equal(t, target, decodedTarget)
}
}
Expand Down

0 comments on commit 2252c53

Please sign in to comment.