Skip to content

Commit

Permalink
Merge #56238
Browse files Browse the repository at this point in the history
56238: tracing: make Span.crdbSpan a pointer r=RaduBerinde,irfansharif a=tbg

A `Span` essentially fans out over any subset of the below:

- CRDB trace span
- net/trace span
- external opentracing-comatible tracer Span

The latter two are already pointers, which means that there is a
convenient check for their absence in tracing-internal code.

This wasn't true for crdbSpan, which has bitten me in the past.

Make things more idiomatic by using a pointer for this one, too.
As currently written, our tracer will always populate the crdbSpan
whenever net/trace or opentracing tracers are active. However,
there is no good reason to do that, and even if there were, it
should not reflect as a fundamental requirement in the code.

One nice outcome of this change is that the noopSpan is now
`&Span{tracer: t}`, and all fields except for the tracer are nil,
meaning that any bug around noop spans will quickly manifest as an NPE.
(One past bug involved erroneously assigning an operation name to the
noopSpan).

Other than a few clarification that resulted from the pointer change,
this commit also improves `startSpanGeneric` to avoid populating a
struct's internals after the initial assignment, a pattern which I
have found easier to get wrong. Now, all ingredients are held in
local variables, which are assigned to `crdbSpan` wholesale.

Release note: None


Co-authored-by: Tobias Grieger <[email protected]>
  • Loading branch information
