Skip to content

Commit d7110e8

Browse files
committed
kvcoord,concurrency: enable buffered writes by default
We would like to do this for as much of the 25.4 release cycle as possible. Buffered writes for weak isolation is still disabled as there is a long tail of test failures to work through. A few of those test failures are fixed in this PR. Epic: none Release note: None
1 parent 639c2e5 commit d7110e8

File tree

7 files changed

+50
-21
lines changed

7 files changed

+50
-21
lines changed

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -100,7 +100,7 @@ kv.transaction.max_intents_bytes integer 4194304 maximum number of bytes used to
100100
kv.transaction.max_refresh_spans_bytes integer 4194304 maximum number of bytes used to track refresh spans in serializable transactions application
101101
kv.transaction.randomized_anchor_key.enabled boolean false dictates whether a transactions anchor key is randomized or not application
102102
kv.transaction.reject_over_max_intents_budget.enabled boolean false if set, transactions that exceed their lock tracking budget (kv.transaction.max_intents_bytes) are rejected instead of having their lock spans imprecisely compressed application
103-
kv.transaction.write_buffering.enabled boolean false if enabled, transactional writes are buffered on the client application
103+
kv.transaction.write_buffering.enabled boolean true if enabled, transactional writes are buffered on the client application
104104
kv.transaction.write_buffering.max_buffer_size byte size 4.0 MiB if non-zero, defines that maximum size of the buffer that will be used to buffer transactional writes per-transaction application
105105
kv.transaction.write_pipelining.locking_reads.enabled boolean true if enabled, transactional locking reads are pipelined through Raft consensus application
106106
kv.transaction.write_pipelining.ranged_writes.enabled boolean true if enabled, transactional ranged writes are pipelined through Raft consensus application

docs/generated/settings/settings.html

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -129,7 +129,7 @@
129129
<tr><td><div id="setting-kv-transaction-max-refresh-spans-bytes" class="anchored"><code>kv.transaction.max_refresh_spans_bytes</code></div></td><td>integer</td><td><code>4194304</code></td><td>maximum number of bytes used to track refresh spans in serializable transactions</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
130130
<tr><td><div id="setting-kv-transaction-randomized-anchor-key-enabled" class="anchored"><code>kv.transaction.randomized_anchor_key.enabled</code></div></td><td>boolean</td><td><code>false</code></td><td>dictates whether a transactions anchor key is randomized or not</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
131131
<tr><td><div id="setting-kv-transaction-reject-over-max-intents-budget-enabled" class="anchored"><code>kv.transaction.reject_over_max_intents_budget.enabled</code></div></td><td>boolean</td><td><code>false</code></td><td>if set, transactions that exceed their lock tracking budget (kv.transaction.max_intents_bytes) are rejected instead of having their lock spans imprecisely compressed</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
132-
<tr><td><div id="setting-kv-transaction-write-buffering-enabled" class="anchored"><code>kv.transaction.write_buffering.enabled</code></div></td><td>boolean</td><td><code>false</code></td><td>if enabled, transactional writes are buffered on the client</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
132+
<tr><td><div id="setting-kv-transaction-write-buffering-enabled" class="anchored"><code>kv.transaction.write_buffering.enabled</code></div></td><td>boolean</td><td><code>true</code></td><td>if enabled, transactional writes are buffered on the client</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
133133
<tr><td><div id="setting-kv-transaction-write-buffering-max-buffer-size" class="anchored"><code>kv.transaction.write_buffering.max_buffer_size</code></div></td><td>byte size</td><td><code>4.0 MiB</code></td><td>if non-zero, defines that maximum size of the buffer that will be used to buffer transactional writes per-transaction</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
134134
<tr><td><div id="setting-kv-transaction-write-pipelining-locking-reads-enabled" class="anchored"><code>kv.transaction.write_pipelining.locking_reads.enabled</code></div></td><td>boolean</td><td><code>true</code></td><td>if enabled, transactional locking reads are pipelined through Raft consensus</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
135135
<tr><td><div id="setting-kv-transaction-write-pipelining-ranged-writes-enabled" class="anchored"><code>kv.transaction.write_pipelining.ranged_writes.enabled</code></div></td><td>boolean</td><td><code>true</code></td><td>if enabled, transactional ranged writes are pipelined through Raft consensus</td><td>Serverless/Dedicated/Self-Hosted</td></tr>

