From 98ee45db48b940ae75073397f187b4c854a9c723 Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Tue, 1 Nov 2022 19:02:48 -0700 Subject: [PATCH] sql: reduce the overhead of EXPLAIN ANALYZE In order to propagate the execution stats across the distributed query plan we use the tracing infrastructure, where each stats object is added as "structured metadata" to the trace. Thus, whenever we're collecting the exec stats for a statement, we must enable tracing. Previously, in many cases we would enable it at the highest verbosity level which has non-trivial overhead. In some cases this was an overkill (e.g. in `EXPLAIN ANALYZE` we don't really care about the trace containing all of the gory details - we won't expose it anyway), so this is now fixed by using the less verbose "structured" verbosity level. As a concrete example of the difference: for a stmt that without `EXPLAIN ANALYZE` takes around 190ms, with `EXPLAIN ANALYZE` it would previously run for about 1.8s and now it takes around 210ms. This required some minor changes to the row-by-row outbox and router setups to collect thats even if the recording is not verbose. Release note (performance improvement): The overhead of running `EXPLAIN ANALYZE` and `EXPLAIN ANALYZE (DISTSQL)` has been significantly reduced. The overhead of `EXPLAIN ANALYZE (DEBUG)` didn't change. --- pkg/sql/flowinfra/BUILD.bazel | 1 + pkg/sql/flowinfra/outbox.go | 11 +++++++---- pkg/sql/instrumentation.go | 12 +++++++++++- pkg/sql/rowflow/BUILD.bazel | 1 + pkg/sql/rowflow/routers.go | 7 +++++-- 5 files changed, 25 insertions(+), 7 deletions(-) diff --git a/pkg/sql/flowinfra/BUILD.bazel b/pkg/sql/flowinfra/BUILD.bazel index f1efd4cfc32a..bff1ba9048c0 100644 --- a/pkg/sql/flowinfra/BUILD.bazel +++ b/pkg/sql/flowinfra/BUILD.bazel @@ -44,6 +44,7 @@ go_library( "//pkg/util/syncutil", "//pkg/util/timeutil", "//pkg/util/tracing", + "//pkg/util/tracing/tracingpb", "@com_github_cockroachdb_errors//:errors", "@com_github_cockroachdb_redact//:redact", "@com_github_gogo_protobuf//proto", diff --git a/pkg/sql/flowinfra/outbox.go b/pkg/sql/flowinfra/outbox.go index 99e2c742e8e6..b45b01564f70 100644 --- a/pkg/sql/flowinfra/outbox.go +++ b/pkg/sql/flowinfra/outbox.go @@ -26,6 +26,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/cockroach/pkg/util/tracing" + "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" "go.opentelemetry.io/otel/attribute" ) @@ -210,10 +211,12 @@ func (m *Outbox) mainLoop(ctx context.Context) error { var span *tracing.Span ctx, span = execinfra.ProcessorSpan(ctx, "outbox") defer span.Finish() - if span != nil && span.IsVerbose() { - m.statsCollectionEnabled = true - span.SetTag(execinfrapb.FlowIDTagKey, attribute.StringValue(m.flowCtx.ID.String())) - span.SetTag(execinfrapb.StreamIDTagKey, attribute.IntValue(int(m.streamID))) + if span != nil { + m.statsCollectionEnabled = span.RecordingType() != tracingpb.RecordingOff + if span.IsVerbose() { + span.SetTag(execinfrapb.FlowIDTagKey, attribute.StringValue(m.flowCtx.ID.String())) + span.SetTag(execinfrapb.StreamIDTagKey, attribute.IntValue(int(m.streamID))) + } } if m.stream == nil { diff --git a/pkg/sql/instrumentation.go b/pkg/sql/instrumentation.go index 7c34be1db875..b098bdc6c828 100644 --- a/pkg/sql/instrumentation.go +++ b/pkg/sql/instrumentation.go @@ -326,7 +326,17 @@ func (ih *instrumentationHelper) Setup( } ih.collectExecStats = true - newCtx, ih.sp = tracing.EnsureChildSpan(ctx, cfg.AmbientCtx.Tracer, "traced statement", tracing.WithRecording(tracingpb.RecordingVerbose)) + // Execution stats are propagated as structured metadata, so we definitely + // need to enable the tracing. We default to the RecordingStructured level + // in order to reduce the overhead of EXPLAIN ANALYZE. + recType := tracingpb.RecordingStructured + if ih.collectBundle || ih.withStatementTrace != nil { + // Use the verbose recording only if we're collecting the bundle (the + // verbose trace is very helpful during debugging) or if we have a + // testing callback. + recType = tracingpb.RecordingVerbose + } + newCtx, ih.sp = tracing.EnsureChildSpan(ctx, cfg.AmbientCtx.Tracer, "traced statement", tracing.WithRecording(recType)) ih.shouldFinishSpan = true return newCtx, true } diff --git a/pkg/sql/rowflow/BUILD.bazel b/pkg/sql/rowflow/BUILD.bazel index ade2874005d5..c081e469100b 100644 --- a/pkg/sql/rowflow/BUILD.bazel +++ b/pkg/sql/rowflow/BUILD.bazel @@ -28,6 +28,7 @@ go_library( "//pkg/util/mon", "//pkg/util/syncutil", "//pkg/util/tracing", + "//pkg/util/tracing/tracingpb", "@com_github_cockroachdb_errors//:errors", "@com_github_cockroachdb_redact//:redact", "@io_opentelemetry_go_otel//attribute", diff --git a/pkg/sql/rowflow/routers.go b/pkg/sql/rowflow/routers.go index cc8d59df3d34..5ca9b1e91732 100644 --- a/pkg/sql/rowflow/routers.go +++ b/pkg/sql/rowflow/routers.go @@ -32,6 +32,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/mon" "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/cockroach/pkg/util/tracing" + "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" "github.com/cockroachdb/errors" "github.com/cockroachdb/redact" "go.opentelemetry.io/otel/attribute" @@ -262,7 +263,7 @@ func (rb *routerBase) setupStreams( // init must be called after setupStreams but before Start. func (rb *routerBase) init(ctx context.Context, flowCtx *execinfra.FlowCtx, types []*types.T) { // Check if we're recording stats. - if s := tracing.SpanFromContext(ctx); s != nil && s.IsVerbose() { + if s := tracing.SpanFromContext(ctx); s != nil && s.RecordingType() != tracingpb.RecordingOff { rb.statsCollectionEnabled = true } @@ -313,7 +314,9 @@ func (rb *routerBase) Start(ctx context.Context, wg *sync.WaitGroup, _ context.C if rb.statsCollectionEnabled { ctx, span = execinfra.ProcessorSpan(ctx, "router output") defer span.Finish() - span.SetTag(execinfrapb.StreamIDTagKey, attribute.IntValue(int(ro.streamID))) + if span.IsVerbose() { + span.SetTag(execinfrapb.StreamIDTagKey, attribute.IntValue(int(ro.streamID))) + } ro.stats.Inputs = make([]execinfrapb.InputStats, 1) }