Skip to content

Commit

Permalink
log: add span options to AnnotateCtxWithSpan
Browse files Browse the repository at this point in the history
Before this patch, AnnotateCtxWithSpan would always create either a root
span or a child span (if the caller had a parent span). This patch gives
it the option to create a follows-from span or always a root span. These
options are used for some spans created by long-running processes: such
background processes don't generally want children because the
relationship between the parent and the child is trivial, and reflecting
this relationship in traces only makes for ugly-looking long traces.

Release note: None
  • Loading branch information
andreimatei committed Sep 17, 2021
1 parent c92ecb9 commit a744d12
Show file tree
Hide file tree
Showing 13 changed files with 58 additions and 27 deletions.
2 changes: 1 addition & 1 deletion pkg/kv/kvclient/kvcoord/txn_coord_sender.go
Original file line number Diff line number Diff line change
Expand Up @@ -478,7 +478,7 @@ func (tc *TxnCoordSender) Send(
return nil, tc.finalizeNonLockingTxnLocked(ctx, ba)
}

ctx, sp := tc.AnnotateCtxWithSpan(ctx, OpTxnCoordSender)
ctx, sp := tc.AnnotateCtxWithSpan(ctx, OpTxnCoordSender, log.ChildSpan)
defer sp.Finish()

// Associate the txnID with the trace.
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvprober/kvprober.go
Original file line number Diff line number Diff line change
Expand Up @@ -179,7 +179,7 @@ func (p *Prober) Start(ctx context.Context, stopper *stop.Stopper) error {
defer t.Stop()
t.Reset(d())

ctx, sp := ambient.AnnotateCtxWithSpan(ctx, desc)
ctx, sp := ambient.AnnotateCtxWithSpan(ctx, desc, log.RootSpan)
defer sp.Finish()

ctx, cancel := stopper.WithCancelOnQuiesce(ctx)
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/liveness/liveness.go
Original file line number Diff line number Diff line change
Expand Up @@ -696,7 +696,7 @@ func (nl *NodeLiveness) Start(ctx context.Context, opts NodeLivenessStartOptions
ambient.AddLogTag("liveness-hb", nil)
ctx, cancel := opts.Stopper.WithCancelOnQuiesce(context.Background())
defer cancel()
ctx, sp := ambient.AnnotateCtxWithSpan(ctx, "liveness heartbeat loop")
ctx, sp := ambient.AnnotateCtxWithSpan(ctx, "liveness heartbeat loop", log.RootSpan)
defer sp.Finish()

incrementEpoch := true
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/replica_raftstorage.go
Original file line number Diff line number Diff line change
Expand Up @@ -442,7 +442,7 @@ func (r *Replica) GetSnapshot(
rangeID := r.RangeID

startKey := r.mu.state.Desc.StartKey
ctx, sp := r.AnnotateCtxWithSpan(ctx, "snapshot")
ctx, sp := r.AnnotateCtxWithSpan(ctx, "snapshot", log.ChildSpan)
defer sp.Finish()

log.Eventf(ctx, "new engine snapshot for replica %s", r)
Expand Down
6 changes: 3 additions & 3 deletions pkg/server/admin_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -318,7 +318,7 @@ func TestAdminAPIDatabases(t *testing.T) {
ts := s.(*TestServer)

ac := log.AmbientContext{Tracer: s.ClusterSettings().Tracer}
ctx, span := ac.AnnotateCtxWithSpan(context.Background(), "test")
ctx, span := ac.AnnotateCtxWithSpan(context.Background(), "test", log.RootSpan)
defer span.Finish()

const testdb = "test"
Expand Down Expand Up @@ -648,7 +648,7 @@ func TestAdminAPITableDetails(t *testing.T) {
schemaName := "testschema"

ac := log.AmbientContext{Tracer: s.ClusterSettings().Tracer}
ctx, span := ac.AnnotateCtxWithSpan(context.Background(), "test")
ctx, span := ac.AnnotateCtxWithSpan(context.Background(), "test", log.RootSpan)
defer span.Finish()

tableSchema := `nulls_allowed INT8,
Expand Down Expand Up @@ -794,7 +794,7 @@ func TestAdminAPIZoneDetails(t *testing.T) {

// Create database and table.
ac := log.AmbientContext{Tracer: s.ClusterSettings().Tracer}
ctx, span := ac.AnnotateCtxWithSpan(context.Background(), "test")
ctx, span := ac.AnnotateCtxWithSpan(context.Background(), "test", log.RootSpan)
defer span.Finish()
setupQueries := []string{
"CREATE DATABASE test",
Expand Down
2 changes: 1 addition & 1 deletion pkg/server/diagnostics/reporter.go
Original file line number Diff line number Diff line change
Expand Up @@ -125,7 +125,7 @@ func (r *Reporter) PeriodicallyReportDiagnostics(ctx context.Context, stopper *s
// NOTE: This can be slow because of cloud detection; use cloudinfo.Disable() in
// tests to avoid that.
func (r *Reporter) ReportDiagnostics(ctx context.Context) {
ctx, span := r.AmbientCtx.AnnotateCtxWithSpan(ctx, "usageReport")
ctx, span := r.AmbientCtx.AnnotateCtxWithSpan(ctx, "usageReport", log.RootSpan)
defer span.Finish()

report := r.CreateReport(ctx, telemetry.ResetCounts)
Expand Down
2 changes: 1 addition & 1 deletion pkg/server/diagnostics/update_checker.go
Original file line number Diff line number Diff line change
Expand Up @@ -100,7 +100,7 @@ func (u *UpdateChecker) PeriodicallyCheckForUpdates(ctx context.Context, stopper
// The returned boolean indicates if the check succeeded (and thus does not need
// to be re-attempted by the scheduler after a retry-interval).
func (u *UpdateChecker) CheckForUpdates(ctx context.Context) bool {
ctx, span := u.AmbientCtx.AnnotateCtxWithSpan(ctx, "usageReport")
ctx, span := u.AmbientCtx.AnnotateCtxWithSpan(ctx, "version update check", log.RootSpan)
defer span.Finish()

url := u.buildUpdatesURL(ctx)
Expand Down
10 changes: 5 additions & 5 deletions pkg/server/migration.go
Original file line number Diff line number Diff line change
Expand Up @@ -42,7 +42,7 @@ var _ serverpb.MigrationServer = &migrationServer{}
func (m *migrationServer) ValidateTargetClusterVersion(
ctx context.Context, req *serverpb.ValidateTargetClusterVersionRequest,
) (*serverpb.ValidateTargetClusterVersionResponse, error) {
ctx, span := m.server.AnnotateCtxWithSpan(ctx, "validate-cluster-version")
ctx, span := m.server.AnnotateCtxWithSpan(ctx, "validate-cluster-version", log.ChildSpan)
defer span.Finish()
ctx = logtags.AddTag(ctx, "validate-cluster-version", nil)

Expand Down Expand Up @@ -89,7 +89,7 @@ func (m *migrationServer) BumpClusterVersion(
ctx context.Context, req *serverpb.BumpClusterVersionRequest,
) (*serverpb.BumpClusterVersionResponse, error) {
const opName = "bump-cluster-version"
ctx, span := m.server.AnnotateCtxWithSpan(ctx, opName)
ctx, span := m.server.AnnotateCtxWithSpan(ctx, opName, log.ChildSpan)
defer span.Finish()
ctx = logtags.AddTag(ctx, opName, nil)

Expand Down Expand Up @@ -156,7 +156,7 @@ func (m *migrationServer) SyncAllEngines(
ctx context.Context, _ *serverpb.SyncAllEnginesRequest,
) (*serverpb.SyncAllEnginesResponse, error) {
const opName = "sync-all-engines"
ctx, span := m.server.AnnotateCtxWithSpan(ctx, opName)
ctx, span := m.server.AnnotateCtxWithSpan(ctx, opName, log.ChildSpan)
defer span.Finish()
ctx = logtags.AddTag(ctx, opName, nil)

Expand Down Expand Up @@ -188,7 +188,7 @@ func (m *migrationServer) PurgeOutdatedReplicas(
ctx context.Context, req *serverpb.PurgeOutdatedReplicasRequest,
) (*serverpb.PurgeOutdatedReplicasResponse, error) {
const opName = "purged-outdated-replicas"
ctx, span := m.server.AnnotateCtxWithSpan(ctx, opName)
ctx, span := m.server.AnnotateCtxWithSpan(ctx, opName, log.ChildSpan)
defer span.Finish()
ctx = logtags.AddTag(ctx, opName, nil)

Expand All @@ -215,7 +215,7 @@ func (m *migrationServer) DeprecateBaseEncryptionRegistry(
ctx context.Context, req *serverpb.DeprecateBaseEncryptionRegistryRequest,
) (*serverpb.DeprecateBaseEncryptionRegistryResponse, error) {
const opName = "deprecate-base-encryption-registry"
ctx, span := m.server.AnnotateCtxWithSpan(ctx, opName)
ctx, span := m.server.AnnotateCtxWithSpan(ctx, opName, log.ChildSpan)
defer span.Finish()
ctx = logtags.AddTag(ctx, opName, nil)

Expand Down
8 changes: 4 additions & 4 deletions pkg/server/node.go
Original file line number Diff line number Diff line change
Expand Up @@ -365,9 +365,9 @@ func (n *Node) AnnotateCtx(ctx context.Context) context.Context {

// AnnotateCtxWithSpan is a convenience wrapper; see AmbientContext.
func (n *Node) AnnotateCtxWithSpan(
ctx context.Context, opName string,
ctx context.Context, opName string, spanOpt log.SpanOption,
) (context.Context, *tracing.Span) {
return n.storeCfg.AmbientCtx.AnnotateCtxWithSpan(ctx, opName)
return n.storeCfg.AmbientCtx.AnnotateCtxWithSpan(ctx, opName, spanOpt)
}

// start starts the node by registering the storage instance for the RPC
Expand Down Expand Up @@ -870,7 +870,7 @@ func (n *Node) recordJoinEvent(ctx context.Context) {
}

_ = n.stopper.RunAsyncTask(ctx, "record-join", func(bgCtx context.Context) {
ctx, span := n.AnnotateCtxWithSpan(bgCtx, "record-join-event")
ctx, span := n.AnnotateCtxWithSpan(bgCtx, "record-join-event", log.ChildSpan)
defer span.Finish()
retryOpts := base.DefaultRetryOptions()
retryOpts.Closer = n.stopper.ShouldQuiesce()
Expand Down Expand Up @@ -1364,7 +1364,7 @@ func (n *Node) GossipSubscription(
func (n *Node) Join(
ctx context.Context, req *roachpb.JoinNodeRequest,
) (*roachpb.JoinNodeResponse, error) {
ctx, span := n.AnnotateCtxWithSpan(ctx, "alloc-{node,store}-id")
ctx, span := n.AnnotateCtxWithSpan(ctx, "alloc-{node,store}-id", log.ChildSpan)
defer span.Finish()

activeVersion := n.storeCfg.Settings.Version.ActiveVersion(ctx)
Expand Down
4 changes: 2 additions & 2 deletions pkg/server/server.go
Original file line number Diff line number Diff line change
Expand Up @@ -828,9 +828,9 @@ func (s *Server) AnnotateCtx(ctx context.Context) context.Context {

// AnnotateCtxWithSpan is a convenience wrapper; see AmbientContext.
func (s *Server) AnnotateCtxWithSpan(
ctx context.Context, opName string,
ctx context.Context, opName string, spanOpt log.SpanOption,
) (context.Context, *tracing.Span) {
return s.cfg.AmbientCtx.AnnotateCtxWithSpan(ctx, opName)
return s.cfg.AmbientCtx.AnnotateCtxWithSpan(ctx, opName, spanOpt)
}

// ClusterID returns the ID of the cluster this server is a part of.
Expand Down
2 changes: 1 addition & 1 deletion pkg/ts/db.go
Original file line number Diff line number Diff line change
Expand Up @@ -187,7 +187,7 @@ func (p *poller) poll() {
return
}

ctx, span := p.AnnotateCtxWithSpan(bgCtx, "ts-poll")
ctx, span := p.AnnotateCtxWithSpan(bgCtx, "ts-poll", log.ChildSpan)
defer span.Finish()

if err := p.db.StoreData(ctx, p.r, data); err != nil {
Expand Down
39 changes: 35 additions & 4 deletions pkg/util/log/ambient_context.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@ package log

import (
"context"
"fmt"

"github.com/cockroachdb/cockroach/pkg/util/tracing"
"github.com/cockroachdb/logtags"
Expand Down Expand Up @@ -153,12 +154,13 @@ func (ac *AmbientContext) annotateCtxInternal(ctx context.Context) context.Conte
// AnnotateCtxWithSpan annotates the given context with the information in
// AmbientContext (see AnnotateCtx) and opens a span.
//
// If the given context has a span, the new span is a child of that span.
// Otherwise, the Tracer in AmbientContext is used to create a new root span.
// spanOpt controls the relationship between the new span and the caller's span.
// If the caller does not have a span, then the new span will be a root regardless
// of the option.
//
// The caller is responsible for closing the span (via Span.Finish).
func (ac *AmbientContext) AnnotateCtxWithSpan(
ctx context.Context, opName string,
ctx context.Context, opName string, spanOpt SpanOption,
) (context.Context, *tracing.Span) {
switch ctx {
case context.TODO(), context.Background():
Expand All @@ -173,5 +175,34 @@ func (ac *AmbientContext) AnnotateCtxWithSpan(
}
}

return tracing.EnsureChildSpan(ctx, ac.Tracer, opName)
var sp *tracing.Span
switch spanOpt {
case FollowsFromSpan:
ctx, sp = tracing.EnsureForkSpan(ctx, ac.Tracer, opName)
case ChildSpan:
ctx, sp = tracing.EnsureChildSpan(ctx, ac.Tracer, opName)
case RootSpan:
ctx, sp = ac.Tracer.StartSpanCtx(ctx, opName)
default:
panic(fmt.Sprintf("unsupported SpanReference: %v", spanOpt))
}
return ctx, sp
}

// SpanOption controls the type of span created by AnnotateCtxWithSpan.
type SpanOption int

const (
// FollowsFromSpan creates a span that's not included in the caller's
// recording. If the caller does not have a span, a root span is created. See
// stop.FollowsFromSpan for more details.
FollowsFromSpan SpanOption = iota

// ChildSpan creates a span that a child of the caller's span. If the caller
// does not have a span, a root span is created. See stop.ChildSpan for more
// details.
ChildSpan

// RootSpan creates a root span.
RootSpan
)
4 changes: 2 additions & 2 deletions pkg/util/log/ambient_context_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -52,7 +52,7 @@ func TestAnnotateCtxSpan(t *testing.T) {
ctx1 := tracing.ContextWithSpan(context.Background(), sp1)
Event(ctx1, "a")

ctx2, sp2 := ac.AnnotateCtxWithSpan(ctx1, "child")
ctx2, sp2 := ac.AnnotateCtxWithSpan(ctx1, "child", ChildSpan)
Event(ctx2, "b")

Event(ctx1, "c")
Expand All @@ -76,7 +76,7 @@ func TestAnnotateCtxSpan(t *testing.T) {
// caller.

ac.Tracer = tracer
ctx, sp := ac.AnnotateCtxWithSpan(context.Background(), "s")
ctx, sp := ac.AnnotateCtxWithSpan(context.Background(), "s", RootSpan)
require.Equal(t, sp, tracing.SpanFromContext(ctx))
require.NotNil(t, sp)
require.False(t, sp.IsVerbose())
Expand Down

0 comments on commit a744d12

Please sign in to comment.