Skip to content

Commit

Permalink
Merge #57733
Browse files Browse the repository at this point in the history
57733: sql: add sql.trace.stmt.enable_threshold r=yuzefovich,andreimatei a=asubiotto

Release note (sql change): add sql.trace.stmt.enable_thresold. Similar to
sql.trace.txn.enable_threshold, this logs a trace of any statements that take
longer than the specified duration. This new setting allows for more
granularity than the transaction threshold since it applies to individual
statements and does not include overhead such as communication with a SQL
client.

Co-authored-by: Alfonso Subiotto Marques <[email protected]>
  • Loading branch information
craig[bot] and asubiotto committed Dec 10, 2020
2 parents 709f99a + b4f24fd commit 50ebbb3
Show file tree
Hide file tree
Showing 4 changed files with 90 additions and 35 deletions.
3 changes: 2 additions & 1 deletion docs/generated/settings/settings.html
Original file line number Diff line number Diff line change
Expand Up @@ -91,7 +91,8 @@
<tr><td><code>sql.temp_object_cleaner.cleanup_interval</code></td><td>duration</td><td><code>30m0s</code></td><td>how often to clean up orphaned temporary objects</td></tr>
<tr><td><code>sql.trace.log_statement_execute</code></td><td>boolean</td><td><code>false</code></td><td>set to true to enable logging of executed statements</td></tr>
<tr><td><code>sql.trace.session_eventlog.enabled</code></td><td>boolean</td><td><code>false</code></td><td>set to true to enable session tracing. Note that enabling this may have a non-trivial negative performance impact.</td></tr>
<tr><td><code>sql.trace.txn.enable_threshold</code></td><td>duration</td><td><code>0s</code></td><td>duration beyond which all transactions are traced (set to 0 to disable)</td></tr>
<tr><td><code>sql.trace.stmt.enable_threshold</code></td><td>duration</td><td><code>0s</code></td><td>duration beyond which all statements are traced (set to 0 to disable). This applies to individual statements within a transaction and is therefore finer-grained than sql.trace.txn.enable_threshold.</td></tr>
<tr><td><code>sql.trace.txn.enable_threshold</code></td><td>duration</td><td><code>0s</code></td><td>duration beyond which all transactions are traced (set to 0 to disable). This setting is coarser grained thansql.trace.stmt.enable_threshold because it applies to all statements within a transaction as well as client communication (e.g. retries).</td></tr>
<tr><td><code>timeseries.storage.enabled</code></td><td>boolean</td><td><code>true</code></td><td>if set, periodic timeseries data is stored within the cluster; disabling is not recommended unless you are storing the data elsewhere</td></tr>
<tr><td><code>timeseries.storage.resolution_10s.ttl</code></td><td>duration</td><td><code>240h0m0s</code></td><td>the maximum age of time series data stored at the 10 second resolution. Data older than this is subject to rollup and deletion.</td></tr>
<tr><td><code>timeseries.storage.resolution_30m.ttl</code></td><td>duration</td><td><code>2160h0m0s</code></td><td>the maximum age of time series data stored at the 30 minute resolution. Data older than this is subject to deletion.</td></tr>
Expand Down
70 changes: 70 additions & 0 deletions pkg/sql/conn_executor_exec.go
Original file line number Diff line number Diff line change
Expand Up @@ -619,9 +619,29 @@ func (ex *connExecutor) execStmtInOpenState(
p.stmt = stmt
p.cancelChecker = cancelchecker.NewCancelChecker(ctx)
p.autoCommit = os.ImplicitTxn.Get() && !ex.server.cfg.TestingKnobs.DisableAutoCommit

var stmtThresholdSpan *tracing.Span
alreadyRecording := ex.transitionCtx.sessionTracing.Enabled()
stmtTraceThreshold := traceStmtThreshold.Get(&ex.planner.execCfg.Settings.SV)
if !alreadyRecording && stmtTraceThreshold > 0 {
ctx, stmtThresholdSpan = createRootOrChildSpan(ctx, "trace-stmt-threshold", ex.transitionCtx.tracer)
stmtThresholdSpan.StartRecording(tracing.SnowballRecording)
}

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

if stmtThresholdSpan != nil {
logTraceAboveThreshold(
ctx,
stmtThresholdSpan.GetRecording(),
fmt.Sprintf("SQL stmt %s", stmt.AST.String()),
stmtTraceThreshold,
timeutil.Since(ex.phaseTimes[sessionQueryReceived]),
)
}

if err := res.Err(); err != nil {
return makeErrEvent(err)
}
Expand Down Expand Up @@ -1443,3 +1463,53 @@ func (ex *connExecutor) recordTransaction(ev txnEvent, implicit bool, txnStart t
ex.extraTxnState.numRows,
)
}

