Skip to content

Commit

Permalink
builtins: add builtin to retrieve the payload(s) for a span.
Browse files Browse the repository at this point in the history
The `crdb_internal.payloads_for_span` builtin retrieves all
payloads for a given span ID, given that the span is part of an
active trace. The payloads are returned in JSONB format. If the
span is not found, or if the span does not have any payloads, the
builtin returns an empty JSON object.

With the appropriate usage of this builtin and the
`crdb_internal.trace_id` builtin as shown in the `contention_event`
logic test, all payloads for the current trace may be surfaced.

Release note (sql change): add `payloads_for_span` builtin that
takes in a span ID and returns its paylods in JSONB format. If
the span is not found, or if the span does not have any payloads,
the builtin returns an empty JSON object.
  • Loading branch information
angelapwen committed Feb 18, 2021
1 parent c76fe62 commit a83187b
Show file tree
Hide file tree
Showing 8 changed files with 65 additions and 19 deletions.
2 changes: 2 additions & 0 deletions docs/generated/sql/functions.md
Original file line number Diff line number Diff line change
Expand Up @@ -2638,6 +2638,8 @@ SELECT * FROM crdb_internal.check_consistency(true, ‘\x02’, ‘\x04’)</p>
</span></td></tr>
<tr><td><a name="crdb_internal.num_inverted_index_entries"></a><code>crdb_internal.num_inverted_index_entries(val: jsonb, version: <a href="int.html">int</a>) &rarr; <a href="int.html">int</a></code></td><td><span class="funcdesc"><p>This function is used only by CockroachDB’s developers for testing purposes.</p>
</span></td></tr>
<tr><td><a name="crdb_internal.payloads_for_span"></a><code>crdb_internal.payloads_for_span(span ID: <a href="int.html">int</a>) &rarr; jsonb</code></td><td><span class="funcdesc"><p>Returns the payload(s) of the span whose ID is passed in the argument.</p>
</span></td></tr>
<tr><td><a name="crdb_internal.pretty_key"></a><code>crdb_internal.pretty_key(raw_key: <a href="bytes.html">bytes</a>, skip_fields: <a href="int.html">int</a>) &rarr; <a href="string.html">string</a></code></td><td><span class="funcdesc"><p>This function is used only by CockroachDB’s developers for testing purposes.</p>
</span></td></tr>
<tr><td><a name="crdb_internal.range_stats"></a><code>crdb_internal.range_stats(key: <a href="bytes.html">bytes</a>) &rarr; jsonb</code></td><td><span class="funcdesc"><p>This function is used to retrieve range statistics information as a JSON object.</p>
Expand Down
8 changes: 0 additions & 8 deletions pkg/sql/crdb_internal.go
Original file line number Diff line number Diff line change
Expand Up @@ -66,7 +66,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/cockroach/pkg/util/tracing"
"github.com/cockroachdb/errors"
ptypes "github.com/gogo/protobuf/types"
)

