Skip to content

Commit 43e5c88

Browse files
authored
Merge pull request #156282 from dhartunian/backport25.2-148542
release-25.2: sql: add probabilistic transaction tracing
2 parents 3490f8a + f7bf9ad commit 43e5c88

File tree

12 files changed

+286
-22
lines changed

12 files changed

+286
-22
lines changed

docs/generated/settings/settings-for-tenants.txt

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -387,7 +387,8 @@ sql.temp_object_cleaner.cleanup_interval duration 30m0s how often to clean up or
387387
sql.temp_object_cleaner.wait_interval duration 30m0s how long after creation a temporary object will be cleaned up application
388388
sql.log.all_statements.enabled (alias: sql.trace.log_statement_execute) boolean false set to true to enable logging of all executed statements application
389389
sql.trace.stmt.enable_threshold duration 0s enables tracing on all statements; statements executing for longer than this duration will have their trace logged (set to 0 to disable); note that enabling this may have a negative performance impact; this setting applies to individual statements within a transaction and is therefore finer-grained than sql.trace.txn.enable_threshold application
390-
sql.trace.txn.enable_threshold duration 0s enables tracing on all transactions; transactions open for longer than this duration will have their trace logged (set to 0 to disable); note that enabling this may have a negative performance impact; 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) application
390+
sql.trace.txn.enable_threshold duration 0s enables transaction traces for transactions exceeding this duration, used with `sql.trace.txn.sample_rate` application
391+
sql.trace.txn.sample_rate float 1 enables probabilistic transaction tracing. It should be used in conjunction with `sql.trace.txn.enable_threshold`. A percentage of transactions between 0 and 1.0 will have tracing enabled, and only those which exceed the configured threshold will be logged. application
391392
sql.ttl.changefeed_replication.disabled boolean false if true, deletes issued by TTL will not be replicated via changefeeds (this setting will be ignored by changefeeds that have the ignore_disable_changefeed_replication option set; such changefeeds will continue to replicate all TTL deletes) application
392393
sql.ttl.default_delete_batch_size integer 100 default amount of rows to delete in a single query during a TTL job application
393394
sql.ttl.default_delete_rate_limit integer 100 default delete rate limit (rows per second) per node for each TTL job. Use 0 to signify no rate limit. application

