Skip to content

Commit

Permalink
tracing,bench: introduce BenchmarkTracing
Browse files Browse the repository at this point in the history
BenchmarkTracing measures the overhead of always-on tracing. It also
reports the memory utilization.

    make bench PKG=./pkg/bench \
      TESTFLAGS='-benchtime=5000x -count 20' \
      BENCHES='BenchmarkTracing' BENCHTIMEOUT=50m

Plumbing that into `benchstat` (old is without always-on tracing, new is
with):

    name                                            old time/op    new time/op    delta
    Tracing/Cockroach/tracing=x/Scan1-16               172µs ± 2%     164µs ± 4%   -4.87%  (p=0.000 n=17+19)
    Tracing/Cockroach/tracing=x/Insert-16              253µs ± 5%     249µs ± 5%   -1.68%  (p=0.028 n=19+20)
    Tracing/MultinodeCockroach/tracing=x/Scan1-16      413µs ± 5%     491µs ±30%  +18.89%  (p=0.000 n=19+20)
    Tracing/MultinodeCockroach/tracing=x/Insert-16     543µs ±12%     539µs ± 7%     ~     (p=0.925 n=20+20)

    name                                            old alloc/op   new alloc/op   delta
    Tracing/Cockroach/tracing=x/Scan1-16              23.0kB ± 0%    23.3kB ± 2%   +0.95%  (p=0.000 n=17+17)
    Tracing/Cockroach/tracing=x/Insert-16             38.7kB ±21%    38.1kB ± 7%     ~     (p=0.687 n=20+19)
    Tracing/MultinodeCockroach/tracing=x/Scan1-16     72.0kB ±15%    75.1kB ±10%   +4.28%  (p=0.000 n=19+20)
    Tracing/MultinodeCockroach/tracing=x/Insert-16     102kB ±19%     101kB ±11%     ~     (p=0.813 n=20+19)

    name                                            old allocs/op  new allocs/op  delta
    Tracing/Cockroach/tracing=x/Scan1-16                 247 ± 0%       248 ± 1%   +0.21%  (p=0.004 n=16+17)
    Tracing/Cockroach/tracing=x/Insert-16                309 ± 0%       310 ± 2%     ~     (p=0.059 n=17+20)
    Tracing/MultinodeCockroach/tracing=x/Scan1-16        762 ± 3%       766 ± 3%   +0.58%  (p=0.015 n=17+19)
    Tracing/MultinodeCockroach/tracing=x/Insert-16       734 ± 3%       733 ± 2%     ~     (p=0.402 n=19+19)

Release note: None
  • Loading branch information
irfansharif committed Jan 12, 2021
1 parent 623063e commit 36992b3
Show file tree
Hide file tree
Showing 6 changed files with 111 additions and 4 deletions.
51 changes: 51 additions & 0 deletions pkg/bench/bench_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -400,6 +400,53 @@ func BenchmarkSQL(b *testing.B) {
})
}

// BenchmarkTracing measures the overhead of always-on tracing. It also reports
// the memory utilization.
//
// TODO(irfansharif): This benchmark is only useful while we transition between
// the legacy trace.mode, and the "always-on" mode introduced in 21.1. We can
// remove it in 21.2.
func BenchmarkTracing(b *testing.B) {
skip.UnderShort(b)
defer log.Scope(b).Close(b)

for _, dbFn := range []func(*testing.B, BenchmarkFn){
benchmarkCockroach,
benchmarkMultinodeCockroach,
} {
dbName := runtime.FuncForPC(reflect.ValueOf(dbFn).Pointer()).Name()
dbName = strings.TrimPrefix(dbName, "github.com/cockroachdb/cockroach/pkg/bench.benchmark")

b.Run(dbName, func(b *testing.B) {
dbFn(b, func(b *testing.B, db *sqlutils.SQLRunner) {
for _, tracingEnabled := range []bool{true, false} {
var tracingMode string
if tracingEnabled {
tracingMode = "background"
} else {
tracingMode = "legacy"
}
db.Exec(b, fmt.Sprintf("SET CLUSTER SETTING trace.mode = %s", tracingMode))
b.Run(fmt.Sprintf("tracing=%s", tracingMode[:1]), func(b *testing.B) {
for _, runFn := range []func(*testing.B, *sqlutils.SQLRunner, int){
runBenchmarkScan1,
runBenchmarkInsert,
} {
fnName := runtime.FuncForPC(reflect.ValueOf(runFn).Pointer()).Name()
fnName = strings.TrimPrefix(fnName, "github.com/cockroachdb/cockroach/pkg/bench.runBenchmark")
b.Run(fnName, func(b *testing.B) {
b.ReportAllocs()

runFn(b, db, 1 /* count */)
})
}
})
}
})
})
}
}

