Skip to content

Commit 6d30ef4

Browse files
craig[bot]kyle-a-wong
andcommitted
Merge #148542
148542: sql: add probabilistic transaction tracing r=alyshanjahani-crl a=dhartunian Previously, tracing transactions in CRDB could be enabled using a single cluster setting: `sql.trace.txn.enable_threshold`, which, if it was non-zero, would enable tracing on all transactions and only keep and emit the traces for transactions which exceeded the execution time threshold. This has proven to be difficult to use as it reuquires paying tracing cost on the entire workload. This change modifies the transaction tracing capabilities of CRDB to incorporate a sample rate. A new cluster setting is introduced, named `sql.trace.txn.sample_rate` which is zero by default. When set to a positive number between 0 and 1, this will set the probability of a transaction having tracing enabled. For tracing to emit a trace for a transaction, both settings will now need to be set and will need to both trigger for successful output to the `SQL_EXEC` logging channel. Transactions first pass through the sample rate filter which governs whether a trace is captured, then they will be evaluated through the `enable_threshold` which governs whether the trace is emitted to the logs. Epic: None Resolves: CRDB-51662 Release note (ops change, sql change): In order to selectively capture traces for transactions running in an active workload without haing to capture them via statement diagnostic bundles, customers can now use the `sql.trace.txn.sample_rate` cluster setting to enable tracing for a fraction of their workload. The `sql.trace.txn.enable_threshold` will still need to be set to a positive value to provide a filter for how slow a transaction needs to be after being sampled to merit emitting a trace. Traces are emitted to the `SQL_EXEC` logging channel. Co-authored-by: Kyle Wong <[email protected]>
2 parents dd7f87c + 4bc28d1 commit 6d30ef4

File tree

11 files changed

+156
-26
lines changed

11 files changed

+156
-26
lines changed

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

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -388,7 +388,8 @@ sql.temp_object_cleaner.cleanup_interval duration 30m0s how often to clean up or
388388
sql.temp_object_cleaner.wait_interval duration 30m0s how long after creation a temporary object will be cleaned up application
389389
sql.log.all_statements.enabled (alias: sql.trace.log_statement_execute) boolean false set to true to enable logging of all executed statements application
390390
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
391-
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
391+
sql.trace.txn.enable_threshold duration 0s enables transaction traces for transactions exceeding this duration, used with `sql.trace.txn.sample_rate` application
392+
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
392393
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
393394
sql.ttl.default_delete_batch_size integer 100 default amount of rows to delete in a single query during a TTL job application
394395
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
@@ -343,7 +343,8 @@
343343
<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>
344344
<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>
345345
<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>
346-
<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>
346+
<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>
347+
<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>
347348
<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>
348349
<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>
349350
<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
@@ -3706,6 +3706,7 @@ func (ex *connExecutor) execStmtInNoTxnState(
37063706
ex.txnIsolationLevelToKV(ctx, s.Modes.Isolation),
37073707
ex.omitInRangefeeds(),
37083708
ex.bufferedWritesEnabled(ctx),
3709+
ex.rng.internal,
37093710
)
37103711
case *tree.ShowCommitTimestamp:
37113712
return ex.execShowCommitTimestampInNoTxnState(ctx, s, res)
@@ -3740,6 +3741,7 @@ func (ex *connExecutor) execStmtInNoTxnState(
37403741
ex.txnIsolationLevelToKV(ctx, tree.UnspecifiedIsolation),
37413742
ex.omitInRangefeeds(),
37423743
ex.bufferedWritesEnabled(ctx),
3744+
ex.rng.internal,
37433745
)
37443746
}
37453747
}
@@ -3774,6 +3776,7 @@ func (ex *connExecutor) beginImplicitTxn(
37743776
ex.txnIsolationLevelToKV(ctx, tree.UnspecifiedIsolation),
37753777
ex.omitInRangefeeds(),
37763778
ex.bufferedWritesEnabled(ctx),
3779+
ex.rng.internal,
37773780
)
37783781
}
37793782

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: 19 additions & 14 deletions
Original file line numberDiff line numberDiff line change
@@ -257,25 +257,30 @@ var SecondaryTenantScatterEnabled = settings.RegisterBoolSetting(
257257
settings.WithName("sql.virtual_cluster.feature_access.manual_range_scatter.enabled"),
258258
)
259259