docs/generated/settings/settings.html

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -342,7 +342,8 @@
342342
<tr><td><div id="setting-sql-temp-object-cleaner-wait-interval" class="anchored"><code>sql.temp_object_cleaner.wait_interval</code></div></td><td>duration</td><td><code>30m0s</code></td><td>how long after creation a temporary object will be cleaned up</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
343343
<tr><td><div id="setting-sql-trace-log-statement-execute" class="anchored"><code>sql.log.all_statements.enabled<br />(alias: sql.trace.log_statement_execute)</code></div></td><td>boolean</td><td><code>false</code></td><td>set to true to enable logging of all executed statements</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
344344
<tr><td><div id="setting-sql-trace-stmt-enable-threshold" class="anchored"><code>sql.trace.stmt.enable_threshold</code></div></td><td>duration</td><td><code>0s</code></td><td>enables tracing on all statements; statements executing for longer than this duration will have their trace logged (set to 0 to disable); note that enabling this may have a negative performance impact; this setting applies to individual statements within a transaction and is therefore finer-grained than sql.trace.txn.enable_threshold</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
345-
<tr><td><div id="setting-sql-trace-txn-enable-threshold" class="anchored"><code>sql.trace.txn.enable_threshold</code></div></td><td>duration</td><td><code>0s</code></td><td>enables tracing on all transactions; transactions open for longer than this duration will have their trace logged (set to 0 to disable); note that enabling this may have a negative performance impact; 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)</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
345+
<tr><td><div id="setting-sql-trace-txn-enable-threshold" class="anchored"><code>sql.trace.txn.enable_threshold</code></div></td><td>duration</td><td><code>0s</code></td><td>enables transaction traces for transactions exceeding this duration, used with `sql.trace.txn.sample_rate`</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
346+
<tr><td><div id="setting-sql-trace-txn-sample-rate" class="anchored"><code>sql.trace.txn.sample_rate</code></div></td><td>float</td><td><code>1</code></td><td>enables probabilistic transaction tracing. It should be used in conjunction with `sql.trace.txn.enable_threshold`. A percentage of transactions between 0 and 1.0 will have tracing enabled, and only those which exceed the configured threshold will be logged.</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
346347
<tr><td><div id="setting-sql-ttl-changefeed-replication-disabled" class="anchored"><code>sql.ttl.changefeed_replication.disabled</code></div></td><td>boolean</td><td><code>false</code></td><td>if true, deletes issued by TTL will not be replicated via changefeeds (this setting will be ignored by changefeeds that have the ignore_disable_changefeed_replication option set; such changefeeds will continue to replicate all TTL deletes)</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
347348
<tr><td><div id="setting-sql-ttl-default-delete-batch-size" class="anchored"><code>sql.ttl.default_delete_batch_size</code></div></td><td>integer</td><td><code>100</code></td><td>default amount of rows to delete in a single query during a TTL job</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
348349
<tr><td><div id="setting-sql-ttl-default-delete-rate-limit" class="anchored"><code>sql.ttl.default_delete_rate_limit</code></div></td><td>integer</td><td><code>100</code></td><td>default delete rate limit (rows per second) per node for each TTL job. Use 0 to signify no rate limit.</td><td>Serverless/Dedicated/Self-Hosted</td></tr>

pkg/sql/conn_executor_exec.go

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -3664,6 +3664,7 @@ func (ex *connExecutor) execStmtInNoTxnState(
36643664
ex.txnIsolationLevelToKV(ctx, s.Modes.Isolation),
36653665
ex.omitInRangefeeds(),
36663666
ex.bufferedWritesEnabled(ctx),
3667+
ex.rng.internal,
36673668
)
36683669
case *tree.ShowCommitTimestamp:
36693670
return ex.execShowCommitTimestampInNoTxnState(ctx, s, res)
@@ -3698,6 +3699,7 @@ func (ex *connExecutor) execStmtInNoTxnState(
36983699
ex.txnIsolationLevelToKV(ctx, tree.UnspecifiedIsolation),
36993700
ex.omitInRangefeeds(),
37003701
ex.bufferedWritesEnabled(ctx),
3702+
ex.rng.internal,
37013703
)
37023704
}
37033705
}
@@ -3732,6 +3734,7 @@ func (ex *connExecutor) beginImplicitTxn(
37323734
ex.txnIsolationLevelToKV(ctx, tree.UnspecifiedIsolation),
37333735
ex.omitInRangefeeds(),
37343736
ex.bufferedWritesEnabled(ctx),
3737+
ex.rng.internal,
37353738
)
37363739
}
37373740

pkg/sql/conn_fsm.go

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -12,6 +12,7 @@
1212
package sql
1313