craig[bot] and tbg committed Nov 4, 2020
2 parents 25edeb8 + 8010c90 commit b03fb74
Show file tree
Hide file tree
Showing 5 changed files with 185 additions and 152 deletions.
53 changes: 15 additions & 38 deletions pkg/util/tracing/shadow.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,6 @@ import (

"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/logtags"
lightstep "github.com/lightstep/lightstep-tracer-go"
opentracing "github.com/opentracing/opentracing-go"
zipkin "github.com/openzipkin-contrib/zipkin-go-opentracing"
Expand Down Expand Up @@ -76,55 +75,33 @@ func (st *shadowTracer) Close() {
st.manager.Close(st)
}

// otLogTagsOption is an opentracing.StartSpanOption that inserts the log
// tags into newly created spans.
type otLogTagsOption logtags.Buffer

func (o *otLogTagsOption) Apply(opts *opentracing.StartSpanOptions) {
tags := (*logtags.Buffer)(o).Get()
if len(tags) == 0 {
return
}
if opts.Tags == nil {
opts.Tags = map[string]interface{}{}
}
for _, tag := range tags {
opts.Tags[tagName(tag.Key())] = tag.Value()
}
}

// linkShadowSpan creates and links a Shadow Span to the passed-in Span (i.e.
// fills in s.shadowTr and s.shadowSpan). This should only be called when
// shadow tracing is enabled.
//
// The Shadow Span will have a parent if parentShadowCtx is not nil.
// parentType is ignored if parentShadowCtx is nil.
// makeShadowSpan creates an otSpan for construction of a Span.
// This must be called with a non-nil shadowTr, which must have
// been confirmed to be compatible with parentShadowCtx.
//
// The tags (including logTags) from s are copied to the Shadow Span.
func linkShadowSpan(
s *Span,
// The span contained in `otSpan` will have a parent if parentShadowCtx
// is not nil. parentType is ignored if parentShadowCtx is nil.
func makeShadowSpan(
shadowTr *shadowTracer,
parentShadowCtx opentracing.SpanContext,
parentType opentracing.SpanReferenceType,
) {
opName string,
startTime time.Time,
) otSpan {
// Create the shadow lightstep Span.
var opts []opentracing.StartSpanOption
opts := make([]opentracing.StartSpanOption, 0, 2)
// Replicate the options, using the lightstep context in the reference.
opts = append(opts, opentracing.StartTime(s.crdb.startTime))
if s.crdb.logTags != nil {
opts = append(opts, (*otLogTagsOption)(s.crdb.logTags))
}
if s.crdb.mu.tags != nil {
opts = append(opts, s.crdb.mu.tags)
}
opts = append(opts, opentracing.StartTime(startTime))
if parentShadowCtx != nil {
opts = append(opts, opentracing.SpanReference{
Type: parentType,
ReferencedContext: parentShadowCtx,
})
}
s.ot.shadowTr = shadowTr
s.ot.shadowSpan = shadowTr.StartSpan(s.crdb.operation, opts...)
return otSpan{
shadowTr: shadowTr,
shadowSpan: shadowTr.StartSpan(opName, opts...),
}
}

func createLightStepTracer(token string) (shadowTracerManager, opentracing.Tracer) {
Expand Down
127 changes: 65 additions & 62 deletions pkg/util/tracing/span.go
Original file line number Diff line number Diff line change
Expand Up @@ -79,6 +79,36 @@ type SpanStats interface {
Stats() map[string]string
}

type crdbSpanMu struct {
syncutil.Mutex
// duration is initialized to -1 and set on Finish().
duration time.Duration

// recording maintains state once StartRecording() is called.
recording struct {
recordingType RecordingType
recordedLogs []opentracing.LogRecord
// children contains the list of child spans started after this Span
// started recording.
children []*crdbSpan
// remoteSpan contains the list of remote child spans manually imported.
remoteSpans []tracingpb.RecordedSpan
}

// tags are only set when recording. These are tags that have been added to
// this Span, and will be appended to the tags in logTags when someone
// needs to actually observe the total set of tags that is a part of this
// Span.
// TODO(radu): perhaps we want a recording to capture all the tags (even
// those that were set before recording started)?
tags opentracing.Tags

stats SpanStats

// The Span's associated baggage.
Baggage map[string]string
}

type crdbSpan struct {
// The traceID, probabilistically unique.
traceID uint64
Expand All @@ -102,35 +132,7 @@ type crdbSpan struct {
// Atomic flag used to avoid taking the mutex in the hot path.
recording int32

mu struct {
syncutil.Mutex
// duration is initialized to -1 and set on Finish().
duration time.Duration

// recording maintains state once StartRecording() is called.
recording struct {
recordingType RecordingType
recordedLogs []opentracing.LogRecord
// children contains the list of child spans started after this Span
// started recording.
children []*crdbSpan
// remoteSpan contains the list of remote child spans manually imported.
remoteSpans []tracingpb.RecordedSpan
}

// tags are only set when recording. These are tags that have been added to
// this Span, and will be appended to the tags in logTags when someone
// needs to actually observe the total set of tags that is a part of this
// Span.
// TODO(radu): perhaps we want a recording to capture all the tags (even
// those that were set before recording started)?
tags opentracing.Tags

stats SpanStats

// The Span's associated baggage.
Baggage map[string]string
}
mu crdbSpanMu
}

func (s *crdbSpan) isRecording() bool {
Expand Down Expand Up @@ -181,8 +183,10 @@ type otSpan struct {
type Span struct {
tracer *Tracer // never nil

// Internal trace Span. Can be zero.
crdb crdbSpan
// Internal trace Span; nil if not tracing to crdb.
// When not-nil, allocated together with the surrounding Span for
// performance.
crdb *crdbSpan
// x/net/trace.Trace instance; nil if not tracing to x/net/trace.
netTr trace.Trace
// Shadow tracer and Span; zero if not using a shadow tracer.
Expand All @@ -194,10 +198,7 @@ func (s *Span) isBlackHole() bool {
}

func (s *Span) isNoop() bool {
// NB: this is the same as `s` being zero with the exception
// of the `tracer` field. However, `Span` is not comparable,
// so this can't be expressed easily.
return s.isBlackHole() && s.crdb.traceID == 0
return s.crdb == nil && s.netTr == nil && s.ot == (otSpan{})
}

// IsRecording returns true if the Span is recording its events.
Expand Down Expand Up @@ -405,41 +406,45 @@ func (s *Span) Finish() {

// Meta returns the information which needs to be propagated across
// process boundaries in order to derive child spans from this Span.
//
// TODO(andrei, radu): Should this return nil for a WithForceRealSpan Span
// that's not currently recording? That might save work and allocations when
// creating child spans.
// This may return nil, which is a valid input to `WithRemoteParent`,
// if the Span has been optimized out.
func (s *Span) Meta() *SpanMeta {
s.crdb.mu.Lock()
defer s.crdb.mu.Unlock()
n := len(s.crdb.mu.Baggage)
// In the common case, we have no baggage, so avoid making an empty map.
var baggageCopy map[string]string
if n > 0 {
baggageCopy = make(map[string]string, n)
}
for k, v := range s.crdb.mu.Baggage {
baggageCopy[k] = v
var traceID uint64
var spanID uint64
var recordingType RecordingType
var baggage map[string]string

if s.crdb != nil {
traceID, spanID = s.crdb.traceID, s.crdb.spanID
s.crdb.mu.Lock()
defer s.crdb.mu.Unlock()
n := len(s.crdb.mu.Baggage)
// In the common case, we have no baggage, so avoid making an empty map.
if n > 0 {
baggage = make(map[string]string, n)
}
for k, v := range s.crdb.mu.Baggage {
baggage[k] = v
}
if s.crdb.isRecording() {
recordingType = s.crdb.mu.recording.recordingType
}
}

var shadowTrTyp string
var shadowCtx opentracing.SpanContext
if s.ot.shadowSpan != nil {
shadowTrTyp, _ = s.ot.shadowTr.Type()
shadowCtx = s.ot.shadowSpan.Context()
}

var recordingType RecordingType
if s.crdb.isRecording() {
recordingType = s.crdb.mu.recording.recordingType
}

return &SpanMeta{
traceID: s.crdb.traceID,
spanID: s.crdb.spanID,
traceID: traceID,
spanID: spanID,
shadowTracerType: shadowTrTyp,
shadowCtx: shadowCtx,
recordingType: recordingType,
Baggage: baggageCopy,
Baggage: baggage,
}
}

Expand Down Expand Up @@ -624,11 +629,9 @@ func (s *crdbSpan) getRecordingLocked() tracingpb.RecordedSpan {
}
}
if s.logTags != nil {
tags := s.logTags.Get()
for i := range tags {
tag := &tags[i]
addTag(tagName(tag.Key()), tag.ValueStr())
}
setLogTags(s.logTags.Get(), func(remappedKey string, tag *logtags.Tag) {
addTag(remappedKey, tag.ValueStr())
})
}
if len(s.mu.tags) > 0 {
for k, v := range s.mu.tags {
Expand Down
6 changes: 3 additions & 3 deletions pkg/util/tracing/span_options.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,7 @@ type spanOptions struct {
}

func (opts *spanOptions) parentTraceID() uint64 {
if opts.Parent != nil {
if opts.Parent != nil && !opts.Parent.isNoop() {
return opts.Parent.crdb.traceID
} else if opts.RemoteParent != nil {
return opts.RemoteParent.traceID
Expand All @@ -39,7 +39,7 @@ func (opts *spanOptions) parentTraceID() uint64 {
}

func (opts *spanOptions) parentSpanID() uint64 {
if opts.Parent != nil {
if opts.Parent != nil && !opts.Parent.isNoop() {
return opts.Parent.crdb.spanID
} else if opts.RemoteParent != nil {
return opts.RemoteParent.spanID
Expand All @@ -49,7 +49,7 @@ func (opts *spanOptions) parentSpanID() uint64 {

func (opts *spanOptions) recordingType() RecordingType {
var recordingType RecordingType
if opts.Parent != nil {
if opts.Parent != nil && !opts.Parent.isNoop() {
recordingType = opts.Parent.crdb.getRecordingType()
} else if opts.RemoteParent != nil {
recordingType = opts.RemoteParent.recordingType
Expand Down
17 changes: 13 additions & 4 deletions pkg/util/tracing/tags.go
Original file line number Diff line number Diff line change
Expand Up @@ -47,9 +47,18 @@ func RegisterTagRemapping(logTag, spanTag string) {
tagRemap[logTag] = spanTag
}

func tagName(logTag string) string {
if v, ok := tagRemap[logTag]; ok {
return v
// setLogTags calls the provided function for each tag pair from the provided log tags.
// It takes into account any prior calls to RegisterTagRemapping.
func setLogTags(logTags []logtags.Tag, setTag func(remappedKey string, value *logtags.Tag)) {
tagName := func(logTag string) string {
if v, ok := tagRemap[logTag]; ok {
return v
}
return logTag
}

for i := range logTags {
tag := &logTags[i]
setTag(tagName(tag.Key()), tag)
}
return logTag
}
Loading

0 comments on commit b03fb74

Please sign in to comment.