pkg/ccl/logictestccl/testdata/logic_test/read_committed

Lines changed: 20 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -515,6 +515,11 @@ GRANT ALL ON t TO testuser2;
515515

516516
user testuser
517517

518+
# The following transaction looks at the lock table, which makes it sensitive to whether or not
519+
# replicated locks are buffered.
520+
statement ok
521+
SET kv_transaction_buffered_writes_enabled=false
522+
518523
query I
519524
BEGIN ISOLATION LEVEL READ COMMITTED;
520525
SELECT * FROM t WHERE a = 1 FOR SHARE;
@@ -532,6 +537,9 @@ database_name schema_name table_name lock_key_pretty lock_strength durabili
532537
# 1: Shared
533538
# 2: None
534539

540+
statement ok
541+
SET kv_transaction_buffered_writes_enabled=false
542+
535543
query I
536544
BEGIN ISOLATION LEVEL READ COMMITTED;
537545
SELECT * FROM t FOR UPDATE SKIP LOCKED
@@ -544,23 +552,35 @@ user testuser2
544552
# 1: Shared
545553
# 2: Exclusive
546554

555+
statement ok
556+
SET kv_transaction_buffered_writes_enabled=false
557+
547558
query I
548559
BEGIN ISOLATION LEVEL READ COMMITTED;
549560
SELECT * FROM t FOR SHARE SKIP LOCKED;
550561
COMMIT;
551562
----
552563
1
553564

565+
statement ok
566+
RESET kv_transaction_buffered_writes_enabled
567+
554568
user root
555569

556570
statement ok
557571
COMMIT
558572

573+
statement ok
574+
RESET kv_transaction_buffered_writes_enabled
575+
559576
user testuser
560577

561578
statement ok
562579
COMMIT
563580

581+
statement ok
582+
RESET kv_transaction_buffered_writes_enabled
583+
564584
subtest end
565585

566586
subtest regression_130661