// createRootOrChildSpan is used to create spans for txns and stmts. It inspects
// parentCtx for an existing span and creates a root span if none is found, or a
// child span if one is found. A context derived from parentCtx which
// additionally contains the new span is also returned.
// TODO(asubiotto): Use tracing.EnsureChildSpan once that is available.
func createRootOrChildSpan(
parentCtx context.Context, opName string, tracer *tracing.Tracer,
) (context.Context, *tracing.Span) {
// WithForceRealSpan is used to support the use of session tracing, which
// may start recording on this span.
var sp *tracing.Span
if parentSp := tracing.SpanFromContext(parentCtx); parentSp != nil {
// Create a child span for this operation.
sp = parentSp.Tracer().StartSpan(
opName,
tracing.WithParentAndAutoCollection(parentSp),
tracing.WithCtxLogTags(parentCtx),
tracing.WithForceRealSpan(),
)
} else {
// Create a root span for this operations.
sp = tracer.StartSpan(
opName, tracing.WithCtxLogTags(parentCtx), tracing.WithForceRealSpan(),
)
}
return tracing.ContextWithSpan(parentCtx, sp), sp
}

// logTraceAboveThreshold logs a span's recording if the duration is above a
// given threshold. It is used when txn or stmt threshold tracing is enabled.
// This function assumes that sp is non-nil and threshold tracing was enabled.
func logTraceAboveThreshold(
ctx context.Context, r tracing.Recording, opName string, threshold, elapsed time.Duration,
) {
if elapsed < threshold {
return
}
if r == nil {
log.Warning(ctx, "missing trace when threshold tracing was enabled")
return
}
dump := r.String()
if len(dump) == 0 {
return
}
// Note that log lines larger than 65k are truncated in the debug zip (see
// #50166).
log.Infof(ctx, "%s took %s, exceeding threshold of %s:\n%s", opName, elapsed, threshold, dump)
}
17 changes: 16 additions & 1 deletion pkg/sql/exec_util.go
Original file line number Diff line number Diff line change
Expand Up @@ -155,7 +155,22 @@ var allowCrossDatabaseSeqOwner = settings.RegisterPublicBoolSetting(
// if they are not output.
var traceTxnThreshold = settings.RegisterPublicDurationSetting(
"sql.trace.txn.enable_threshold",
"duration beyond which all transactions are traced (set to 0 to disable)", 0,
"duration beyond which all transactions are traced (set to 0 to "+
"disable). This setting is coarser grained than"+
"sql.trace.stmt.enable_threshold because it applies to all statements "+
"within a transaction as well as client communication (e.g. retries).", 0,
)

// traceStmtThreshold is identical to traceTxnThreshold except it applies to
// individual statements in a transaction. The motivation for this setting is
// to be able to reduce the noise associated with a larger transaction (e.g.
// round trips to client).
var traceStmtThreshold = settings.RegisterPublicDurationSetting(
"sql.trace.stmt.enable_threshold",
"duration beyond which all statements are traced (set to 0 to disable). "+
"This applies to individual statements within a transaction and is therefore "+
"finer-grained than sql.trace.txn.enable_threshold.",
0,
)

// traceSessionEventLogEnabled can be used to enable the event log
Expand Down
35 changes: 2 additions & 33 deletions pkg/sql/txn_state.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/util/contextutil"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/metric"
"github.com/cockroachdb/cockroach/pkg/util/mon"
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
Expand Down Expand Up @@ -157,25 +156,8 @@ func (ts *txnState) resetForNewSQLTxn(
// (automatic or user-directed) retries. The span is closed by finishSQLTxn().
// TODO(andrei): figure out how to close these spans on server shutdown? Ties
// into a larger discussion about how to drain SQL and rollback open txns.
var sp *tracing.Span
opName := sqlTxnName

// Create a span for the new txn. The span is always WithForceRealSpan to support the
// use of session tracing, which may start recording on it.
if parentSp := tracing.SpanFromContext(connCtx); parentSp != nil {
// Create a child span for this SQL txn.
sp = parentSp.Tracer().StartSpan(
opName,
tracing.WithParentAndAutoCollection(parentSp),
tracing.WithCtxLogTags(connCtx),
tracing.WithForceRealSpan(),
)
} else {
// Create a root span for this SQL txn.
sp = tranCtx.tracer.StartSpan(
opName, tracing.WithCtxLogTags(connCtx), tracing.WithForceRealSpan())
}

txnCtx, sp := createRootOrChildSpan(connCtx, opName, tranCtx.tracer)
if txnType == implicitTxn {
sp.SetTag("implicit", "true")
}
Expand All @@ -188,9 +170,6 @@ func (ts *txnState) resetForNewSQLTxn(
ts.recordingStart = timeutil.Now()
}

// Put the new span in the context.
txnCtx := tracing.ContextWithSpan(connCtx, sp)

ts.sp = sp
ts.Ctx, ts.cancel = contextutil.WithCancel(txnCtx)

Expand Down Expand Up @@ -233,17 +212,7 @@ func (ts *txnState) finishSQLTxn() {
}

if ts.recordingThreshold > 0 {
if r := ts.sp.GetRecording(); r != nil {
if elapsed := timeutil.Since(ts.recordingStart); elapsed >= ts.recordingThreshold {
dump := r.String()
if len(dump) > 0 {
log.Infof(ts.Ctx, "SQL txn took %s, exceeding tracing threshold of %s:\n%s",
elapsed, ts.recordingThreshold, dump)
}
}
} else {
log.Warning(ts.Ctx, "missing trace when sampled was enabled")
}
logTraceAboveThreshold(ts.Ctx, ts.sp.GetRecording(), "SQL txn", ts.recordingThreshold, timeutil.Since(ts.recordingStart))
}

ts.sp.Finish()
Expand Down

0 comments on commit 50ebbb3

Please sign in to comment.