diff --git a/pkg/util/tracing/shadow.go b/pkg/util/tracing/shadow.go index 64047e019a8f..de2567bdd88e 100644 --- a/pkg/util/tracing/shadow.go +++ b/pkg/util/tracing/shadow.go @@ -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" @@ -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) { diff --git a/pkg/util/tracing/span.go b/pkg/util/tracing/span.go index d47f5ceb181c..dd5bfe01c8d2 100644 --- a/pkg/util/tracing/span.go +++ b/pkg/util/tracing/span.go @@ -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 @@ -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 { @@ -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. @@ -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. @@ -405,22 +406,31 @@ 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 { @@ -428,18 +438,13 @@ func (s *Span) Meta() *SpanMeta { 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, } } @@ -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 { diff --git a/pkg/util/tracing/span_options.go b/pkg/util/tracing/span_options.go index c285b8d26c37..9311a80a8ffe 100644 --- a/pkg/util/tracing/span_options.go +++ b/pkg/util/tracing/span_options.go @@ -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 @@ -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 @@ -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 diff --git a/pkg/util/tracing/tags.go b/pkg/util/tracing/tags.go index 76d4e403f01f..1a425207acac 100644 --- a/pkg/util/tracing/tags.go +++ b/pkg/util/tracing/tags.go @@ -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 } diff --git a/pkg/util/tracing/tracer.go b/pkg/util/tracing/tracer.go index 3d1a5e75e9d6..676143f06741 100644 --- a/pkg/util/tracing/tracer.go +++ b/pkg/util/tracing/tracer.go @@ -215,49 +215,103 @@ func (t *Tracer) startSpanGeneric(opName string, opts spanOptions) *Span { return t.noopSpan } - s := &Span{ - tracer: t, - crdb: crdbSpan{ - operation: opName, - startTime: time.Now(), - parentSpanID: opts.parentSpanID(), - logTags: opts.LogTags, - }, + if opts.LogTags == nil && opts.Parent != nil && !opts.Parent.isNoop() { + // If no log tags are specified in the options, use the parent + // span's, if any. This behavior is the reason logTags are + // fundamentally different from tags, which are strictly per span, + // for better or worse. + opts.LogTags = opts.Parent.crdb.logTags } - s.crdb.mu.duration = -1 // unfinished - traceID := opts.parentTraceID() - if traceID == 0 { - traceID = uint64(rand.Int63()) - } - s.crdb.traceID = traceID - s.crdb.spanID = uint64(rand.Int63()) + startTime := time.Now() - shadowTr := t.getShadowTracer() + // First, create any external spans that we may need (opentracing, net/trace). + // We do this early so that they are available when we construct the main Span, + // which makes it easier to avoid one-offs when populating the tags and baggage + // items for the top-level Span. + var ot otSpan { + shadowTr := t.getShadowTracer() + // Make sure not to derive spans created using an old // shadow tracer via a new one. typ1, ok1 := opts.shadowTrTyp() // old typ2, ok2 := shadowTr.Type() // new - if ok1 && ok2 && typ1 != typ2 { - // If both are set and don't agree, ignore shadow tracer - // for the new span. It's fine if the old one isn't set - // but the new one is (we won't use it, though we could) - // or if the old one is and new one isn't (in which case - // the supposedly latest config has no shadow tracing - // enabled). - shadowTr = nil + // If both are set and don't agree, ignore shadow tracer + // for the new span to avoid compat issues between the + // two underlying tracers. + if ok2 && (!ok1 || typ1 == typ2) { + var shadowCtx opentracing.SpanContext + if opts.Parent != nil && opts.Parent.ot.shadowSpan != nil { + shadowCtx = opts.Parent.ot.shadowSpan.Context() + } + ot = makeShadowSpan(shadowTr, shadowCtx, opts.RefType, opName, startTime) + // If LogTags are given, pass them as tags to the shadow span. + // Regular tags are populated later, via the top-level Span. + if opts.LogTags != nil { + setLogTags(opts.LogTags.Get(), func(remappedKey string, tag *logtags.Tag) { + _ = ot.shadowSpan.SetTag(remappedKey, tag.Value()) + }) + } } } - if shadowTr != nil { - var shadowCtx opentracing.SpanContext - if opts.Parent != nil && opts.Parent.ot.shadowSpan != nil { - shadowCtx = opts.Parent.ot.shadowSpan.Context() + var netTr trace.Trace + if t.useNetTrace() { + netTr = trace.New("tracing", opName) + netTr.SetMaxEvents(maxLogsPerSpan) + + // If LogTags are given, pass them as tags to the shadow span. + // Regular tags are populated later, via the top-level Span. + if opts.LogTags != nil { + setLogTags(opts.LogTags.Get(), func(remappedKey string, tag *logtags.Tag) { + netTr.LazyPrintf("%s:%v", remappedKey, tag) + }) } - linkShadowSpan(s, shadowTr, shadowCtx, opts.RefType) } + // Now that `ot` and `netTr` are properly set up, make the Span. + + traceID := opts.parentTraceID() + if traceID == 0 { + // NB: it is tempting to use the traceID and spanID from the + // possibly populated otSpan in this case, but the opentracing + // interface doesn't give us a good way to extract these. + traceID = uint64(rand.Int63()) + } + spanID := uint64(rand.Int63()) + + // Now allocate the main *Span and contained crdbSpan. + // Allocate these together to save on individual allocs. + // + // NB: at the time of writing, it's not possible to start a Span + // that *only* contains `ot` or `netTr`. This is just an artifact + // of the history of this code and may change in the future. + helper := struct { + Span Span + crdbSpan crdbSpan + }{} + + helper.crdbSpan = crdbSpan{ + traceID: traceID, + spanID: spanID, + operation: opName, + startTime: startTime, + parentSpanID: opts.parentSpanID(), + logTags: opts.LogTags, + mu: crdbSpanMu{ + duration: -1, // unfinished + }, + } + helper.Span = Span{ + tracer: t, + crdb: &helper.crdbSpan, + ot: ot, + netTr: netTr, + } + + s := &helper.Span + // Start recording if necessary. We inherit the recording type of the local parent, if any, // over the remote parent, if any. If neither are specified, we're not recording. recordingType := opts.recordingType() @@ -265,34 +319,24 @@ func (t *Tracer) startSpanGeneric(opName string, opts spanOptions) *Span { if recordingType != NoRecording { var p *crdbSpan if opts.Parent != nil { - p = &opts.Parent.crdb + p = opts.Parent.crdb } s.crdb.enableRecording(p, recordingType, opts.SeparateRecording) } - if t.useNetTrace() { - s.netTr = trace.New("tracing", opName) - s.netTr.SetMaxEvents(maxLogsPerSpan) - if opts.LogTags != nil { - tags := opts.LogTags.Get() - for i := range tags { - tag := &tags[i] - s.netTr.LazyPrintf("%s:%v", tagName(tag.Key()), tag.Value()) - } - } - } - - // Set initial tags. + // Set initial tags. These will propagate to the crdbSpan, ot, and netTr + // as appropriate. // // NB: this could be optimized. for k, v := range opts.Tags { s.SetTag(k, v) } - // Copy baggage from parent. + // Copy baggage from parent. This similarly fans out over the various + // spans contained in Span. // // NB: this could be optimized. - if opts.Parent != nil { + if opts.Parent != nil && !opts.Parent.isNoop() { opts.Parent.crdb.mu.Lock() m := opts.Parent.crdb.mu.Baggage for k, v := range m {