Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

sql: reduce the overhead of EXPLAIN ANALYZE #91117

Merged
merged 1 commit into from
Nov 3, 2022
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
1 change: 1 addition & 0 deletions pkg/sql/flowinfra/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Expand Down
11 changes: 7 additions & 4 deletions pkg/sql/flowinfra/outbox.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
)

Expand Down Expand Up @@ -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 {
Expand Down
12 changes: 11 additions & 1 deletion pkg/sql/instrumentation.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
}
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/rowflow/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Expand Down
7 changes: 5 additions & 2 deletions pkg/sql/rowflow/routers.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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
}

Expand Down Expand Up @@ -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)
}

Expand Down