Documentation
¶
Overview ¶
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) - 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.
The tracing package internally makes use of OpenTelemetry[2]. This gives us the ability to configure external collectors for tracing information, like Jaeger, DataDog, Lightstep or Zipkin.
-----------------------------------------------------------------------------
[1]: https://research.google/pubs/pub36356/ [2]: https://opentelemetry.io [3]: `Recording.String` [4]: `ChildSpan` [5]: `ForkSpan`. "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 `WithParent` and `WithDetachedRecording`.
[8]: `Tracer.{InjectMetaInto,ExtractMetaFrom}` [9]: `SpanMeta` [10]: `{Client,Server}Interceptor` [11]: `SpanFromContext` [12]: WithForceRealSpan
Index ¶
- Constants
- Variables
- func AggregatorEventToBytes(_ context.Context, event AggregatorEvent) ([]byte, error)
- func CheckRecordedSpans(rec tracingpb.Recording, expected string) error
- func ContextWithRecordingSpan(ctx context.Context, tr *Tracer, opName string) (_ context.Context, finishAndGetRecording func() tracingpb.Recording)
- func ContextWithSpan(ctx context.Context, sp *Span) context.Context
- func CountLogMessages(sp tracingpb.RecordedSpan, msg string) int
- func FindMsgInRecording(recording tracingpb.Recording, msg string) int
- func LogsContainMsg(sp tracingpb.RecordedSpan, msg string) bool
- func RedactAndTruncateError(err error) string
- func RegisterTagRemapping(logTag, spanTag string)
- func SpanInclusionFuncForClient(parent *Span) bool
- func SpanInclusionFuncForServer(t *Tracer, spanMeta SpanMeta) bool
- func TraceToJSON(trace tracingpb.Recording) (string, error)
- type AggregatorEvent
- type Carrier
- type EventConsumptionStatus
- type EventListener
- type LazyTag
- type MapCarrier
- type MetadataCarrier
- type RegistrySpan
- type SnapshotID
- type SnapshotInfo
- type Span
- func ChildSpan(ctx context.Context, opName string, os ...SpanOption) (context.Context, *Span)
- func EnsureChildSpan(ctx context.Context, tr *Tracer, name string, os ...SpanOption) (context.Context, *Span)
- func EnsureForkSpan(ctx context.Context, tr *Tracer, opName string) (context.Context, *Span)
- func ForkSpan(ctx context.Context, opName string) (context.Context, *Span)
- func SpanFromContext(ctx context.Context) *Span
- func (sp *Span) Finish()
- func (sp *Span) FinishAndGetConfiguredRecording() tracingpb.Recording
- func (sp *Span) FinishAndGetRecording(recType tracingpb.RecordingType) tracingpb.Recording
- func (sp *Span) FinishAndGetTraceRecording(recType tracingpb.RecordingType) Trace
- func (sp *Span) GetConfiguredRecording() tracingpb.Recording
- func (sp *Span) GetFullTraceRecording(recType tracingpb.RecordingType) Trace
- func (sp *Span) GetLazyTag(key string) (interface{}, bool)
- func (sp *Span) GetRecording(recType tracingpb.RecordingType) tracingpb.Recording
- func (sp *Span) GetTraceRecording(recType tracingpb.RecordingType) Trace
- func (sp *Span) ImportRemoteRecording(remoteRecording tracingpb.Recording)
- func (sp *Span) ImportTrace(trace Trace)
- func (sp *Span) IsSterile() bool
- func (sp *Span) IsVerbose() bool
- func (sp *Span) MaybeRecordStackHistory(since time.Time)
- func (sp *Span) Meta() SpanMeta
- func (sp *Span) OperationName() string
- func (sp *Span) Record(msg string)
- func (sp *Span) RecordStructured(item Structured)
- func (sp *Span) Recordf(format string, args ...interface{})
- func (sp *Span) RecordingType() tracingpb.RecordingType
- func (sp *Span) Redactable() bool
- func (sp *Span) SetLazyTag(key string, value interface{})
- func (sp *Span) SetLazyTagLocked(key string, value interface{})deprecated
- func (sp *Span) SetOtelStatus(code codes.Code, msg string)
- func (sp *Span) SetRecordingType(to tracingpb.RecordingType)
- func (sp *Span) SetTag(key string, value attribute.Value)
- func (sp *Span) SpanID() tracingpb.SpanID
- func (sp *Span) String() string
- func (sp *Span) TraceID() tracingpb.TraceID
- func (sp *Span) Tracer() *Tracer
- func (sp *Span) UpdateGoroutineIDToCurrent()
- type SpanMeta
- type SpanOption
- func WithDetachedRecording() SpanOption
- func WithEventListeners(eventListeners ...EventListener) SpanOption
- func WithFollowsFrom() SpanOption
- func WithForceRealSpan() SpanOption
- func WithLogTags(tags *logtags.Buffer) SpanOption
- func WithParent(sp *Span) SpanOption
- func WithRecording(recType tracingpb.RecordingType) SpanOption
- func WithRemoteParentFromLocalSpan(sp *Span) SpanOption
- func WithRemoteParentFromSpanMeta(parent SpanMeta) SpanOption
- func WithRemoteParentFromTraceInfo(ti *tracingpb.TraceInfo) SpanOption
- func WithSpanKind(kind oteltrace.SpanKind) SpanOption
- func WithSterile() SpanOption
- type SpanRegistry
- type SpanReusePercentOpt
- type SpansSnapshot
- type Structured
- type T
- type Trace
- type Tracer
- func (t *Tracer) ActiveSpansRegistryEnabled() bool
- func (t *Tracer) AlwaysTrace() bool
- func (t *Tracer) Close()
- func (t *Tracer) ExtractMetaFrom(carrier Carrier) (SpanMeta, error)
- func (t *Tracer) GetActiveSpanByID(spanID tracingpb.SpanID) RegistrySpan
- func (t *Tracer) GetActiveSpansRegistry() *SpanRegistry
- func (t *Tracer) GetAutomaticSnapshot(id SnapshotID) (SpansSnapshot, error)
- func (t *Tracer) GetAutomaticSnapshots() []SnapshotInfo
- func (t *Tracer) GetSnapshot(id SnapshotID) (SpansSnapshot, error)
- func (t *Tracer) GetSnapshots() []SnapshotInfo
- func (t *Tracer) HasExternalSink() bool
- func (t *Tracer) InjectMetaInto(sm SpanMeta, carrier Carrier)
- func (t *Tracer) PanicOnUseAfterFinish() bool
- func (t *Tracer) PeriodicSnapshotsLoop(sv *settings.Values, done <-chan struct{})
- func (t *Tracer) Redactable() bool
- func (t *Tracer) SaveAutomaticSnapshot() SnapshotInfo
- func (t *Tracer) SaveSnapshot() SnapshotInfo
- func (t *Tracer) SetActiveSpansRegistryEnabled(to bool)
- func (t *Tracer) SetOpenTelemetryTracer(tr oteltrace.Tracer)
- func (t *Tracer) SetRedactable(to bool)
- func (t *Tracer) ShouldRecordAsyncSpans() bool
- func (t *Tracer) SpanRegistry() *SpanRegistry
- func (t *Tracer) StartSpan(operationName string, os ...SpanOption) *Span
- func (t *Tracer) StartSpanCtx(ctx context.Context, operationName string, os ...SpanOption) (context.Context, *Span)
- func (t *Tracer) TestingGetStatsAndReset() (int, int)
- func (t *Tracer) TestingRecordAsyncSpans()
- func (t *Tracer) VisitSpans(visitor func(span RegistrySpan) error) error
- type TracerOption
- func WithClusterSettings(sv *settings.Values) TracerOption
- func WithSpanReusePercent(percent uint32) TracerOption
- func WithTestingKnobs(knobs TracerTestingKnobs) TracerOption
- func WithTracingMode(opt TracingMode) TracerOption
- func WithUseAfterFinishOpt(panicOnUseAfterFinish, debugUseAfterFinish bool) TracerOption
- type TracerTestingKnobs
- type TracingAggregator
- type TracingMode
Constants ¶
const (
SpanKindTagKey = "span.kind"
)
These constants are used to form keys to represent tracing context information in "carriers" to be transported across RPC boundaries.
Variables ¶
var EnableActiveSpansRegistry = settings.RegisterBoolSetting( settings.ApplicationLevel, "trace.span_registry.enabled", "if set, ongoing traces can be seen at https://<ui>/#/debug/tracez", envutil.EnvOrDefaultBool("COCKROACH_REAL_SPANS", false), settings.WithPublic)
EnableActiveSpansRegistry controls Tracers configured as WithTracingMode(TracingModeFromEnv) (which is the default). When enabled, spans are allocated and registered with the active spans registry until finished. When disabled, span creation is short-circuited for a small performance improvement.
var WithClientSpanKind = WithSpanKind(oteltrace.SpanKindClient)
WithClientSpanKind is a shorthand for server spans, frequently saving allocations.
var WithServerSpanKind = WithSpanKind(oteltrace.SpanKindServer)
WithServerSpanKind is a shorthand for server spans, frequently saving allocations.
var ZipkinCollector = settings.RegisterStringSetting( settings.ApplicationLevel, "trace.zipkin.collector", "the address of a Zipkin instance to receive traces, as <host>:<port>. "+ "If no port is specified, 9411 will be used.", envutil.EnvOrDefaultString("COCKROACH_ZIPKIN", ""), settings.WithValidateString(func(_ *settings.Values, s string) error { if s == "" { return nil } _, _, err := addr.SplitHostPort(s, "9411") return err }), settings.WithPublic, )
ZipkinCollector is the cluster setting that specifies the Zipkin instance to send traces to, if any.
Functions ¶
func AggregatorEventToBytes ¶
func AggregatorEventToBytes(_ context.Context, event AggregatorEvent) ([]byte, error)
AggregatorEventToBytes marshals an event into a byte slice.
func CheckRecordedSpans ¶
CheckRecordedSpans checks whether a recording looks like an expected one represented by a string with one line per expected span and one line per expected event (i.e. log message), with a tab-indentation for child spans.
if err := CheckRecordedSpans(Span.GetRecording(), ` span: root event: a span: child event: [ambient] b event: c `); err != nil { t.Fatal(err) }
The event lines can (and generally should) omit the file:line part that they might contain (depending on the level at which they were logged).
Note: this test function is in this file because it needs to be used by both tests in the tracing package and tests outside of it, and the function itself depends on tracing.
func ContextWithRecordingSpan ¶
func ContextWithRecordingSpan( ctx context.Context, tr *Tracer, opName string, ) (_ context.Context, finishAndGetRecording func() tracingpb.Recording)
ContextWithRecordingSpan returns a context with an embedded trace Span. The Span is derived from the provided Tracer. The recording is collected and the span is Finish()ed through the returned callback.
The returned callback can be called multiple times.
Note that to convert the recorded spans into text, you can use Recording.String(). Tests can also use FindMsgInRecording().
func ContextWithSpan ¶
ContextWithSpan returns a Context wrapping the supplied Span.
func CountLogMessages ¶
func CountLogMessages(sp tracingpb.RecordedSpan, msg string) int
CountLogMessages counts the messages containing msg.
func FindMsgInRecording ¶
FindMsgInRecording returns the index of the first Span containing msg in its logs, or -1 if no Span is found.
func LogsContainMsg ¶
func LogsContainMsg(sp tracingpb.RecordedSpan, msg string) bool
LogsContainMsg returns true if a Span's logs contain the given message.
func RedactAndTruncateError ¶
RedactAndTruncateError redacts the error and truncates the string representation of the error to a fixed length.
func RegisterTagRemapping ¶
func RegisterTagRemapping(logTag, spanTag string)
RegisterTagRemapping sets the Span tag name that corresponds to the given log tag name. Should be called as part of an init() function.
func SpanInclusionFuncForClient ¶
SpanInclusionFuncForClient is used as a SpanInclusionFunc for the client-side of RPCs, deciding for which operations the gRPC tracing interceptor should create a span.
We use this to circumvent the interceptor's work when tracing is disabled. Otherwise, the interceptor causes an increase in the number of packets (even with an Empty context!).
See #17177.
func SpanInclusionFuncForServer ¶
SpanInclusionFuncForServer is used as a SpanInclusionFunc for the server-side of RPCs, deciding for which operations the gRPC tracing interceptor should create a span.
Types ¶
type AggregatorEvent ¶
type AggregatorEvent interface { // Identity returns a AggregatorEvent that when combined with another // event returns the other AggregatorEvent unchanged. Identity() AggregatorEvent // Combine combines two AggregatorEvents together. Combine(other AggregatorEvent) // ProtoName returns the fully qualified name of the underlying proto that is // a AggregatorEvent. ProtoName() string // String returns the string representation of the AggregatorEvent. String() string }
AggregatorEvent describes an event that can be aggregated and stored by the TracingAggregator. A AggregatorEvent also implements the tracing.LazyTag interface to render its information on the associated tracing span.
type Carrier ¶
Carrier is what's used to capture the serialized data. Each carrier is inextricably linked to a corresponding format. See serializationFormat for more details.
type EventConsumptionStatus ¶
type EventConsumptionStatus int
EventConsumptionStatus describes whether the structured event has been "consumed" by the EventListener.
const ( // EventNotConsumed indicates that the event wasn't "consumed" by the // EventListener, so other listeners as well as any ancestor spans should be // notified about the event. EventNotConsumed EventConsumptionStatus = iota // EventConsumed indicates that the event has been "consumed" by the // EventListener, so neither other listeners for the span nor the ancestor // spans should be notified about it. EventConsumed )
type EventListener ¶
type EventListener interface { // Notify is invoked on every Structured event recorded by the span and its // children, recursively. // // The caller holds the mutex of the span. // // Notify will not be called concurrently on the same span. Notify(event Structured) EventConsumptionStatus }
EventListener is an object that can be registered to listen for Structured events recorded by the span and its children.
type LazyTag ¶
type LazyTag interface { // Render produces the list of key-value tags for the span's recording. Render() []attribute.KeyValue }
LazyTag is a tag value that expands into a series of key-value tags when included in a recording. LazyTags can be used when it's useful to maintain a single, structured tag in a Span (for example because the tag is accessed through Span.GetLazyTag(key)), but that tag should render as multiple key-value pairs when the Span's recording is collected. Recordings can only contain string tags, for simplicity.
type MapCarrier ¶
MapCarrier is an implementation of the Carrier interface for a map of string pairs.
func (MapCarrier) ForEach ¶
func (c MapCarrier) ForEach(fn func(key, val string) error) error
ForEach implements the Carrier interface.
func (MapCarrier) Set ¶
func (c MapCarrier) Set(key, val string)
Set implements the Carrier interface.
type MetadataCarrier ¶
MetadataCarrier is an implementation of the Carrier interface for gRPC metadata.
func (MetadataCarrier) ForEach ¶
func (w MetadataCarrier) ForEach(fn func(key, val string) error) error
ForEach implements the Carrier interface.
func (MetadataCarrier) Set ¶
func (w MetadataCarrier) Set(key, val string)
Set implements the Carrier interface.
type RegistrySpan ¶
type RegistrySpan interface { // TraceID returns the id of the trace that this span is part of. TraceID() tracingpb.TraceID // SpanID returns the span's ID. SpanID() tracingpb.SpanID // GetFullRecording returns the recording of the trace rooted at this span. // // This includes the recording of child spans created with the // WithDetachedRecording option. In other situations, the recording of such // children is not included in the parent's recording but, in the case of the // span registry, we want as much information as possible to be included. GetFullRecording(recType tracingpb.RecordingType) Trace // SetRecordingType sets the recording mode of the span and its children, // recursively. Setting it to RecordingOff disables further recording. // Everything recorded so far remains in memory. SetRecordingType(to tracingpb.RecordingType) // RecordingType returns the span's current recording type. RecordingType() tracingpb.RecordingType }
RegistrySpan is the interface used by clients of the active span registry.
Note that RegistrySpans can be Finish()ed concurrently with their use, so all methods must work with concurrent Finish() calls.
type SnapshotID ¶
type SnapshotID int
SnapshotID identifies a spans snapshot. The ID can be used to retrieve a specific snapshot from the Tracer.
type SnapshotInfo ¶
type SnapshotInfo struct { ID SnapshotID CapturedAt time.Time }
SnapshotInfo represents minimal info about a stored snapshot, as returned by Tracer.GetSnapshots().
type Span ¶
type Span struct {
// contains filtered or unexported fields
}
Span is the tracing Span that we use in CockroachDB. Depending on the tracing configuration, it can hold anywhere between zero and three destinations for trace information:
1. external OpenTelemetry-compatible trace collector (Jaeger, Zipkin, Lightstep), 2. /debug/requests endpoint (net/trace package); mostly useful for local debugging 3. CRDB-internal trace span (powers SQL session tracing).
The CockroachDB-internal Span (crdbSpan) is more complex because rather than reporting to some external sink, the caller's "owner" must propagate the trace data back across process boundaries towards the root of the trace span tree; see WithParent and WithRemoteParentFromSpanMeta, respectively.
Additionally, the internal span type also supports turning on, stopping, and restarting its data collection (see Span.StartRecording), and this is used extensively in SQL session tracing.
Span is a fairly thin wrapper around spanInner, dealing with guarding against use-after-Finish and reference counting for pooling and re-allocating.
func ChildSpan ¶
ChildSpan creates a child span of the current one, if any, via the WithParent option. Recordings from child spans are automatically propagated to the parent span, and the tags are inherited from the context's log tags automatically. Also see `ForkSpan`, for the other kind of derived span relation.
A context wrapping the newly created span is returned, along with the span itself. If non-nil, the caller is responsible for eventually Finish()ing it.
func EnsureChildSpan ¶
func EnsureChildSpan( ctx context.Context, tr *Tracer, name string, os ...SpanOption, ) (context.Context, *Span)
EnsureChildSpan looks at the supplied Context. If it contains a Span, returns a child span via the WithParent option; otherwise starts a new Span. In both cases, a context wrapping the Span is returned along with the newly created Span.
The caller is responsible for closing the Span (via Span.Finish).
func EnsureForkSpan ¶
EnsureForkSpan is like ForkSpan except that, if there is no span in ctx, it creates a root span.
func ForkSpan ¶
ForkSpan forks the current span, if any[1]. Forked spans "follow from" the original, and are typically used to trace operations that may outlive the parent (think async tasks). See the package-level documentation for more details.
The recordings from these spans will not be automatically propagated to the parent span[2]. Also see `ChildSpan`, for the other kind of derived span relation.
A context wrapping the newly created span is returned, along with the span itself. If non-nil, the caller is responsible for eventually Finish()ing it.
[1]: Looking towards the provided context to see if one exists. [2]: Unless configured differently by tests, see
TestingRecordAsyncSpans.
func SpanFromContext ¶
SpanFromContext returns the *Span contained in the Context, if any.
func (*Span) Finish ¶
func (sp *Span) Finish()
Finish marks the Span as completed. It is illegal to use a Span after calling Finish().
Finishing a nil *Span is a noop.
func (*Span) FinishAndGetConfiguredRecording ¶
FinishAndGetConfiguredRecording is like FinishAndGetRecording, except that the type of recording returned is the type that the span was configured to record.
Returns nil if the span is not currently recording (even if it had been recording in the past).
func (*Span) FinishAndGetRecording ¶
func (sp *Span) FinishAndGetRecording(recType tracingpb.RecordingType) tracingpb.Recording
FinishAndGetRecording finishes the span and gets a recording at the same time. This is offered as a combined operation because, otherwise, the caller would be forced to collect the recording before finishing and so the span would appear to be unfinished in the recording (it's illegal to collect the recording after the span finishes, except by using this method).
Returns nil if the span is not currently recording (even if it had been recording in the past).
func (*Span) FinishAndGetTraceRecording ¶
func (sp *Span) FinishAndGetTraceRecording(recType tracingpb.RecordingType) Trace
FinishAndGetTraceRecording finishes the span and returns its recording as a Trace.
See also FinishAndGetRecording(), which returns a tracingpb.Recording.
func (*Span) GetConfiguredRecording ¶
GetConfiguredRecording is like GetRecording, except the type of recording it returns is the one that the span has been previously configured with.
Returns nil if the span is not currently recording (even if it had been recording in the past).
func (*Span) GetFullTraceRecording ¶
func (sp *Span) GetFullTraceRecording(recType tracingpb.RecordingType) Trace
func (*Span) GetLazyTag ¶
GetLazyTag returns the value of the tag with the given key. If that tag doesn't exist, the bool retval is false.
func (*Span) GetRecording ¶
func (sp *Span) GetRecording(recType tracingpb.RecordingType) tracingpb.Recording
GetRecording retrieves the current recording, if the Span has recording enabled. This can be called while spans that are part of the recording are still open; it can run concurrently with operations on those spans.
Returns nil if the span is not currently recording (even if it had been recording in the past).
recType indicates the type of information to be returned: structured info or structured + verbose info. The caller can ask for either regardless of the current recording mode (and also regardless of past recording modes) but, of course, GetRecording(RecordingVerbose) will not return verbose info if it was never collected.
As a performance optimization, GetRecording does not return tags when recType == RecordingStructured. Returning tags requires expensive stringification.
A few internal tags are added to denote span properties:
"_unfinished": The span was never Finish()ed. "_verbose": The span is a verbose one. "_dropped_logs": The span dropped events due to size limits. "_dropped_children": Some (direct) child spans were dropped because of the trace size limit. "_dropped_indirect_children": Some indirect child spans were dropped because of the trace size limit.
If recType is RecordingStructured, the return value will be nil if the span doesn't have any structured events.
func (*Span) GetTraceRecording ¶
func (sp *Span) GetTraceRecording(recType tracingpb.RecordingType) Trace
GetTraceRecording returns the span's recording as a Trace.
See also GetRecording(), which returns a tracingpb.Recording.
func (*Span) ImportRemoteRecording ¶
ImportRemoteRecording adds the spans in remoteRecording as children of the receiver. As a result of this, the imported recording will be a part of the GetRecording() output for the receiver. All the structured events from the trace are passed to the receiver's event listeners.
This function is used to import a recording from another node.
func (*Span) ImportTrace ¶
ImportTrace takes a trace recording and, depending on the receiver's recording mode, adds it as a child to sp. All the structured events from the trace are passed to the receiver's event listeners.
ImportTrace takes ownership of trace; the caller should not use it anymore. The caller can call Trace.PartialClone() to make a sufficient copy for passing into ImportTrace.
func (*Span) IsSterile ¶
IsSterile returns true if this span does not want to have children spans. In that case, trying to create a child span will result in the would-be child being a root span.
func (*Span) MaybeRecordStackHistory ¶
MaybeRecordStackHistory records in the span found in the passed context, if there is one and it is verbose or has a sink, any stacks found for the current goroutine in the currently stored tracer automatic snapshots, since the passed time (generally when this goroutine started processing this request/op). See the "trace.snapshot.rate" setting for controlling whether such automatic snapshots are available to be searched and if so at what granularity.
func (*Span) Meta ¶
Meta returns the information which needs to be propagated across process boundaries in order to derive child spans from this Span. This may return an Empty SpanMeta (which is a valid input to WithRemoteParentFromSpanMeta) if the Span has been optimized out.
func (*Span) OperationName ¶
OperationName returns the name of this span assigned when the span was created.
func (*Span) Record ¶
Record provides a way to record free-form text into verbose spans. Recordings may be dropped due to sizing constraints.
TODO(tbg): make sure `msg` is lint-forced to be const.
func (*Span) RecordStructured ¶
func (sp *Span) RecordStructured(item Structured)
RecordStructured adds a Structured payload to the Span. It will be added to the recording even if the Span is not verbose; however it will be discarded if the underlying Span has been optimized out (i.e. is nil). Payloads may also be dropped due to sizing constraints.
RecordStructured does not take ownership of item; it marshals it into an Any proto.
func (*Span) RecordingType ¶
func (sp *Span) RecordingType() tracingpb.RecordingType
RecordingType returns the range's current recording mode.
func (*Span) Redactable ¶
Redactable returns true if this Span's tracer is marked redactable.
func (*Span) SetLazyTag ¶
SetLazyTag adds a tag to the span. The tag's value is expected to implement either fmt.Stringer or LazyTag, and is only stringified using one of the two on demand:
- if the Span has an otel span or a net.Trace, the tag is stringified immediately and passed to the external trace (see SetLazyStatusTag if you want to avoid that).
- if/when the span's recording is collected, the tag is stringified on demand. If the recording is collected multiple times, the tag is stringified multiple times (so, the tag can evolve over time). Since generally the collection of a recording can happen asynchronously, the implementation of Stringer or LazyTag should be thread-safe.
func (*Span) SetLazyTagLocked
deprecated
SetLazyTagLocked is the same as SetLazyTag but assumes that the mutex of sp is being held.
Deprecated: this method should not be used because it's introduced only to go around some tech debt (#100438). Once that issue is addressed, this method should be removed.
func (*Span) SetOtelStatus ¶
SetOtelStatus sets the status of the OpenTelemetry span (if any).
func (*Span) SetRecordingType ¶
func (sp *Span) SetRecordingType(to tracingpb.RecordingType)
SetRecordingType sets the recording mode of the span and its children, recursively. Setting it to RecordingOff disables further recording. Everything recorded so far remains in memory.
func (*Span) SetTag ¶
SetTag adds a tag to the span. If there is a pre-existing tag set for the key, it is overwritten.
func (*Span) UpdateGoroutineIDToCurrent ¶
func (sp *Span) UpdateGoroutineIDToCurrent()
UpdateGoroutineIDToCurrent updates the span's goroutine ID to the current goroutine. This should be called when a different goroutine takes ownership of a span.
type SpanMeta ¶
type SpanMeta struct {
// contains filtered or unexported fields
}
SpanMeta is information about a Span that is not local to this process. Typically, SpanMeta is populated from information about a Span on the other end of an RPC, and is used to derive a child span via `Tracer.StartSpan`. For local spans, SpanMeta is not used, as the *Span directly can be derived from.
SpanMeta contains the trace and span identifiers of the parent, along with additional metadata. In particular, this specifies whether the child should be recording, in which case the contract is that the recording is to be returned to the caller when the child finishes, so that the caller can inductively construct the entire trace.
func SpanMetaFromProto ¶
SpanMetaFromProto converts a TraceInfo proto to SpanMeta.
type SpanOption ¶
type SpanOption interface {
// contains filtered or unexported methods
}
SpanOption is the interface satisfied by options to `Tracer.StartSpan`. A synopsis of the options follows. For details, see their comments.
- WithParent: create a child Span with a local parent. - WithRemoteParentFromSpanMeta: create a child Span with a remote parent. - WithFollowsFrom: hint that child may outlive parent. - WithLogTags: populates the Span tags from a `logtags.Buffer`. - WithCtxLogTags: like WithLogTags, but takes a `context.Context`. - WithTags: adds tags to a Span on creation. - WithForceRealSpan: prevents optimizations that can avoid creating a real span. - WithDetachedRecording: don't include the recording in the parent.
func WithDetachedRecording ¶
func WithDetachedRecording() SpanOption
WithDetachedRecording configures the span to not be included in the parent's recording (if any) under most circumstances. Usually, a parent span includes all its children in its recording. However, sometimes that's not desired; sometimes the creator of a child span has a different plan for how the recording of that child will end up being collected and reported to where it ultimately needs to go. Still, even in these cases, a parent-child relationship is still useful (for example for the purposes of the active spans registry), so the child span cannot simply be created as a root.
For example, in the case of DistSQL, each processor in a flow has its own span, as a child of the flow. The DistSQL infrastructure organizes the collection of each processor span recording independently, without relying on collecting the recording of the flow's span.
In the case when the parent's recording is collected through the span registry, this option is ignore since, in that case, we want as much info as possible.
func WithEventListeners ¶
func WithEventListeners(eventListeners ...EventListener) SpanOption
WithEventListeners registers eventListeners to the span. The listeners are notified of Structured events recorded by the span and its children. Once the span is finished, the listeners are not notified of events any more even from surviving child spans.
The listeners will also be notified of StructuredEvents recorded on remote spans, when the remote recording is imported by the span or one of its children. Note, the listeners will be notified of StructuredEvents in the imported remote recording out of order.
WithEventListeners implies a `RecordingStructured` recording type for the span. If the recording type has been explicitly set to `RecordingVerbose` via the `WithRecording(...) option, that will be respected instead.
The caller should not mutate `eventListeners` after calling WithEventListeners.
func WithFollowsFrom ¶
func WithFollowsFrom() SpanOption
WithFollowsFrom instructs StartSpan to link the child span to its parent using a different kind of relationship than the regular parent-child one, should a child span be created (i.e. should WithParent or WithRemoteParentFromSpanMeta be supplied as well). This relationship was called "follows-from" in the old OpenTracing API. This only matters if the trace is sent to an OpenTelemetry tracer; CRDB itself ignores it (what matters for CRDB is the WithDetachedTrace option). OpenTelemetry does not have a concept of a follows-from relationship at the moment; specifying this option results in the child having a Link to the parent. TODO(andrei): OpenTelemetry used to have a FollowsFrom relationship, but then it was removed for the topic to be reconsidered more deeply. Let's keep an eye on https://github.com/open-telemetry/opentelemetry-specification/issues/65 and see how the thinking evolves.
A WithFollowsFrom child is expected to run asynchronously with respect to the parent span (for example: asynchronous cleanup work), whereas a "regular" child span is not (i.e. the parent span typically waits for the child to Finish()).
There is no penalty for getting this wrong, but it can help external trace systems visualize the traces better.
func WithForceRealSpan ¶
func WithForceRealSpan() SpanOption
WithForceRealSpan forces StartSpan to create of a real Span regardless of the Tracer's tracing mode.
When tracing is disabled all spans are nil; these spans aren't capable of recording, so this option should be passed to StartSpan if the caller wants to be able to call SetVerbose(true) on the span later. If the span should be recording from the beginning, use WithRecording() instead.
func WithLogTags ¶
func WithLogTags(tags *logtags.Buffer) SpanOption
WithLogTags returns a SpanOption that sets the Span tags to the given log tags. When applied, the returned option will apply any logtag name->Span tag name remapping that has been registered via RegisterTagRemapping.
Note that there is no need to use this option with StartSpanCtx, as that will already propagate the log tags from the Context supplied to it to the Span. However, if a WithLogTags option is supplied, it will be used and replaces the Context-derived tags.
func WithParent ¶
func WithParent(sp *Span) SpanOption
WithParent instructs StartSpan to create a child Span from a (local) parent Span.
In case when the parent span is created with a different Tracer (generally, when the parent lives in a different process), WithRemoteParentFromSpanMeta should be used.
WithParent will be a no-op (i.e. the span resulting from applying this option will be a root span, just as if this option hadn't been specified) in the following cases:
- if `sp` is nil
- if `sp` is a sterile span (i.e. a span explicitly marked as not wanting children).
The child inherits the parent's log tags. The data collected in the child trace will be retrieved automatically when the parent's data is retrieved, meaning that the caller has no obligation (and in fact must not) manually propagate the recording to the parent Span.
The child will start recording if the parent is recording at the time of child instantiation.
By default, children are derived using a ChildOf relationship, which corresponds to the expectation that the parent span will wait for the child to Finish(). If this expectation does not hold, WithFollowsFrom should be added to the StartSpan invocation.
WithParent increments sp's reference count. As such, the resulting option must be passed to StartSpan(opt). Once passed to StartSpan(opt), opt cannot be reused. The child span will be responsible for ultimately doing the decrement. The fact that WithParent takes a reference on sp is important to avoid the possibility of deadlocks when buggy code uses a Span after Finish() (which is illegal). See comments on Span.refCnt for details. By taking the reference, it becomes safe (from a deadlock perspective) to call WithParent(sp) concurrently with sp.Finish(). Note that calling WithParent(sp) after sp was re-allocated cannot result in deadlocks regardless of the reference counting.
func WithRecording ¶
func WithRecording(recType tracingpb.RecordingType) SpanOption
WithRecording configures the span to record in the given mode.
The recording mode can be changed later with SetVerbose().
func WithRemoteParentFromLocalSpan ¶
func WithRemoteParentFromLocalSpan(sp *Span) SpanOption
WithRemoteParentFromLocalSpan is equivalent to WithRemoteParentFromSpanMeta(sp.Meta()), but doesn't allocate. The span will be created with parent info, but without being linked into the parent. This is useful when the child needs to be created with a different Tracer than the parent - e.g. when a tenant is calling into the local KV server.
func WithRemoteParentFromSpanMeta ¶
func WithRemoteParentFromSpanMeta(parent SpanMeta) SpanOption
WithRemoteParentFromSpanMeta instructs StartSpan to create a child span descending from a parent described via a SpanMeta. Generally this parent span lives in a different process.
For the purposes of trace recordings, there's no mechanism ensuring that the child's recording will be passed to the parent span. When that's desired, it has to be done manually by calling Span.GetRecording() and propagating the result to the parent by calling Span.ImportRemoteRecording().
The canonical use case for this is around RPC boundaries, where a server handling a request wants to create a child span descending from a parent on a remote machine.
node 1 (network) node 2 -------------------------------------------------------------------------- Span.Meta() ----------> sp2 := Tracer.StartSpan(WithRemoteParentFromSpanMeta(.))
doSomething(sp2)
Span.ImportRemoteRecording(.) <---------- sp2.FinishAndGetRecording()
By default, the child span is derived using a ChildOf relationship, which corresponds to the expectation that the parent span will usually wait for the child to Finish(). If this expectation does not hold, WithFollowsFrom should be added to the StartSpan invocation.
If you're in possession of a TraceInfo instead of a SpanMeta, prefer using WithRemoteParentFromTraceInfo instead. If the TraceInfo is heap-allocated, WithRemoteParentFromTraceInfo will not allocate (whereas WithRemoteParentFromSpanMeta allocates).
func WithRemoteParentFromTraceInfo ¶
func WithRemoteParentFromTraceInfo(ti *tracingpb.TraceInfo) SpanOption
WithRemoteParentFromTraceInfo is like WithRemoteParentFromSpanMeta, except the remote parent info is passed in as *TraceInfo. This is equivalent to WithRemoteParentFromSpanMeta(SpanMetaFromProto(ti)), but more efficient because it doesn't allocate.
func WithSpanKind ¶
func WithSpanKind(kind oteltrace.SpanKind) SpanOption
WithSpanKind configures a span with an OpenTelemetry kind. This option only matters if OpenTelemetry tracing is enabled; the CRDB tracer ignores it otherwise.
func WithSterile ¶
func WithSterile() SpanOption
WithSterile configures the span to not permit any child spans. The would-be children of a sterile span end up being root spans.
Since WithParent(<sterile span>) is a noop, it is allowed to create children of sterile span with any Tracer. This is unlike children of any other spans, which must be created with the same Tracer as the parent.
type SpanRegistry ¶
type SpanRegistry struct {
// contains filtered or unexported fields
}
SpanRegistry is a map that references all non-Finish'ed local root spans, i.e. those for which no WithLocalParent(<non-nil>) option was supplied. The map is keyed on the span ID, which is deterministically unique.
In normal operation, a local root crdbSpan is inserted on creation and removed on .Finish().
The map can be introspected by `Tracer.VisitSpans`. A Span can also be retrieved from its ID by `Tracer.GetActiveSpanByID`.
func (*SpanRegistry) VisitRoots ¶
func (r *SpanRegistry) VisitRoots(visitor func(span RegistrySpan) error) error
VisitRoots calls the visitor callback for every local root span in the registry. Iterations stops when the visitor returns an error. If that error is iterutils.StopIteration(), then VisitRoots() returns nil.
The callback should not hold on to the span after it returns.
func (*SpanRegistry) VisitSpans ¶
func (r *SpanRegistry) VisitSpans(visitor func(span RegistrySpan))
VisitSpans calls the visitor callback for every span in the registry.
The callback should not hold on to the span after it returns.
type SpanReusePercentOpt ¶
type SpanReusePercentOpt uint
SpanReusePercentOpt is the option produced by WithSpanReusePercent(), configuring the Tracer's span reuse policy.
type SpansSnapshot ¶
type SpansSnapshot struct { // CapturedAt is the time when the snapshot was collected. CapturedAt time.Time // Traces contains the collected traces. Each "local route" corresponds to one // trace. "Detached recording" spans are included in the parent's trace. Traces []tracingpb.Recording // Stacks is a map from groutine ID to the goroutine's stack trace. All // goroutines running at the time when this snapshot is produced are // represented here. A goroutine referenced by a span's GoroutineID field will // not be present if it has finished since the respective span was started. Stacks map[int]string // Err is set if an error was encountered when producing the snapshot. The // snapshot will be incomplete in this case. Err error }
SpansSnapshot represents a snapshot of all the open spans at a certain point in time.
type Structured ¶
Structured is an opaque protobuf that can be attached to a trace via `Span.RecordStructured`.
type T ¶
type T interface {
Fatalf(format string, args ...interface{})
}
T is a subset of testing.TB.
type Trace ¶
type Trace struct { Root tracingpb.RecordedSpan // Children are the traces of the child spans. The slice is kept sorted by // child start time. // // Children are added via addChildren(). Once a child is added to a Trace, the // Trace takes ownership; only the Trace can modify that child, since it needs // to maintain the NumSpans and StructuredRecordsSizeBytes bookkeeping. Children []Trace // NumSpans tracks the number of spans in the recording: 1 for the root plus // the size of the child traces recursively. NumSpans int // StructuredRecordsSizeBytes tracks the total size of structured logs in Root // and all the Children, recursively. StructuredRecordsSizeBytes int64 // DroppedDirectChildren maintains info about whether any direct children were // omitted from Children because of recording limits. // // When set, DroppedDirectChildren and DroppedIndirectChildren are also // reflected in corresponding tags on Root. DroppedDirectChildren bool // DroppedIndirectChildren maintains info about whether any indirect children // were omitted from Children because of recording limits. DroppedIndirectChildren bool }
Trace represents the recording of a span and all its descendents.
func (*Trace) Flatten ¶
func (t *Trace) Flatten() []tracingpb.RecordedSpan
Flatten flattens the trace into a slice of spans. The root is the first span, and parents come before children. Otherwise, the spans are not sorted.
See SortSpans() for sorting the result in order to turn it into a tracingpb.Recording.
func (*Trace) PartialClone ¶
PartialClone performs a deep copy of the trace. The immutable slices are not copied: logs, tags and stats.
func (*Trace) ToRecording ¶
ToRecording converts the Trace to a tracingpb.Recording by flattening it and sorting the spans.
type Tracer ¶
type Tracer struct {
// contains filtered or unexported fields
}
Tracer implements tracing requests. It supports:
forwarding events to x/net/trace instances
recording traces. Recorded events can be retrieved at any time.
OpenTelemetry tracing. This is implemented by maintaining a "shadow" OpenTelemetry Span inside each of our spans.
Even when tracing is disabled, we still use this Tracer (with x/net/trace and lightstep disabled) because of its recording capability (verbose tracing needs to work in all cases).
Tracer is currently stateless so we could have a single instance; however, this won't be the case if the cluster settings move away from using global state.
func NewTracer ¶
func NewTracer() *Tracer
NewTracer creates a Tracer with default options.
See NewTracerWithOpt() for controlling various configuration options.
func NewTracerWithOpt ¶
func NewTracerWithOpt(ctx context.Context, opts ...TracerOption) *Tracer
NewTracerWithOpt creates a Tracer and configures it according to the passed-in options.
func (*Tracer) ActiveSpansRegistryEnabled ¶
ActiveSpansRegistryEnabled returns true if this tracer is configured to register spans with the activeSpansRegistry
func (*Tracer) AlwaysTrace ¶
AlwaysTrace returns true if operations should be traced regardless of the context.
func (*Tracer) Close ¶
func (t *Tracer) Close()
Close cleans up any resources associated with a Tracer.
func (*Tracer) ExtractMetaFrom ¶
ExtractMetaFrom is used to deserialize a span metadata (if any) from the given Carrier. This, alongside InjectMetaFrom, can be used to carry span metadata across process boundaries. See serializationFormat for more details.
func (*Tracer) GetActiveSpanByID ¶
func (t *Tracer) GetActiveSpanByID(spanID tracingpb.SpanID) RegistrySpan
GetActiveSpanByID retrieves any active root span given its ID.
func (*Tracer) GetActiveSpansRegistry ¶
func (t *Tracer) GetActiveSpansRegistry() *SpanRegistry
GetActiveSpansRegistry returns a pointer to the registry containing all in-flight on the node.
func (*Tracer) GetAutomaticSnapshot ¶
func (t *Tracer) GetAutomaticSnapshot(id SnapshotID) (SpansSnapshot, error)
GetAutomaticSnapshot is a variant of GetSnapshot but for retrieving automatic snapshots.
func (*Tracer) GetAutomaticSnapshots ¶
func (t *Tracer) GetAutomaticSnapshots() []SnapshotInfo
GetAutomaticSnapshots returns info on all stored automatic span snapshots.
func (*Tracer) GetSnapshot ¶
func (t *Tracer) GetSnapshot(id SnapshotID) (SpansSnapshot, error)
GetSnapshot returns the snapshot with the given ID. If the ID is below the minimum stored snapshot, then the requested snapshot must have been garbage-collected and errSnapshotTooOld is returned. If the snapshot id is beyond the maximum stored ID, errSnapshotDoesntExist is returned.
Note that SpansSpanshot has an Err field through which errors are returned. In these error cases, the snapshot will be incomplete.
func (*Tracer) GetSnapshots ¶
func (t *Tracer) GetSnapshots() []SnapshotInfo
GetSnapshots returns info on all stored span snapshots.
func (*Tracer) HasExternalSink ¶
HasExternalSink returns whether the tracer is configured to report to an external tracing collector.
func (*Tracer) InjectMetaInto ¶
InjectMetaInto is used to serialize the given span metadata into the given Carrier. This, alongside ExtractMetaFrom, can be used to carry span metadata across process boundaries. See serializationFormat for more details.
func (*Tracer) PanicOnUseAfterFinish ¶
PanicOnUseAfterFinish returns true if the Tracer is configured to crash when a Span is used after it was previously Finish()ed. Crashing is supposed to be best-effort as, in the future, reliably detecting use-after-Finish might not be possible (i.e. it's not possible if the Span is reused for a different operation before the use-after-Finish occurs).
func (*Tracer) PeriodicSnapshotsLoop ¶
PeriodicSnapshotsLoop runs until done closes, calling SaveAutomaticSnapshot if enabled at the interval set by the trace.snapshot.rate setting.
func (*Tracer) Redactable ¶
Redactable returns true if the tracer is configured to emit redactable logs. This value will affect the redactability of messages from already open Spans.
func (*Tracer) SaveAutomaticSnapshot ¶
func (t *Tracer) SaveAutomaticSnapshot() SnapshotInfo
SaveAutomaticSnapshot is like SaveSnapshot but saves the snapshot created in the "automatic" snapshots buffer instead of the manual snapshot buffer.
func (*Tracer) SaveSnapshot ¶
func (t *Tracer) SaveSnapshot() SnapshotInfo
SaveSnapshot collects a snapshot of the currently open spans (i.e. the contents of the Tracer's active spans registry) saves it in-memory in the Tracer. The snapshot's ID is returned; the snapshot can be retrieved subsequently through t.GetSnapshot(id).
Snapshots also include a dump of all the goroutine stack traces.
func (*Tracer) SetActiveSpansRegistryEnabled ¶
SetActiveSpansRegistryEnabled controls whether spans are created and registered with activeSpansRegistry.
func (*Tracer) SetOpenTelemetryTracer ¶
SetOpenTelemetryTracer sets the OpenTelemetry tracer to use as a "shadow tracer". A nil value means that no otel tracer will be used.
func (*Tracer) SetRedactable ¶
SetRedactable changes the redactability of the Tracer. This affects any future trace spans created.
func (*Tracer) ShouldRecordAsyncSpans ¶
ShouldRecordAsyncSpans returns whether or not we should include recordings from async child spans in the parent span. See TestingRecordAsyncSpans, this mode is only used in tests.
func (*Tracer) SpanRegistry ¶
func (t *Tracer) SpanRegistry() *SpanRegistry
SpanRegistry exports the registry containing all currently-open spans.
func (*Tracer) StartSpan ¶
func (t *Tracer) StartSpan(operationName string, os ...SpanOption) *Span
StartSpan starts a Span. See SpanOption for details.
func (*Tracer) StartSpanCtx ¶
func (t *Tracer) StartSpanCtx( ctx context.Context, operationName string, os ...SpanOption, ) (context.Context, *Span)
StartSpanCtx starts a Span and returns it alongside a wrapping Context derived from the supplied Context. Any log tags found in the supplied Context are propagated into the Span; this behavior can be modified by passing WithLogTags explicitly.
See SpanOption for other options that can be passed.
func (*Tracer) TestingGetStatsAndReset ¶
TestingGetStatsAndReset returns the number of spans created and allocated since the previous TestingGetStatsAndReset() call. The difference created-allocated indicates how many times spans have been reused through the Tracer's pool.
Panics if the MaintainAllocationCounters testing knob was not set.
func (*Tracer) TestingRecordAsyncSpans ¶
func (t *Tracer) TestingRecordAsyncSpans()
TestingRecordAsyncSpans is a test-only helper that configures the tracer to include recordings from forked/async child spans, when retrieving the recording for a parent span.
func (*Tracer) VisitSpans ¶
func (t *Tracer) VisitSpans(visitor func(span RegistrySpan) error) error
VisitSpans calls the visitor callback for every local root span in the registry. Iterations stops when the visitor returns an error. If that error is iterutils.StopIteration(), then VisitSpans() returns nil.
The callback should not hold on to the span after it returns.
type TracerOption ¶
type TracerOption interface {
// contains filtered or unexported methods
}
TracerOption is implemented by the arguments to the Tracer constructor.
func WithClusterSettings ¶
func WithClusterSettings(sv *settings.Values) TracerOption
WithClusterSettings configures the Tracer according to the relevant cluster settings. Future changes to those cluster settings will update the Tracer.
func WithSpanReusePercent ¶
func WithSpanReusePercent(percent uint32) TracerOption
WithSpanReusePercent configures the Tracer span reuse ratio, overriding the environmental default.
func WithTestingKnobs ¶
func WithTestingKnobs(knobs TracerTestingKnobs) TracerOption
WithTestingKnobs configures the Tracer with the specified knobs.
func WithTracingMode ¶
func WithTracingMode(opt TracingMode) TracerOption
WithTracingMode configures the Tracer's tracing mode.
func WithUseAfterFinishOpt ¶
func WithUseAfterFinishOpt(panicOnUseAfterFinish, debugUseAfterFinish bool) TracerOption
WithUseAfterFinishOpt allows control over the Tracer's behavior when a Span is used after it had been Finish()ed.
panicOnUseAfterFinish configures the Tracer to panic when it detects that any method on a Span is called after that Span was Finish()ed. If not set, such uses are tolerated as silent no-ops.
debugUseAfterFinish configures the Tracer to collect stack traces on every Span.Finish() call. These are presented when the finished span is reused. This option is expensive. It's invalid to debugUseAfterFinish if panicOnUseAfterFinish is not set.
type TracerTestingKnobs ¶
type TracerTestingKnobs struct { // Clock allows the time source for spans to be controlled. Clock timeutil.TimeSource // UseNetTrace, if set, forces the Traces to always create spans which record // to net.Trace objects. UseNetTrace bool // MaintainAllocationCounters, if set, configures the Tracer to maintain // counters about span creation. See Tracer.GetStatsAndReset(). MaintainAllocationCounters bool // ReleaseSpanToPool, if set, if called just before a span is put in the // sync.Pool for reuse. If the hook returns false, the span will not be put in // the pool. ReleaseSpanToPool func(*Span) bool }
TracerTestingKnobs contains knobs for a Tracer.
type TracingAggregator ¶
type TracingAggregator struct {
// contains filtered or unexported fields
}
A TracingAggregator can be used to aggregate and render AggregatorEvents that are emitted as part of its tracing spans' recording.
func TracingAggregatorForContext ¶
func TracingAggregatorForContext(ctx context.Context) *TracingAggregator
TracingAggregatorForContext creates a TracingAggregator if the provided context has a tracing span.
func (*TracingAggregator) ForEachAggregatedEvent ¶
func (b *TracingAggregator) ForEachAggregatedEvent(f func(name string, event AggregatorEvent))
ForEachAggregatedEvent executes f on each event in the TracingAggregator's in-memory map.
func (*TracingAggregator) Notify ¶
func (b *TracingAggregator) Notify(event Structured) EventConsumptionStatus
Notify implements the EventListener interface.
type TracingMode ¶
type TracingMode int
TracingMode specifies whether span creation is enabled or disabled by default, when other conditions that don't explicitly turn tracing on don't apply.
const ( // TracingModeFromEnv configures tracing according to enableTracingByDefault. TracingModeFromEnv TracingMode = iota // TracingModeOnDemand means that Spans will no be created unless there's a // particular reason to create them (i.e. a span being created with // WithForceRealSpan(), a net.Trace or OpenTelemetry tracers attached). TracingModeOnDemand // TracingModeActiveSpansRegistry means that Spans are always created. // Currently-open spans are accessible through the active spans registry. // // If no net.Trace/OpenTelemetry tracer is attached, spans do not record // events by default (i.e. do not accumulate log messages via Span.Record() or // a history of finished child spans). In order for a span to record events, // it must be started with the WithRecording() option). TracingModeActiveSpansRegistry )