Skip to content

Commit

Permalink
Merge #97895
Browse files Browse the repository at this point in the history
97895: kvserver: add stack history to trace of slow requests r=andreimatei a=dt

First commit is #97829.

This uses the new tracing.MaybeRecordStackHistory helper to add any recorded history
of the request evaluation's stack to the trace before returning the result (success or
error) if the request is considered to have been 'slow'. A new setting controls how slow
a request must be to trigger a search for the history of its stack, and that history is only
available if and at the granularity controlled by trace.snapshot.rate setting.

A history of the stack can be particularly useful when a request returns an error caused
by a timeout or context cancellation, as often by the time such a request is returning, it
is no longer at the same point where it spent all the time that caused it to be so slow.
Instead, this patch in conjunction with the tracer's snapshotting facility allows asking
'what was i doing all that time?' and having the tracer answer, even if the code asking
was doing work that was not directly instrumented to make itself visible in traces.

Example statement bundle trace, after sprinkling some sleeps in pkg/storage code (trimmed for brevity):
<img width="1529" alt="Screenshot 2023-03-01 at 9 13 32 PM" src="https://user-images.githubusercontent.com/15615/222313217-af508bd0-67e4-4fe6-9fae-d18b924e74f5.png">


Release note (ops change): the setting kv.slow_requests.trace_stack_history.threshold can be used to attach available stack history from tracer snapshots to the trace of slow requests.
Epic: none.

Co-authored-by: David Taylor <[email protected]>
  • Loading branch information
craig[bot] and dt committed Mar 9, 2023
2 parents b47dba8 + a104275 commit 9b05952
Show file tree
Hide file tree
Showing 5 changed files with 148 additions and 0 deletions.
11 changes: 11 additions & 0 deletions pkg/server/node.go
Original file line number Diff line number Diff line change
Expand Up @@ -165,6 +165,13 @@ var (
"controls if server side traces are redacted for tenant operations",
true,
).WithPublic()

slowRequestHistoricalStackThreshold = settings.RegisterDurationSetting(
settings.SystemOnly,
"kv.trace.slow_request_stacks.threshold",
`duration spent in processing above any available stack history is appended to its trace, if automatic trace snapshots are enabled`,
time.Second*30,
)
)

type nodeMetrics struct {
Expand Down Expand Up @@ -1187,6 +1194,10 @@ func (n *Node) batchInternal(
if br.Error != nil {
panic(kvpb.ErrorUnexpectedlySet(n.stores, br))
}
if timeutil.Since(tStart) > slowRequestHistoricalStackThreshold.Get(&n.storeCfg.Settings.SV) {
tracing.SpanFromContext(ctx).MaybeRecordStackHistory(tStart)
}

n.metrics.callComplete(timeutil.Since(tStart), pErr)
br.Error = pErr

Expand Down
53 changes: 53 additions & 0 deletions pkg/util/tracing/tracer_snapshots.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb"
"github.com/cockroachdb/errors"
"github.com/petermattis/goid"
)

// SpansSnapshot represents a snapshot of all the open spans at a certain point
Expand Down Expand Up @@ -299,3 +300,55 @@ func (t *Tracer) runPeriodicSnapshotsLoop(
}
}
}

// 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 (sp *Span) MaybeRecordStackHistory(since time.Time) {
if sp == nil || !sp.i.hasVerboseSink() {
return
}

t := sp.Tracer()
id := int(goid.Get())

var prevStack string

t.snapshotsMu.Lock()
defer t.snapshotsMu.Unlock()
for i := 0; i < t.snapshotsMu.autoSnapshots.Len(); i++ {
s := t.snapshotsMu.autoSnapshots.Get(i)
if s.CapturedAt.Before(since) {
continue
}
stack, ok := s.Stacks[id]
if ok {
sp.RecordStructured(stackDelta(prevStack, stack, timeutil.Since(s.CapturedAt)))
prevStack = stack
}
}
}

