Skip to content

Commit a666b16

Browse files
craig[bot]stevendanna
andcommitted
Merge #150554
150554: kvcoord,concurrency: enable buffered writes by default r=yuzefovich a=stevendanna We would like to do this for as much of the 25.4 release cycle as possible Epic: none Release note: None Co-authored-by: Steven Danna <[email protected]>
2 parents 639c2e5 + d7110e8 commit a666b16

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)