pkg/ccl/testccl/sqlccl/BUILD.bazel

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -34,6 +34,7 @@ go_test(
3434
"//pkg/kv/kvpb",
3535
"//pkg/kv/kvserver",
3636
"//pkg/kv/kvserver/concurrency/isolation",
37+
"//pkg/kv/kvserver/concurrency/lock",
3738
"//pkg/kv/kvserver/protectedts/ptpb",
3839
"//pkg/multitenant/mtinfopb",
3940
"//pkg/roachpb",

pkg/ccl/testccl/sqlccl/read_committed_test.go

Lines changed: 24 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -19,6 +19,7 @@ import (
1919
"github.com/cockroachdb/cockroach/pkg/kv/kvpb"
2020
"github.com/cockroachdb/cockroach/pkg/kv/kvserver"
2121
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/isolation"
22+
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock"
2223
"github.com/cockroachdb/cockroach/pkg/sql"
2324
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
2425
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
@@ -153,14 +154,34 @@ func TestReadCommittedReadTimestampNotSteppedOnCommit(t *testing.T) {
153154
// Keep track of the read timestamps of the read committed transaction during
154155
// each KV operation.
155156
var txnReadTimestamps []hlc.Timestamp
157+
var txnShouldParallelCommit bool
156158
filterFunc := func(ctx context.Context, ba *kvpb.BatchRequest, _ *kvpb.BatchResponse) *kvpb.Error {
157159
if ba.Txn == nil || ba.Txn.IsoLevel != isolation.ReadCommitted {
158160
return nil
159161
}
160-
req := ba.Requests[0]
161-
method := req.GetInner().Method()
162-
if method == kvpb.ConditionalPut || (method == kvpb.EndTxn && req.GetEndTxn().IsParallelCommit()) {
162+
req := ba.Requests[len(ba.Requests)-1]
163+
164+
var recordRead bool
165+
switch req.GetInner().Method() {
166+
case kvpb.ConditionalPut:
167+
recordRead = true
168+
txnShouldParallelCommit = true
169+
case kvpb.Get:
170+
recordRead = req.GetGet().KeyLockingStrength == lock.Exclusive
171+
case kvpb.EndTxn:
172+
if txnShouldParallelCommit {
173+
recordRead = req.GetEndTxn().IsParallelCommit()
174+
} else {
175+
recordRead = true
176+
}
177+
default:
178+
recordRead = false
179+
}
180+
if recordRead {
181+
t.Logf("recording timestamp for %s", req)
163182
txnReadTimestamps = append(txnReadTimestamps, ba.Txn.ReadTimestamp)
183+
} else {
184+
t.Logf("not recording timestamp for %s", req)
164185
}
165186
return nil
166187
}

pkg/kv/kvclient/kvcoord/txn_interceptor_write_buffer.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -40,7 +40,7 @@ var BufferedWritesEnabled = settings.RegisterBoolSetting(
4040
settings.ApplicationLevel,
4141
"kv.transaction.write_buffering.enabled",
4242
"if enabled, transactional writes are buffered on the client",
43-
metamorphic.ConstantWithTestBool("kv.transaction.write_buffering.enabled", false /* defaultValue */),
43+
metamorphic.ConstantWithTestBool("kv.transaction.write_buffering.enabled", true /* defaultValue */),
4444
settings.WithPublic,
4545
)
4646

pkg/kv/kvserver/concurrency/concurrency_manager.go

Lines changed: 2 additions & 15 deletions
Original file line numberDiff line numberDiff line change
@@ -24,7 +24,6 @@ import (
2424
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
2525
"github.com/cockroachdb/cockroach/pkg/storage"
2626
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
27-
"github.com/cockroachdb/cockroach/pkg/util/buildutil"
2827
"github.com/cockroachdb/cockroach/pkg/util/debugutil"
2928
"github.com/cockroachdb/cockroach/pkg/util/hlc"
3029
"github.com/cockroachdb/cockroach/pkg/util/log"
@@ -132,13 +131,7 @@ var UnreplicatedLockReliabilityLeaseTransfer = settings.RegisterBoolSetting(
132131
settings.SystemOnly,
133132
"kv.lock_table.unreplicated_lock_reliability.lease_transfer.enabled",
134133
"whether the replica should attempt to keep unreplicated locks during lease transfers",
135-
metamorphic.ConstantWithTestBool("kv.lock_table.unreplicated_lock_reliability.lease_transfer.enabled", false),
136-
settings.WithValidateBool(func(_ *settings.Values, enabled bool) error {
137-
if enabled && !buildutil.CrdbTestBuild {
138-
return errors.Newf("kv.lock_table.unreplicated_lock_reliability.lease_transfer.enabled is not supported in production builds")
139-
}
140-
return nil
141-
}),
134+
metamorphic.ConstantWithTestBool("kv.lock_table.unreplicated_lock_reliability.lease_transfer.enabled", true),
142135
)
143136

144137
// UnreplicatedLockReliabilityMerge controls whether the replica will attempt to
@@ -147,13 +140,7 @@ var UnreplicatedLockReliabilityMerge = settings.RegisterBoolSetting(
147140
settings.SystemOnly,
148141
"kv.lock_table.unreplicated_lock_reliability.merge.enabled",
149142
"whether the replica should attempt to keep unreplicated locks during range merges",
150-
metamorphic.ConstantWithTestBool("kv.lock_table.unreplicated_lock_reliability.merge.enabled", false),
151-
settings.WithValidateBool(func(_ *settings.Values, enabled bool) error {
152-
if enabled && !buildutil.CrdbTestBuild {
153-
return errors.Newf("kv.lock_table.unreplicated_lock_reliability.merge.enabled is not supported in production builds")
154-
}
155-
return nil
156-
}),
143+
metamorphic.ConstantWithTestBool("kv.lock_table.unreplicated_lock_reliability.merge.enabled", true),
157144
)
158145

159146
var MaxLockFlushSize = settings.RegisterByteSizeSetting(

0 commit comments

Comments
 (0)