1414
import (
15+
"math/rand"
1516
"time"
1617

1718
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/isolation"
@@ -121,6 +122,7 @@ type eventTxnStartPayload struct {
121122
isoLevel isolation.Level
122123
omitInRangefeeds bool
123124
bufferedWritesEnabled bool
125+
rng *rand.Rand
124126
}
125127

126128
// makeEventTxnStartPayload creates an eventTxnStartPayload.
@@ -134,6 +136,7 @@ func makeEventTxnStartPayload(
134136
isoLevel isolation.Level,
135137
omitInRangefeeds bool,
136138
bufferedWritesEnabled bool,
139+
rng *rand.Rand,
137140
) eventTxnStartPayload {
138141
return eventTxnStartPayload{
139142
pri: pri,
@@ -145,6 +148,7 @@ func makeEventTxnStartPayload(
145148
isoLevel: isoLevel,
146149
omitInRangefeeds: omitInRangefeeds,
147150
bufferedWritesEnabled: bufferedWritesEnabled,
151+
rng: rng,
148152
}
149153
}
150154

@@ -601,6 +605,7 @@ func noTxnToOpen(args fsm.Args) error {
601605
payload.isoLevel,
602606
payload.omitInRangefeeds,
603607
payload.bufferedWritesEnabled,
608+
payload.rng,
604609
)
605610
ts.setAdvanceInfo(
606611
advCode,

pkg/sql/exec_util.go

Lines changed: 20 additions & 14 deletions
Original file line numberDiff line numberDiff line change
@@ -260,22 +260,28 @@ var SecondaryTenantScatterEnabled = settings.RegisterBoolSetting(
260260
settings.WithName("sql.virtual_cluster.feature_access.manual_range_scatter.enabled"),
261261
)
262262

263-
// traceTxnThreshold can be used to log SQL transactions that take
264-
// longer than duration to complete. For example, traceTxnThreshold=1s
265-
// will log the trace for any transaction that takes 1s or longer. To
266-
// log traces for all transactions use traceTxnThreshold=1ns. Note
267-
// that any positive duration will enable tracing and will slow down
268-
// all execution because traces are gathered for all transactions even
269-
// if they are not output.
270-
var traceTxnThreshold = settings.RegisterDurationSetting(
263+
// TraceTxnThreshold logs SQL transactions exceeding a duration, captured via
264+
// probabilistic tracing. For example, with `sql.trace.txn.percent` set to 0.5,
265+
// 50% of transactions are traced, and those exceeding this threshold are
266+
// logged.
267+
var TraceTxnThreshold = settings.RegisterDurationSetting(
271268
settings.ApplicationLevel,
272269
"sql.trace.txn.enable_threshold",
273-
"enables tracing on all transactions; transactions open for longer than "+
274-
"this duration will have their trace logged (set to 0 to disable); "+
275-
"note that enabling this may have a negative performance impact; "+
276-
"this setting is coarser-grained than sql.trace.stmt.enable_threshold "+
277-
"because it applies to all statements within a transaction as well as "+
278-
"client communication (e.g. retries)", 0,
270+
"enables transaction traces for transactions exceeding this duration, used "+
271+
"with `sql.trace.txn.sample_rate`",
272+
0,
273+
settings.WithPublic)
274+
275+
// TraceTxnSampleRate Enables probabilistic transaction tracing.
276+
var TraceTxnSampleRate = settings.RegisterFloatSetting(
277+
settings.ApplicationLevel,
278+
"sql.trace.txn.sample_rate",
279+
"enables probabilistic transaction tracing. It should be used in conjunction "+
280+
"with `sql.trace.txn.enable_threshold`. A percentage of transactions between 0 and 1.0 "+
281+
"will have tracing enabled, and only those which exceed the configured "+
282+
"threshold will be logged.",
283+
1.0,
284+
settings.NonNegativeFloatWithMaximum(1.0),
279285
settings.WithPublic)
280286

281287
// TraceStmtThreshold is identical to traceTxnThreshold except it applies to

pkg/sql/explain_bundle_test.go

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -178,7 +178,9 @@ CREATE TABLE users(id UUID DEFAULT gen_random_uuid() PRIMARY KEY, promo_id INT R
178178

179179
t.Run("basic when tracing already enabled", func(t *testing.T) {
180180
r.Exec(t, "SET CLUSTER SETTING sql.trace.txn.enable_threshold='100ms';")
181+
r.Exec(t, "SET CLUSTER SETTING sql.trace.txn.sample_rate='1.0';")
181182
defer r.Exec(t, "SET CLUSTER SETTING sql.trace.txn.enable_threshold='0ms';")
183+
defer r.Exec(t, "SET CLUSTER SETTING sql.trace.txn.sample_rate='0.0';")
182184
rows := r.QueryStr(t, "EXPLAIN ANALYZE (DEBUG) SELECT * FROM abc WHERE c=1")
183185
checkBundle(
184186
t, fmt.Sprint(rows), "public.abc", nil, false, /* expectErrors */

pkg/sql/internal.go

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -480,6 +480,7 @@ func (ie *InternalExecutor) newConnExecutorWithTxn(
480480
// TODO(yuzefovich): re-evaluate whether we want to allow buffered
481481
// writes for internal executor.
482482
false, /* bufferedWritesEnabled */
483+
ex.rng.internal,
483484
)
484485

485486
// Modify the Collection to match the parent executor's Collection.

pkg/sql/trace_test.go

Lines changed: 91 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -10,6 +10,7 @@ import (
1010
gosql "database/sql"
1111
"fmt"
1212
"net/url"
13+
"regexp"
1314
"sort"
1415
"strings"
1516
"testing"
@@ -20,13 +21,16 @@ import (
2021
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
2122
"github.com/cockroachdb/cockroach/pkg/sql"
2223
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descs"
24+
"github.com/cockroachdb/cockroach/pkg/testutils"
2325
"github.com/cockroachdb/cockroach/pkg/testutils/pgurlutils"
2426
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
2527
"github.com/cockroachdb/cockroach/pkg/testutils/skip"
2628
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
2729
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
2830
"github.com/cockroachdb/cockroach/pkg/util/log"
31+
"github.com/cockroachdb/cockroach/pkg/util/log/logtestutils"
2932
"github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb"
33+
"github.com/cockroachdb/errors"
3034
"github.com/cockroachdb/logtags"
3135
"github.com/stretchr/testify/require"
3236
)
@@ -584,3 +588,90 @@ func TestStatementThreshold(t *testing.T) {
584588
r.Exec(t, "select 1")
585589
// TODO(andrei): check the logs for traces somehow.
586590
}
591+
592+
func TestTraceTxnSampleRateAndThreshold(t *testing.T) {
593+
defer leaktest.AfterTest(t)()
594+
defer log.Scope(t).Close(t)
595+
596+
ctx := context.Background()
597+
settings := cluster.MakeTestingClusterSettings()
598+
599+
s, db, _ := serverutils.StartServer(t, base.TestServerArgs{
600+
Settings: settings,
601+
})
602+
defer s.Stopper().Stop(ctx)
603+
604+
appLogsSpy := logtestutils.NewLogSpy(
605+
t,
606+
// This string match is constructed from the log.SqlExec.Infof format
607+
// string found in conn_executor_exec.go:logTraceAboveThreshold
608+
logtestutils.MatchesF("exceeding threshold of"),
609+
)
610+
cleanup := log.InterceptWith(ctx, appLogsSpy)
611+
defer cleanup()
612+
613+
for _, tc := range []struct {
614+
name string
615+
sampleRate float64
616+
threshold time.Duration
617+
exptToTraceEventually bool
618+
}{
619+
{
620+
name: "no sample rate and no threshold",
621+
sampleRate: 0.0,
622+
threshold: 0 * time.Nanosecond,
623+
exptToTraceEventually: false,
624+
},
625+
{
626+
name: "sample rate 1.0 and threshold 1ns should trace",
627+
sampleRate: 1.0,
628+
threshold: 1 * time.Nanosecond,
629+
exptToTraceEventually: true,
630+
},
631+
{
632+
name: "sample rate 0.0 and threshold 1ns should not trace",
633+
sampleRate: 0.0,
634+
threshold: 1 * time.Nanosecond,
635+
exptToTraceEventually: false,
636+
},
637+
{
638+
name: "sample rate 1.0 and threshold 0ns should not trace",
639+
sampleRate: 1.0,
640+
threshold: 0 * time.Nanosecond,
641+
exptToTraceEventually: false,
642+
},
643+
{
644+
name: "sample rate 0.5 and threshold 1ns should trace eventually",
645+
sampleRate: 0.5,
646+
threshold: 1 * time.Nanosecond,
647+
exptToTraceEventually: true,
648+
},
649+
} {
650+
t.Run(tc.name, func(t *testing.T) {
651+
sql.TraceTxnThreshold.Override(ctx, &settings.SV, tc.threshold)
652+
sql.TraceTxnSampleRate.Override(ctx, &settings.SV, tc.sampleRate)
653+
log.FlushAllSync()
654+
appLogsSpy.Reset()
655+
r := sqlutils.MakeSQLRunner(db)
656+
657+
if tc.exptToTraceEventually {
658+
testutils.SucceedsSoon(t, func() error {
659+
r.Exec(t, "SELECT pg_sleep(0.01)")
660+
log.FlushAllSync()
661+
if !appLogsSpy.Has(logtestutils.MatchesF(regexp.QuoteMeta("ExecStmt: SELECT pg_sleep(0.01)"))) {
662+
return errors.New("no sql txn log found (tracing did not happen)")
663+
}
664+
return nil
665+
})
666+
} else {
667+
r.Exec(t, "SELECT pg_sleep(0.01)")
668+
log.FlushAllSync()
669+
670+
spyLogs := appLogsSpy.ReadAll()
671+
if appLogsSpy.Has(logtestutils.MatchesF(regexp.QuoteMeta("ExecStmt: SELECT pg_sleep(0.01)"))) {
672+
t.Fatalf("sql txn log found (tracing happened when it should not have): %v", spyLogs)
673+
}
674+
}
675+
})
676+
}
677+
}

pkg/sql/txn_state.go

Lines changed: 16 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -7,6 +7,7 @@ package sql
77

88
import (
99
"context"
10+
"math/rand"
1011
"sync/atomic"
1112
"time"
1213

@@ -94,6 +95,11 @@ type txnState struct {
9495
// txn context.
9596
txnCancelFn context.CancelFunc
9697

98+
// shouldRecord is used to indicate whether this transaction should record a
99+
// trace. This is set to true if we have a positive sample rate and a
100+
// positive duration trigger for logging.
101+
shouldRecord bool
102+
97103
// recordingThreshold, is not zero, indicates that sp is recording and that
98104
// the recording should be dumped to the log if execution of the transaction
99105
// took more than this.
@@ -193,6 +199,7 @@ func (ts *txnState) resetForNewSQLTxn(
193199
isoLevel isolation.Level,
194200
omitInRangefeeds bool,
195201
bufferedWritesEnabled bool,
202+
rng *rand.Rand,
196203
) (txnID uuid.UUID) {
197204
// Reset state vars to defaults.
198205
ts.sqlTimestamp = sqlTimestamp
@@ -206,8 +213,12 @@ func (ts *txnState) resetForNewSQLTxn(
206213
alreadyRecording := tranCtx.sessionTracing.Enabled()
207214
ctx, cancelFn := context.WithCancel(connCtx)
208215
var sp *tracing.Span
209-
duration := traceTxnThreshold.Get(&tranCtx.settings.SV)
210-
if alreadyRecording || duration > 0 {
216+
duration := TraceTxnThreshold.Get(&tranCtx.settings.SV)
217+
218+
sampleRate := TraceTxnSampleRate.Get(&tranCtx.settings.SV)
219+
ts.shouldRecord = sampleRate > 0 && duration > 0 && rng.Float64() < sampleRate
220+
221+
if alreadyRecording || ts.shouldRecord {
211222
ts.Ctx, sp = tracing.EnsureChildSpan(ctx, tranCtx.tracer, opName,
212223
tracing.WithRecording(tracingpb.RecordingVerbose))
213224
} else if ts.testingForceRealTracingSpans {
@@ -220,7 +231,7 @@ func (ts *txnState) resetForNewSQLTxn(
220231
sp.SetTag("implicit", attribute.StringValue("true"))
221232
}
222233

223-
if !alreadyRecording && (duration > 0) {
234+
if !alreadyRecording && ts.shouldRecord {
224235
ts.recordingThreshold = duration
225236
ts.recordingStart = timeutil.Now()
226237
}
@@ -285,7 +296,7 @@ func (ts *txnState) finishSQLTxn() (txnID uuid.UUID, commitTimestamp hlc.Timesta
285296
ts.mon.Stop(ts.Ctx)
286297
sp := tracing.SpanFromContext(ts.Ctx)
287298

288-
if ts.recordingThreshold > 0 {
299+
if ts.shouldRecord {
289300
if elapsed := timeutil.Since(ts.recordingStart); elapsed >= ts.recordingThreshold {
290301
logTraceAboveThreshold(ts.Ctx,
291302
sp.GetRecording(sp.RecordingType()), /* recording */
@@ -302,6 +313,7 @@ func (ts *txnState) finishSQLTxn() (txnID uuid.UUID, commitTimestamp hlc.Timesta
302313
ts.txnCancelFn()
303314
}
304315
ts.Ctx = nil
316+
ts.shouldRecord = false
305317
ts.recordingThreshold = 0
306318
return func() (txnID uuid.UUID, timestamp hlc.Timestamp) {
307319
ts.mu.Lock()

pkg/sql/txn_state_test.go

Lines changed: 4 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -24,6 +24,7 @@ import (
2424
"github.com/cockroachdb/cockroach/pkg/util/log"
2525
"github.com/cockroachdb/cockroach/pkg/util/metric"
2626
"github.com/cockroachdb/cockroach/pkg/util/mon"
27+
"github.com/cockroachdb/cockroach/pkg/util/randutil"
2728
"github.com/cockroachdb/cockroach/pkg/util/stop"
2829
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
2930
"github.com/cockroachdb/cockroach/pkg/util/tracing"
@@ -213,6 +214,7 @@ func TestTransitions(t *testing.T) {
213214
ctx := context.Background()
214215
stopper := stop.NewStopper()
215216
defer stopper.Stop(ctx)
217+
rng, _ := randutil.NewTestRand()
216218
dummyRewCap := rewindCapability{rewindPos: CmdPos(12)}
217219
testCon := makeTestContext(stopper)
218220
tranCtx := transitionCtx{
@@ -282,7 +284,7 @@ func TestTransitions(t *testing.T) {
282284
ev: eventTxnStart{ImplicitTxn: fsm.True},
283285
evPayload: makeEventTxnStartPayload(pri, tree.ReadWrite, timeutil.Now(),
284286
nil /* historicalTimestamp */, tranCtx, sessiondatapb.Normal, isolation.Serializable,
285-
false /* omitInRangefeeds */, false, /* bufferedWritesEnabled */
287+
false /* omitInRangefeeds */, false /* bufferedWritesEnabled */, rng,
286288
),
287289
expState: stateOpen{ImplicitTxn: fsm.True, WasUpgraded: fsm.False},
288290
expAdv: expAdvance{
@@ -309,7 +311,7 @@ func TestTransitions(t *testing.T) {
309311
ev: eventTxnStart{ImplicitTxn: fsm.False},
310312
evPayload: makeEventTxnStartPayload(pri, tree.ReadWrite, timeutil.Now(),
311313
nil /* historicalTimestamp */, tranCtx, sessiondatapb.Normal, isolation.Serializable,
312-
false /* omitInRangefeeds */, false, /* bufferedWritesEnabled */
314+
false /* omitInRangefeeds */, false /* bufferedWritesEnabled */, rng,
313315
),
314316
expState: stateOpen{ImplicitTxn: fsm.False, WasUpgraded: fsm.False},
315317
expAdv: expAdvance{

0 commit comments

Comments
 (0)