260-
// traceTxnThreshold can be used to log SQL transactions that take
261-
// longer than duration to complete. For example, traceTxnThreshold=1s
262-
// will log the trace for any transaction that takes 1s or longer. To
263-
// log traces for all transactions use traceTxnThreshold=1ns. Note
264-
// that any positive duration will enable tracing and will slow down
265-
// all execution because traces are gathered for all transactions even
266-
// if they are not output.
267-
var traceTxnThreshold = settings.RegisterDurationSetting(
260+
// TraceTxnThreshold logs SQL transactions exceeding a duration, captured via
261+
// probabilistic tracing. For example, with `sql.trace.txn.percent` set to 0.5,
262+
// 50% of transactions are traced, and those exceeding this threshold are
263+
// logged.
264+
var TraceTxnThreshold = settings.RegisterDurationSetting(
268265
settings.ApplicationLevel,
269266
"sql.trace.txn.enable_threshold",
270-
"enables tracing on all transactions; transactions open for longer than "+
271-
"this duration will have their trace logged (set to 0 to disable); "+
272-
"note that enabling this may have a negative performance impact; "+
273-
"this setting is coarser-grained than sql.trace.stmt.enable_threshold "+
274-
"because it applies to all statements within a transaction as well as "+
275-
"client communication (e.g. retries)",
267+
"enables transaction traces for transactions exceeding this duration, used "+
268+
"with `sql.trace.txn.sample_rate`",
276269
0,
277270
settings.WithPublic)
278271

272+
// TraceTxnSampleRate Enables probabilistic transaction tracing.
273+
var TraceTxnSampleRate = settings.RegisterFloatSetting(
274+
settings.ApplicationLevel,
275+
"sql.trace.txn.sample_rate",
276+
"enables probabilistic transaction tracing. It should be used in conjunction "+
277+
"with `sql.trace.txn.enable_threshold`. A percentage of transactions between 0 and 1.0 "+
278+
"will have tracing enabled, and only those which exceed the configured "+
279+
"threshold will be logged.",
280+
1.0,
281+
settings.NonNegativeFloatWithMaximum(1.0),
282+
settings.WithPublic)
283+
279284
// TraceStmtThreshold is identical to traceTxnThreshold except it applies to
280285
// individual statements in a transaction. The motivation for this setting is
281286
// to be able to reduce the noise associated with a larger transaction (e.g.

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
@@ -479,6 +479,7 @@ func (ie *InternalExecutor) newConnExecutorWithTxn(
479479
// TODO(yuzefovich): re-evaluate whether we want to allow buffered
480480
// writes for internal executor.
481481
false, /* bufferedWritesEnabled */
482+
ex.rng.internal,
482483
)
483484

484485
// 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

@@ -92,6 +93,11 @@ type txnState struct {
9293
// txn context.
9394
txnCancelFn context.CancelFunc
9495

96+
// shouldRecord is used to indicate whether this transaction should record a
97+
// trace. This is set to true if we have a positive sample rate and a
98+
// positive duration trigger for logging.
99+
shouldRecord bool
100+
95101
// recordingThreshold, is not zero, indicates that sp is recording and that
96102
// the recording should be dumped to the log if execution of the transaction
97103
// took more than this.
@@ -191,6 +197,7 @@ func (ts *txnState) resetForNewSQLTxn(
191197
isoLevel isolation.Level,
192198
omitInRangefeeds bool,
193199
bufferedWritesEnabled bool,
200+
rng *rand.Rand,
194201
) (txnID uuid.UUID) {
195202
// Reset state vars to defaults.
196203
ts.sqlTimestamp = sqlTimestamp
@@ -204,8 +211,12 @@ func (ts *txnState) resetForNewSQLTxn(
204211
alreadyRecording := tranCtx.sessionTracing.Enabled()
205212
ctx, cancelFn := context.WithCancel(connCtx)
206213
var sp *tracing.Span
207-
duration := traceTxnThreshold.Get(&tranCtx.settings.SV)
208-
if alreadyRecording || duration > 0 {
214+
duration := TraceTxnThreshold.Get(&tranCtx.settings.SV)
215+
216+
sampleRate := TraceTxnSampleRate.Get(&tranCtx.settings.SV)
217+
ts.shouldRecord = sampleRate > 0 && duration > 0 && rng.Float64() < sampleRate
218+
219+
if alreadyRecording || ts.shouldRecord {
209220
ts.Ctx, sp = tracing.EnsureChildSpan(ctx, tranCtx.tracer, opName,
210221
tracing.WithRecording(tracingpb.RecordingVerbose))
211222
} else if ts.testingForceRealTracingSpans {
@@ -218,7 +229,7 @@ func (ts *txnState) resetForNewSQLTxn(
218229
sp.SetTag("implicit", attribute.StringValue("true"))
219230
}
220231

221-
if !alreadyRecording && (duration > 0) {
232+
if !alreadyRecording && ts.shouldRecord {
222233
ts.recordingThreshold = duration
223234
ts.recordingStart = timeutil.Now()
224235
}
@@ -283,7 +294,7 @@ func (ts *txnState) finishSQLTxn() (txnID uuid.UUID, commitTimestamp hlc.Timesta
283294
ts.mon.Stop(ts.Ctx)
284295
sp := tracing.SpanFromContext(ts.Ctx)
285296

286-
if ts.recordingThreshold > 0 {
297+
if ts.shouldRecord {
287298
if elapsed := timeutil.Since(ts.recordingStart); elapsed >= ts.recordingThreshold {
288299
logTraceAboveThreshold(ts.Ctx,
289300
sp.GetRecording(sp.RecordingType()), /* recording */
@@ -300,6 +311,7 @@ func (ts *txnState) finishSQLTxn() (txnID uuid.UUID, commitTimestamp hlc.Timesta
300311
ts.txnCancelFn()
301312
}
302313
ts.Ctx = nil
314+
ts.shouldRecord = false
303315
ts.recordingThreshold = 0
304316
return func() (txnID uuid.UUID, timestamp hlc.Timestamp) {
305317
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)