// CrdbInternalName is the name of the crdb_internal schema.
Expand Down Expand Up @@ -1149,7 +1148,6 @@ CREATE TABLE crdb_internal.node_inflight_trace_spans (
duration INTERVAL, -- The span's duration, measured from start to Finish().
-- A span whose recording is collected before it's finished will
-- have the duration set as the "time of collection - start time".
num_payloads INT NOT NULL, -- The number of structured payloads in this span.
operation STRING NULL -- The span's operation.
)`,
populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error {
Expand Down Expand Up @@ -1177,11 +1175,6 @@ CREATE TABLE crdb_internal.node_inflight_trace_spans (
spanDuration := rec.Duration
operation := rec.Operation

var numStructured int
rec.Structured(func(any *ptypes.Any) {
numStructured++
})

if err := addRow(
// TODO(angelapwen): we're casting uint64s to int64 here,
// is that ok?
Expand All @@ -1195,7 +1188,6 @@ CREATE TABLE crdb_internal.node_inflight_trace_spans (
duration.MakeDuration(spanDuration.Nanoseconds(), 0, 0),
types.DefaultIntervalTypeMetadata,
),
tree.NewDInt(tree.DInt(numStructured)),
tree.NewDString(operation),
); err != nil {
return err
Expand Down
18 changes: 12 additions & 6 deletions pkg/sql/logictest/testdata/logic_test/contention_event
Original file line number Diff line number Diff line change
Expand Up @@ -41,18 +41,24 @@ ROLLBACK

user root

# Check that the number of payloads in the open trace is at least 1.
# TODO(angelapwen): when we have a way to pull particular payloads for
# a trace, we should verify that we're seeing the right contention event.
# As is, the payloads could be something else (though we verified manually
# and there is a contention event).
# Check that there is at least 1 contention event payload in all spans in the
# open trace.
#
# NB: the contention event is not in our trace span but in one of its
# children, so it wouldn't be found if we filtered by the trace span ID.
#
# NB: this needs the 5node-pretend59315 config because otherwise the span is not
# tracked.
#
query B
SELECT count(num_payloads) > 0 FROM crdb_internal.node_inflight_trace_spans WHERE trace_id = crdb_internal.trace_id();
WITH Spans AS (
SELECT span_id FROM crdb_internal.node_inflight_trace_spans
WHERE trace_id = crdb_internal.trace_id()
), PayloadTypes AS (
SELECT jsonb_array_elements(crdb_internal.payloads_for_span(span_id))->>'@type' AS type, crdb_internal.payloads_for_span(span_id)
FROM Spans
) SELECT count(*) > 0
FROM PayloadTypes
WHERE type = 'type.googleapis.com/cockroach.roachpb.ContentionEvent';
----
true
4 changes: 2 additions & 2 deletions pkg/sql/logictest/testdata/logic_test/crdb_internal
Original file line number Diff line number Diff line change
Expand Up @@ -250,10 +250,10 @@ SELECT * FROM crdb_internal.zones WHERE false
zone_id subzone_id target range_name database_name table_name index_name partition_name
raw_config_yaml raw_config_sql raw_config_protobuf full_config_yaml full_config_sql

query IIIIBTTIT colnames
query IIIIBTIT colnames
SELECT * FROM crdb_internal.node_inflight_trace_spans WHERE span_id < 0
----
trace_id parent_span_id span_id goroutine_id finished start_time duration num_payloads operation
trace_id parent_span_id span_id goroutine_id finished start_time duration operation

query ITTTTITTTTTTTTTI colnames
SELECT * FROM crdb_internal.ranges WHERE range_id < 0
Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/logictest/testdata/logic_test/crdb_internal_tenant
Original file line number Diff line number Diff line change
Expand Up @@ -262,10 +262,10 @@ SELECT * FROM crdb_internal.zones WHERE false
zone_id subzone_id target range_name database_name table_name index_name partition_name
raw_config_yaml raw_config_sql raw_config_protobuf full_config_yaml full_config_sql

query IIIIBTTIT colnames
query IIIIBTIT colnames
SELECT * FROM crdb_internal.node_inflight_trace_spans WHERE span_id < 0
----
trace_id parent_span_id span_id goroutine_id finished start_time duration num_payloads operation
trace_id parent_span_id span_id goroutine_id finished start_time duration operation

statement error not fully contained in tenant keyspace
SELECT * FROM crdb_internal.ranges WHERE range_id < 0
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/sem/builtins/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -84,6 +84,7 @@ go_library(
"//pkg/util/uuid",
"@com_github_cockroachdb_apd_v2//:apd",
"@com_github_cockroachdb_errors//:errors",
"@com_github_gogo_protobuf//types",
"@com_github_golang_geo//s1",
"@com_github_knz_strtime//:strtime",
"@com_github_lib_pq//oid",
Expand Down
36 changes: 36 additions & 0 deletions pkg/sql/sem/builtins/builtins.go
Original file line number Diff line number Diff line change
Expand Up @@ -73,6 +73,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/unaccent"
"github.com/cockroachdb/cockroach/pkg/util/uuid"
"github.com/cockroachdb/errors"
pbtypes "github.com/gogo/protobuf/types"
"github.com/knz/strtime"
)

Expand Down Expand Up @@ -3598,6 +3599,41 @@ may increase either contention or retry errors, or both.`,
},
),

"crdb_internal.payloads_for_span": makeBuiltin(
tree.FunctionProperties{Category: categorySystemInfo},
tree.Overload{
Types: tree.ArgTypes{{"span ID", types.Int}},
ReturnType: tree.FixedReturnType(types.Jsonb),
Fn: func(ctx *tree.EvalContext, args tree.Datums) (tree.Datum, error) {
builder := json.NewArrayBuilder(len(args))

spanID := uint64(*(args[0].(*tree.DInt)))
span, found := ctx.Settings.Tracer.GetActiveSpanFromID(spanID)
// A span may not be found if its ID was surfaced previously but its
// corresponding trace has ended by the time this builtin was run.
if !found {
// Returns an empty JSON array.
return tree.NewDJSON(builder.Build()), nil
}

for _, rec := range span.GetRecording() {
rec.Structured(func(item *pbtypes.Any) {
payload, err := protoreflect.MessageToJSON(item, true /* emitDefaults */)
if err != nil {
return
}
if payload != nil {
builder.Add(payload)
}
})
}
return tree.NewDJSON(builder.Build()), nil
},
Info: "Returns the payload(s) of the span whose ID is passed in the argument.",
Volatility: tree.VolatilityVolatile,
},
),

"crdb_internal.locality_value": makeBuiltin(
tree.FunctionProperties{Category: categorySystemInfo},
tree.Overload{
Expand Down
11 changes: 10 additions & 1 deletion pkg/util/tracing/tracer.go
Original file line number Diff line number Diff line change
Expand Up @@ -155,7 +155,8 @@ type Tracer struct {
// In normal operation, a local root Span is inserted on creation and
// removed on .Finish().
//
// The map can be introspected by `Tracer.VisitSpans`.
// The map can be introspected by `Tracer.VisitSpans`. A Span can also be
// retrieved from its ID by `Tracer.GetActiveSpanFromID`.
activeSpans struct {
// NB: it might be tempting to use a sync.Map here, but
// this incurs an allocation per Span (sync.Map does
Expand Down Expand Up @@ -676,6 +677,14 @@ func (t *Tracer) ExtractMetaFrom(carrier Carrier) (*SpanMeta, error) {
}, nil
}

// GetActiveSpanFromID retrieves any active span given its span ID.
func (t *Tracer) GetActiveSpanFromID(spanID uint64) (*Span, bool) {
t.activeSpans.Lock()
span, found := t.activeSpans.m[spanID]
t.activeSpans.Unlock()
return span, found
}

// VisitSpans invokes the visitor with all active Spans. The function will
// gracefully exit if the visitor returns iterutil.StopIteration().
func (t *Tracer) VisitSpans(visitor func(*Span) error) error {
Expand Down

0 comments on commit a83187b

Please sign in to comment.