Skip to content

Commit

Permalink
sql: add query level execution stats to sampled query log
Browse files Browse the repository at this point in the history
Partially addresses: cockroachdb#84729

This change adds:
- network bytes sent
- maximum memory usage
- maximum disk usage
- KV bytes read
- KV rows read
- network messages

fields to the `SampledQuery` telemetry log.

Release justification: low risk, high benefit changes to existing
functionality

Release note (sql change): This change adds::
- network bytes sent
- maximum memory usage
- maximum disk usage
- KV bytes read
- KV rows read
- network messages fields to the `SampledQuery` telemetry log.
  • Loading branch information
Thomas Hardy committed Sep 7, 2022
1 parent 9bd5bb0 commit ad84bed
Show file tree
Hide file tree
Showing 6 changed files with 269 additions and 91 deletions.
15 changes: 11 additions & 4 deletions pkg/sql/exec_log.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/execstats"
"github.com/cockroachdb/cockroach/pkg/sql/opt/exec"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/privilege"
Expand Down Expand Up @@ -395,12 +396,12 @@ func (p *planner) maybeLogStatementInternal(
requiredTimeElapsed = 0
}
if telemetryMetrics.maybeUpdateLastEmittedTime(telemetryMetrics.timeNow(), requiredTimeElapsed) {
var contentionNanos int64
var stats execstats.QueryLevelStats
if queryLevelStats, ok := p.instrumentation.GetQueryLevelStats(); ok {
contentionNanos = queryLevelStats.ContentionTime.Nanoseconds()
stats = *queryLevelStats
}

contentionNanos = telemetryMetrics.getContentionTime(contentionNanos)
stats = telemetryMetrics.getQueryLevelStats(stats)

skippedQueries := telemetryMetrics.resetSkippedQueryCount()
sampledQuery := eventpb.SampledQuery{
Expand Down Expand Up @@ -437,8 +438,14 @@ func (p *planner) maybeLogStatementInternal(
InvertedJoinCount: int64(p.curPlan.instrumentation.joinAlgorithmCounts[exec.InvertedJoin]),
ApplyJoinCount: int64(p.curPlan.instrumentation.joinAlgorithmCounts[exec.ApplyJoin]),
ZigZagJoinCount: int64(p.curPlan.instrumentation.joinAlgorithmCounts[exec.ZigZagJoin]),
ContentionNanos: contentionNanos,
ContentionNanos: stats.ContentionTime.Nanoseconds(),
Regions: p.curPlan.instrumentation.regions,
NetworkBytesSent: stats.NetworkBytesSent,
MaxMemUsage: stats.MaxMemUsage,
MaxDiskUsage: stats.MaxDiskUsage,
KVBytesRead: stats.KVBytesRead,
KVRowsRead: stats.KVRowsRead,
NetworkMessages: stats.NetworkMessages,
}
p.logOperationalEventsOnlyExternally(ctx, &sampledQuery)
} else {
Expand Down
15 changes: 7 additions & 8 deletions pkg/sql/telemetry_logging.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@ import (
"time"

"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/sql/execstats"
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/cockroach/pkg/util/tracing"
Expand Down Expand Up @@ -54,10 +55,8 @@ type TelemetryLoggingTestingKnobs struct {
// getTimeNow allows tests to override the timeutil.Now() function used
// when updating rolling query counts.
getTimeNow func() time.Time
// getContentionNanos allows tests to override the recorded contention time
// for the query. Used to stub non-zero values to populate the log's contention
// time field.
getContentionNanos func() int64
// getQueryLevelMetrics allows tests to override the recorded query level stats.
getQueryLevelStats func() execstats.QueryLevelStats
// getTracingStatus allows tests to override whether the current query has tracing
// enabled or not. Queries with tracing enabled are always sampled to telemetry.
getTracingStatus func() bool
Expand Down Expand Up @@ -91,11 +90,11 @@ func (t *TelemetryLoggingMetrics) maybeUpdateLastEmittedTime(
return false
}

func (t *TelemetryLoggingMetrics) getContentionTime(contentionTimeInNanoseconds int64) int64 {
if t.Knobs != nil && t.Knobs.getContentionNanos != nil {
return t.Knobs.getContentionNanos()
func (t *TelemetryLoggingMetrics) getQueryLevelStats(queryLevelStats execstats.QueryLevelStats) execstats.QueryLevelStats {
if t.Knobs != nil && t.Knobs.getQueryLevelStats != nil {
return t.Knobs.getQueryLevelStats()
}
return contentionTimeInNanoseconds
return queryLevelStats
}

func (t *TelemetryLoggingMetrics) isTracing(_ *tracing.Span, tracingEnabled bool) bool {
Expand Down
Loading

0 comments on commit ad84bed

Please sign in to comment.