func stackDelta(base, change string, age time.Duration) Structured {
if base == "" {
return &tracingpb.CapturedStack{Stack: change, Age: age}
}

var i, lines int
for i = range base {
c := base[len(base)-1-i]
if i > len(change) || change[len(change)-1-i] != c {
break
}
if c == '\n' {
lines++
}
}
return &tracingpb.CapturedStack{
Stack: change[:len(change)-i], SharedSuffix: int32(i), SharedLines: int32(lines),
}
}
58 changes: 58 additions & 0 deletions pkg/util/tracing/tracer_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/testutils/skip"
"github.com/cockroachdb/cockroach/pkg/util/iterutil"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb"
"github.com/cockroachdb/logtags"
"github.com/gogo/protobuf/types"
Expand Down Expand Up @@ -960,3 +961,60 @@ func TestTracerSnapshotLoop(t *testing.T) {
snaps := tr.GetAutomaticSnapshots()
require.NotEmpty(t, snaps)
}

// helper that blocks long enough to appear in the stack.
func blockingFunc1(ch chan<- struct{}) {
ch <- struct{}{} // allow snapshot to start.
ch <- struct{}{} // wait for snapshot to be done.
}

// helper that blocks long enough to appear in the stack.
func blockingFunc2(ch chan<- struct{}) {
ch <- struct{}{}
ch <- struct{}{}
}

// helper that blocks long enough to appear in the stack.
func blockingFunc3(ch chan<- struct{}) {
ch <- struct{}{}
ch <- struct{}{}
}

func blockingCaller(ch chan<- struct{}) {
blockingFunc2(ch)
}

// TestTracerStackHistory tests MaybeRecordStackHistory.
func TestTracerStackHistory(t *testing.T) {
defer leaktest.AfterTest(t)()
tr := NewTracer()

sp := tr.StartSpan("test", WithRecording(tracingpb.RecordingVerbose))
ch := make(chan struct{})
defer close(ch)
go func() {
for range ch {
tr.SaveAutomaticSnapshot()
<-ch // read again to unpark func.
}
}()

blockingFunc1(ch)
started := timeutil.Now()
blockingFunc2(ch)
blockingFunc3(ch)
blockingCaller(ch)

sp.MaybeRecordStackHistory(started)

rec := sp.FinishAndGetRecording(tracingpb.RecordingVerbose)[0]
require.Len(t, rec.StructuredRecords, 3)
require.Len(t, rec.Logs, 3)
require.Len(t, rec.StructuredRecords, 3)
for i := range rec.Logs {
require.NotContains(t, rec.Logs[i].Message, "tracing.blockingFunc1")
}
require.Contains(t, rec.Logs[0].Message, "tracing.blockingFunc2")
require.Contains(t, rec.Logs[1].Message, "tracing.blockingFunc3")
require.Contains(t, rec.Logs[2].Message, "tracing.blockingCaller")
}
11 changes: 11 additions & 0 deletions pkg/util/tracing/tracingpb/recorded_span.go
Original file line number Diff line number Diff line change
Expand Up @@ -198,3 +198,14 @@ func (m OperationMetadata) SafeFormat(s redact.SafePrinter, _ rune) {
}
s.Print("}")
}

func (c CapturedStack) String() string {
age := c.Age.Seconds()
if c.Stack == "" && c.SharedSuffix > 0 {
return fmt.Sprintf("stack as of %.1fs ago had not changed from previous stack", age)
}
if c.SharedLines > 0 {
return fmt.Sprintf("stack as of %.1fs ago: %s\n ...+%d lines matching previous stack", age, c.Stack, c.SharedLines)
}
return fmt.Sprintf("stack as of %.1fs ago: %s", age, c.Stack)
}
15 changes: 15 additions & 0 deletions pkg/util/tracing/tracingpb/recorded_span.proto
Original file line number Diff line number Diff line change
Expand Up @@ -174,3 +174,18 @@ message NormalizedSpan {
map<string, OperationMetadata> children_metadata = 9 [(gogoproto.nullable) = false];
repeated NormalizedSpan children = 6 [(gogoproto.nullable) = false];
}

message CapturedStack {
option (gogoproto.goproto_stringer) = false;
// Stack is the strack trace, or distinct prefix of the stack trace if a previous
// capture of this stack is also being recorded.
string stack = 1;
// SharedSuffix indicates the length of the elided suffix of this stack that
// was identical to the previous capture of this stack.
int32 shared_suffix = 2;
// SharedLines indicated how many newlines were in the elided shared suffix.
int32 shared_lines = 3;
// Age indicates the duration prior to being recorded that this stack was captured.
int64 age = 4 [(gogoproto.casttype) = "time.Duration"];
// NEXT ID: 5.
}

0 comments on commit 9b05952

Please sign in to comment.