From b4f24fdb165c84a0c6cf9d4093b36445645ca01b Mon Sep 17 00:00:00 2001 From: Alfonso Subiotto Marques Date: Thu, 10 Dec 2020 13:30:23 +0100 Subject: [PATCH] sql: add sql.trace.stmt.enable_threshold 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. --- docs/generated/settings/settings.html | 3 +- pkg/sql/conn_executor_exec.go | 70 +++++++++++++++++++++++++++ pkg/sql/exec_util.go | 17 ++++++- pkg/sql/txn_state.go | 35 +------------- 4 files changed, 90 insertions(+), 35 deletions(-) diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index f81045162070..215d9b7b735f 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -91,7 +91,8 @@ sql.temp_object_cleaner.cleanup_intervalduration30m0show often to clean up orphaned temporary objects sql.trace.log_statement_executebooleanfalseset to true to enable logging of executed statements sql.trace.session_eventlog.enabledbooleanfalseset to true to enable session tracing. Note that enabling this may have a non-trivial negative performance impact. -sql.trace.txn.enable_thresholdduration0sduration beyond which all transactions are traced (set to 0 to disable) +sql.trace.stmt.enable_thresholdduration0sduration 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. +sql.trace.txn.enable_thresholdduration0sduration 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). timeseries.storage.enabledbooleantrueif set, periodic timeseries data is stored within the cluster; disabling is not recommended unless you are storing the data elsewhere timeseries.storage.resolution_10s.ttlduration240h0m0sthe maximum age of time series data stored at the 10 second resolution. Data older than this is subject to rollup and deletion. timeseries.storage.resolution_30m.ttlduration2160h0m0sthe maximum age of time series data stored at the 30 minute resolution. Data older than this is subject to deletion. diff --git a/pkg/sql/conn_executor_exec.go b/pkg/sql/conn_executor_exec.go index 7b565b785926..4c725c217fda 100644 --- a/pkg/sql/conn_executor_exec.go +++ b/pkg/sql/conn_executor_exec.go @@ -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) } @@ -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) +} diff --git a/pkg/sql/exec_util.go b/pkg/sql/exec_util.go index 06bf348d0f86..145e5c3f732e 100644 --- a/pkg/sql/exec_util.go +++ b/pkg/sql/exec_util.go @@ -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 diff --git a/pkg/sql/txn_state.go b/pkg/sql/txn_state.go index 933d195c35e6..f0a99e7397cf 100644 --- a/pkg/sql/txn_state.go +++ b/pkg/sql/txn_state.go @@ -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" @@ -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") } @@ -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) @@ -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()