Skip to content

Commit

Permalink
Merge #58516
Browse files Browse the repository at this point in the history
58516: tracing: add package-level documentation for tracing concepts r=irfansharif a=irfansharif

First two commits are from #58509, and can be ignored here.

Release note: None

Co-authored-by: irfan sharif <irfanmahmoudsharif@gmail.com>
  • Loading branch information
craig[bot] and irfansharif committed Jan 8, 2021
2 parents 5931b04 + e9e1f2b commit 7826689
Show file tree
Hide file tree
Showing 9 changed files with 146 additions and 48 deletions.
1 change: 1 addition & 0 deletions pkg/util/tracing/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -5,6 +5,7 @@ go_library(
srcs = [
"context.go",
"crdbspan.go",
"doc.go",
"grpc_interceptor.go",
"otspan.go",
"recording.go",
Expand Down
14 changes: 7 additions & 7 deletions pkg/util/tracing/crdbspan.go
Original file line number Diff line number Diff line change
Expand Up @@ -80,7 +80,7 @@ type crdbSpanMu struct {
structured []Structured

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

func (s *crdbSpan) recordingType() RecordingType {
Expand Down Expand Up @@ -220,14 +220,14 @@ func (s *crdbSpan) setBaggageItemAndTag(restrictedKey, value string) {
}

func (s *crdbSpan) setBaggageItemLocked(restrictedKey, value string) {
if oldVal, ok := s.mu.Baggage[restrictedKey]; ok && oldVal == value {
if oldVal, ok := s.mu.baggage[restrictedKey]; ok && oldVal == value {
// No-op.
return
}
if s.mu.Baggage == nil {
s.mu.Baggage = make(map[string]string)
if s.mu.baggage == nil {
s.mu.baggage = make(map[string]string)
}
s.mu.Baggage[restrictedKey] = value
s.mu.baggage[restrictedKey] = value
}

// getRecordingLocked returns the Span's recording. This does not include
Expand Down Expand Up @@ -286,9 +286,9 @@ func (s *crdbSpan) getRecordingLocked(m mode) tracingpb.RecordedSpan {
}
}

if len(s.mu.Baggage) > 0 {
if len(s.mu.baggage) > 0 {
rs.Baggage = make(map[string]string)
for k, v := range s.mu.Baggage {
for k, v := range s.mu.baggage {
rs.Baggage[k] = v
}
}
Expand Down
106 changes: 106 additions & 0 deletions pkg/util/tracing/doc.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,106 @@
// Copyright 2021 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

// Package tracing encapsulates all tracing facilities used in CockroachDB.
// Tracing exposes a lifetime-of-a-request view of operations, tracking it
// through various internal components and across RPC boundaries. The concepts
// and primitives used in this package are standard for most distributed
// tracing libraries[1][2], but we'll capture it all here anyway.
//
// 1. The Data Model
//
// [Span A] <--- (root span)
// |
// +------+------+
// | |
// [Span B] [Span C] <--- (C is a "child of" A)
// | |
// [Span D] +---+-------+
// | |
// [Span E] [Span F] >>> [Span G] <--- (G "follows from" F)
//
//
// Traces are defined implicitly by their Spans. A Trace can be thought of a
// directed acyclic graph of Spans, where edges between Spans indicate that
// they're causally related. An alternate (and usually the more useful)
// rendering[3] of traces is a temporal one:
//
//
// ––|–––––––|–––––––|–––––––|–––––––|–––––––|–––––––|–––––––|–> time
//
// [Span A···················································]
// [Span B··············································]
// [Span D··········································]
// [Span C········································]
// [Span E·······] [Span F··] [Span G··]
//
//
// The causal relation between spans can be one of two types:
// - Parent-child relation: Typically used when the parent span depends on the
// result of the child span (during an RPC call, the
// client-side span would be the parent of the
// server-side span). See [4].
// - Follows-from relation: Typically used when the first span does not in any
// way depend on the result of the second (think of a
// goroutine that spins off another that outlives it).
// Note that we still refer to the "first" and "second"
// span as being the "parent" and "child" respectively
// (they're still nodes in the DAG, just with a
// different kind of edge between them)[5].
//
// Each Span[6] is logically comprised of the following:
// - An operation name
// - Timing information (start timestamp, duration)
// - A set of zero or more tags (for annotation, visible when rendering spans)
// - A set of zero or more baggage items (data that crosses process boundaries)
// - References to other spans (mediated by the relations described above)
// - Recording data[7] (structured data/messages visible when rendering spans)
//
// Spans are created through a Tracer. Related, Tracers also understand how to
// serialize and deserialize[8] Spans across process boundaries (using only the
// Span metadata[9]). We've defined handy GRPC interceptors[10] that let us do
// this across RPC boundaries.
//
// The tracing package is tightly coupled with the context package. Since we
// want the tracing infrastructure to be plumbed through the various layers in
// the callstack, we tuck the Span object within a context.Context[11].
//
// Since this package is used pervasively, the implementation is very
// performance-sensitive. It tries to avoid allocations (even
// trying to avoid allocating Span objects[12] whenever possible), and avoids
// doing work unless strictly necessary. One example of this is us checking to
// see if a given Span is a "noop span"[13] (i.e. does not have any sinks
// configured). This then lets us short-circuit work that would be discarded
// anyway.
//
// The tracing package internally makes use of an opentracing[2]-compatible
// library. This gives us the ability to configure external collectors for
// tracing information[14], like lightstep or zipkin.
//
// -----------------------------------------------------------------------------
//
// [1]: https://research.google/pubs/pub36356/
// [2]: https://opentracing.io/specification/
// [3]: `Recording.String`
// [4]: `ChildSpan`
// [5]: `ForkCtxSpan`. "forking" a Span is the same as creating a new one
// with a "follows from" relation.
// [6]: `crdbSpan`
// [7]: `Span.SetVerbose`. To understand the specifics of what exactly is
// captured in Span recording, when Spans have children that may be either
// local or remote, look towards `WithParentAnd{Auto,Manual}Collection`
// [8]: `Tracer.{Inject,Extract}`
// [9]: `SpanMeta`
// [10]: `{Client,Server}Interceptor`
// [11]: `SpanFromContext`
// [12]: WithForceRealSpan
// [13]: `Span.isNoop`
// [14]: `shadowTracer`
package tracing
4 changes: 2 additions & 2 deletions pkg/util/tracing/grpc_interceptor.go
Original file line number Diff line number Diff line change
Expand Up @@ -237,7 +237,7 @@ func injectSpanContext(ctx context.Context, tracer *Tracer, clientSpan *Span) co
//
// All gRPC client spans will inject the tracing SpanMeta into the gRPC
// metadata; they will also look in the context.Context for an active
// in-process parent Span and establish a ChildOf reference if such a parent
// in-process parent Span and establish a ChildOf relationship if such a parent
// Span could be found.
func ClientInterceptor(tracer *Tracer, init func(*Span)) grpc.UnaryClientInterceptor {
if init == nil {
Expand Down Expand Up @@ -285,7 +285,7 @@ func ClientInterceptor(tracer *Tracer, init func(*Span)) grpc.UnaryClientInterce
//
// All gRPC client spans will inject the tracing SpanMeta into the gRPC
// metadata; they will also look in the context.Context for an active
// in-process parent Span and establish a ChildOf reference if such a parent
// in-process parent Span and establish a ChildOf relationship if such a parent
// Span could be found.
func StreamClientInterceptor(tracer *Tracer, init func(*Span)) grpc.StreamClientInterceptor {
if init == nil {
Expand Down
11 changes: 5 additions & 6 deletions pkg/util/tracing/shadow.go
Original file line number Diff line number Diff line change
Expand Up @@ -7,12 +7,6 @@
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
//
// A "shadow" tracer can be any opentracing.Tracer implementation that is used
// in addition to the normal functionality of our tracer. It works by attaching
// a shadow Span to every Span, and attaching a shadow context to every Span
// context. When injecting a Span context, we encapsulate the shadow context
// inside ours.

package tracing

Expand Down Expand Up @@ -56,6 +50,11 @@ func (m *zipkinManager) Close(tr opentracing.Tracer) {
_ = m.collector.Close()
}

// A shadowTracer can be any opentracing.Tracer implementation that is used in
// addition to the normal functionality of our tracer. It works by attaching a
// shadow Span to every Span, and attaching a shadow context to every Span
// context. When injecting a Span context, we encapsulate the shadow context
// inside ours.
type shadowTracer struct {
opentracing.Tracer
manager shadowTracerManager
Expand Down
7 changes: 4 additions & 3 deletions pkg/util/tracing/span.go
Original file line number Diff line number Diff line change
Expand Up @@ -58,7 +58,8 @@ type Span struct {
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.
// External opentracing compatible tracer such as lightstep, zipkin, jaeger;
// zero if not using one.
ot otSpan
}

Expand Down Expand Up @@ -240,12 +241,12 @@ func (s *Span) Meta() *SpanMeta {
traceID, spanID = s.crdb.traceID, s.crdb.spanID
s.crdb.mu.Lock()
defer s.crdb.mu.Unlock()
n := len(s.crdb.mu.Baggage)
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 {
for k, v := range s.crdb.mu.baggage {
baggage[k] = v
}
recordingType = s.crdb.mu.recording.recordingType.load()
Expand Down
11 changes: 1 addition & 10 deletions pkg/util/tracing/tracer.go
Original file line number Diff line number Diff line change
Expand Up @@ -421,7 +421,7 @@ func (t *Tracer) startSpanGeneric(
if opts.Parent != nil {
if !opts.Parent.isNoop() {
opts.Parent.crdb.mu.Lock()
m := opts.Parent.crdb.mu.Baggage
m := opts.Parent.crdb.mu.baggage
for k, v := range m {
s.SetBaggageItem(k, v)
}
Expand Down Expand Up @@ -498,15 +498,6 @@ func (t *Tracer) Inject(sc *SpanMeta, format interface{}, carrier interface{}) e
return nil
}

type textMapReaderFn func(handler func(key, val string) error) error

var _ opentracing.TextMapReader = textMapReaderFn(nil)

// ForeachKey is part of the opentracing.TextMapReader interface.
func (fn textMapReaderFn) ForeachKey(handler func(key, val string) error) error {
return fn(handler)
}

var noopSpanContext = &SpanMeta{}

// Extract is part of the opentracing.Tracer interface.
Expand Down
26 changes: 13 additions & 13 deletions pkg/util/tracing/tracingpb/recorded_span.pb.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

14 changes: 7 additions & 7 deletions pkg/util/tracing/tracingpb/recorded_span.proto
Original file line number Diff line number Diff line change
Expand Up @@ -41,7 +41,7 @@ message RecordedSpan {
uint64 trace_id = 1 [(gogoproto.customname) = "TraceID"];
// ID of the span.
uint64 span_id = 2 [(gogoproto.customname) = "SpanID"];
// Span ID of the parent span.
// ID of the parent span.
uint64 parent_span_id = 3 [(gogoproto.customname) = "ParentSpanID"];
// Operation name.
string operation = 4;
Expand All @@ -53,11 +53,11 @@ message RecordedSpan {
// Time when the span was started.
google.protobuf.Timestamp start_time = 7 [(gogoproto.nullable) = false,
(gogoproto.stdtime) = true];
// Duration is the span's duration, measured from start to Finish().
// The span's duration, measured from start to Finish().
//
// A spans whose recording is collected before it's finished will have the
// duration set as the time of collection - start_time. Such a span will have
// an "unfinished" tag.
// A span whose recording is collected before it's finished will have the
// duration set as the "time of collection - start time". Such a span will
// have an "unfinished" tag.
google.protobuf.Duration duration = 8 [(gogoproto.nullable) = false,
(gogoproto.stdduration) = true];

Expand All @@ -66,8 +66,8 @@ message RecordedSpan {

// Stats collected in this span.
//
// Deprecated - remove in 21.2 cycle. Stats will use the structured field
// below; 21.1 nodes already know to look there.
// TODO(tbg): remove in 21.2 cycle. Stats will use the structured field below;
// 21.1 nodes already know to look there.
google.protobuf.Any deprecated_stats = 10;

// InternalStructured are payloads attached to this Span.
Expand Down

0 comments on commit 7826689

Please sign in to comment.