// runBenchmarkUpdate benchmarks updating count random rows in a table.
func runBenchmarkUpdate(b *testing.B, db *sqlutils.SQLRunner, count int) {
defer func() {
Expand Down Expand Up @@ -506,6 +553,10 @@ func runBenchmarkDelete(b *testing.B, db *sqlutils.SQLRunner, rows int) {
b.StopTimer()
}

func runBenchmarkScan1(b *testing.B, db *sqlutils.SQLRunner, count int) {
runBenchmarkScan(b, db, count, 1)
}

// runBenchmarkScan benchmarks scanning a table containing count rows.
func runBenchmarkScan(b *testing.B, db *sqlutils.SQLRunner, count int, limit int) {
defer func() {
Expand Down
1 change: 0 additions & 1 deletion pkg/server/node.go
Original file line number Diff line number Diff line change
Expand Up @@ -959,7 +959,6 @@ func (n *Node) setupSpanForIncomingRPC(
if br == nil {
return
}
// TODO(irfansharif): How come this span is never Finish()-ed? #58721.
if grpcSpan != nil {
// If our local span descends from a parent on the other
// end of the RPC (i.e. the !isLocalRequest) case,
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/colflow/vectorized_flow.go
Original file line number Diff line number Diff line change
Expand Up @@ -947,7 +947,6 @@ func (s *vectorizedFlowCreator) setupOutput(
// Start a separate recording so that GetRecording will return
// the recordings for only the child spans containing stats.
ctx, span := tracing.ChildSpanRemote(ctx, "")
// TODO(irfansharif): How come this span is never Finish()-ed? #58721.
if atomic.AddInt32(&s.numOutboxesDrained, 1) == atomic.LoadInt32(&s.numOutboxes) {
// At the last outbox, we can accurately retrieve stats for the
// whole flow from parent monitors. These stats are added to a
Expand All @@ -965,6 +964,7 @@ func (s *vectorizedFlowCreator) setupOutput(
})
}
finishVectorizedStatsCollectors(ctx, vscs)
span.Finish()
return []execinfrapb.ProducerMetadata{{TraceData: span.GetRecording()}}
},
},
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/conn_executor_exec.go
Original file line number Diff line number Diff line change
Expand Up @@ -636,14 +636,14 @@ func (ex *connExecutor) execStmtInOpenState(
if !alreadyRecording && stmtTraceThreshold > 0 {
ctx, stmtThresholdSpan = createRootOrChildSpan(ctx, "trace-stmt-threshold", ex.transitionCtx.tracer)
stmtThresholdSpan.SetVerbose(true)
// TODO(irfansharif): How come this span is never Finish()-ed? #58721.
}

if err := ex.dispatchToExecutionEngine(ctx, p, res); err != nil {
return nil, nil, err
}

if stmtThresholdSpan != nil {
stmtThresholdSpan.Finish()
logTraceAboveThreshold(
ctx,
stmtThresholdSpan.GetRecording(),
Expand Down
57 changes: 57 additions & 0 deletions pkg/sql/tests/kv_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/base"
kv2 "github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/errors"
)

Expand Down Expand Up @@ -148,12 +149,26 @@ type kvSQL struct {
}

func newKVSQL(b *testing.B) kvInterface {
return newKVSQLWithTracing(b, false)
}

func newKVSQLWithTracing(b *testing.B, enabled bool) kvInterface {
s, db, _ := serverutils.StartServer(b, base.TestServerArgs{UseDatabase: "bench"})

if _, err := db.Exec(`CREATE DATABASE IF NOT EXISTS bench`); err != nil {
b.Fatal(err)
}

var tracingMode string
if enabled {
tracingMode = "background"
} else {
tracingMode = "legacy"
}
if _, err := db.Exec(fmt.Sprintf("SET CLUSTER SETTING trace.mode = %s", tracingMode)); err != nil {
b.Fatal(err)
}

kv := &kvSQL{}
kv.db = db
kv.doneFn = func() {
Expand Down Expand Up @@ -297,3 +312,45 @@ func BenchmarkKV(b *testing.B) {
})
}
}

func BenchmarkTracing(b *testing.B) {
defer log.Scope(b).Close(b)

for i, opFn := range []func(kvInterface, int, int) error{
kvInterface.Insert,
kvInterface.Update,
kvInterface.Delete,
kvInterface.Scan,
} {
opName := runtime.FuncForPC(reflect.ValueOf(opFn).Pointer()).Name()
opName = strings.TrimPrefix(opName, "github.com/cockroachdb/cockroach/pkg/sql/tests_test.kvInterface.")
for _, tracingEnabled := range []bool{true, false} {
var trace = "f"
if tracingEnabled {
trace = "t"
}

for _, rows := range []int{1, 10, 100, 1000, 10000} {
b.Run(fmt.Sprintf("tracing=%s/%s/rows=%d", trace, opName, rows), func(b *testing.B) {

b.ReportAllocs()

kv := newKVSQLWithTracing(b, tracingEnabled)
defer kv.done()

if err := kv.prep(rows, i != 0 /* Insert */ && i != 2 /* Delete */); err != nil {
b.Fatal(err)
}

b.ResetTimer()
for i := 0; i < b.N; i++ {
if err := opFn(kv, rows, i); err != nil {
b.Fatal(err)
}
}
b.StopTimer()
})
}
}
}
}
2 changes: 1 addition & 1 deletion pkg/util/tracing/tracer.go
Original file line number Diff line number Diff line change
Expand Up @@ -263,7 +263,7 @@ func (t *Tracer) AlwaysTrace() bool {

// startSpanGeneric is the implementation of StartSpanCtx and StartSpan. In
// the latter case, ctx == noCtx and the returned Context is the supplied one;
// otherwise the returned Context reflects the returned Span.
// otherwise the returned Context embeds the returned Span.
func (t *Tracer) startSpanGeneric(
ctx context.Context, opName string, opts spanOptions,
) (context.Context, *Span) {
Expand Down

0 comments on commit 36992b3

Please sign in to comment.