Skip to content

Commit 65900fa

Browse files
committed
kvserver: reproduce range-size backpressure for spanconfig updates
Previously, spanconfig updates could be blocked by range-size backpressure, creating a catch-22 situation where garbage collection and protected timestamp removal operations were prevented from running, which in turn prevented range splitting and alleviation of the backpressure condition. This was inadequate because it could lead to cluster-wide issues where protected timestamp cleanup operations would fail indefinitely, preventing backups, changefeeds, and other operations from completing properly. To address this, this commit adds a comprehensive test that reproduces the issue by repeatedly writing spanconfig updates for a single key until the range becomes too large to split, triggering backpressure. The test demonstrates how spanconfig updates (including protected timestamp deletions) get blocked by backpressure, preventing the very operations needed to resolve the backpressure condition. Fixes: #146982 Release note: None
1 parent 95e1d21 commit 65900fa

File tree

2 files changed

+251
-0
lines changed

2 files changed

+251
-0
lines changed

pkg/kv/kvserver/BUILD.bazel

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -303,6 +303,7 @@ go_test(
303303
"client_replica_gc_test.go",
304304
"client_replica_raft_overload_test.go",
305305
"client_replica_test.go",
306+
"client_spanconfig_backpressure_test.go",
306307
"client_spanconfigs_test.go",
307308
"client_split_burst_test.go",
308309
"client_split_test.go",
Lines changed: 250 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,250 @@
1+
// Copyright 2025 The Cockroach Authors.
2+
//
3+
// Use of this software is governed by the CockroachDB Software License
4+
// included in the /LICENSE file.
5+
6+
package kvserver_test
7+
8+
import (
9+
"context"
10+
"fmt"
11+
math "math"
12+
"strings"
13+
"testing"
14+
15+
"github.com/cockroachdb/cockroach/pkg/base"
16+
"github.com/cockroachdb/cockroach/pkg/keys"
17+
"github.com/cockroachdb/cockroach/pkg/kv/kvserver"
18+
"github.com/cockroachdb/cockroach/pkg/roachpb"
19+
"github.com/cockroachdb/cockroach/pkg/spanconfig"
20+
"github.com/cockroachdb/cockroach/pkg/testutils"
21+
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
22+
"github.com/cockroachdb/cockroach/pkg/util/hlc"
23+
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
24+
"github.com/cockroachdb/cockroach/pkg/util/log"
25+
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
26+
"github.com/cockroachdb/errors"
27+
"github.com/stretchr/testify/require"
28+
)
29+
30+
func TestSpanConfigUpdatesBlockedByRangeSizeBackpressureOnDefaultRangesWithKVAccessor(
31+
t *testing.T,
32+
) {
33+
defer leaktest.AfterTest(t)()
34+
defer log.Scope(t).Close(t)
35+
36+
ctx := context.Background()
37+
38+
const (
39+
overloadMaxRangeBytes = 64 << 20 // Set to 64 MiB, a saner value than the default of 512 MiB.
40+
overloadMinRangeBytes = 16 << 10
41+
numWrites = 64 << 10 // 65,536 writes, this was calculated by (64 MiB / 2 KiB) * 2
42+
// (2 KiB is the size of the spanconfig record `spanConfig2Kib`).
43+
// See func exceedsMultipleOfSplitSize in /pkg/kv/kvserver/replica_metrics.go for the logic.
44+
defaultMaxBytes = 512 << 20 // Default max bytes for a range.
45+
)
46+
tc := serverutils.StartServerOnly(t, base.TestServerArgs{
47+
Knobs: base.TestingKnobs{
48+
Store: &kvserver.StoreTestingKnobs{
49+
DisableMergeQueue: true,
50+
// Keep split queue enabled to see natural behaviour.
51+
}},
52+
})
53+
defer tc.Stopper().Stop(ctx)
54+
55+
store, err := tc.GetStores().(*kvserver.Stores).GetStore(tc.GetFirstStoreID())
56+
require.NoError(t, err)
57+
58+
waitForSpanConfig := func(t *testing.T, tc serverutils.TestServerInterface, tablePrefix roachpb.Key, exp int64) {
59+
testutils.SucceedsSoon(t, func() error {
60+
_, r := getFirstStoreReplica(t, tc, tablePrefix)
61+
conf, err := r.LoadSpanConfig(ctx)
62+
if err != nil {
63+
return err
64+
}
65+
if log.V(1) {
66+
log.Dev.Infof(ctx, "RangeMaxBytes for tablePrefix %s: %d\n", tablePrefix, conf.RangeMaxBytes)
67+
}
68+
if conf.RangeMaxBytes != exp {
69+
return fmt.Errorf("expected %d, got %d", exp, conf.RangeMaxBytes)
70+
}
71+
return nil
72+
})
73+
}
74+
75+
// System spanconfig to set the range max bytes to 64 MiB.
76+
systemSpanConfig := roachpb.SpanConfig{
77+
RangeMaxBytes: 64 << 20, // 64 MiB.
78+
RangeMinBytes: 16 << 20, // 16 MiB.
79+
}
80+
81+
// This is a fat spanconfig with all fields set to maximum int64 and int32 values.
82+
// This is done to have a spanconfig that is large enough to trigger backpressure
83+
// without having to write a million records.
84+
// Having this be 2 KiB gives us (64 << 20 / 2 << 10) * 2 = 65,536 writes.
85+
// See func exceedsMultipleOfSplitSize in /pkg/kv/kvserver/replica_metrics.go for the logic.
86+
spanConfig2KiB := roachpb.SpanConfig{ // 2078 bytes ~ 2 KiB.
87+
RangeMaxBytes: math.MaxInt64, // Maximum int64 value.
88+
RangeMinBytes: math.MaxInt64, // Maximum int64 value.
89+
GCPolicy: roachpb.GCPolicy{
90+
TTLSeconds: math.MaxInt32, // Maximum int32 value.
91+
ProtectionPolicies: []roachpb.ProtectionPolicy{
92+
{
93+
ProtectedTimestamp: hlc.MaxTimestamp,
94+
},
95+
{
96+
ProtectedTimestamp: hlc.MaxTimestamp,
97+
},
98+
},
99+
},
100+
NumReplicas: math.MaxInt32, // Maximum int32 value.
101+
GlobalReads: true,
102+
NumVoters: math.MaxInt32,
103+
VoterConstraints: []roachpb.ConstraintsConjunction{
104+
{
105+
Constraints: []roachpb.Constraint{
106+
{Key: "max_key", Value: strings.Repeat("x", 1000)}, // Very long constraint value.
107+
},
108+
},
109+
},
110+
LeasePreferences: []roachpb.LeasePreference{
111+
{
112+
Constraints: []roachpb.Constraint{
113+
{Key: "max_key", Value: strings.Repeat("y", 1000)}, // Very long constraint value.
114+
},
115+
},
116+
},
117+
}
118+
119+
configBytes, err := protoutil.Marshal(&spanConfig2KiB)
120+
require.NoError(t, err)
121+
122+
log.Dev.Infof(ctx, "Size of configBytes: %d bytes (%d KiB)\n", len(configBytes), len(configBytes)>>10)
123+
124+
spanConfigTablePrefix := keys.SystemSQLCodec.TablePrefix(keys.SpanConfigurationsTableID)
125+
126+
log.Dev.Infof(ctx, "Targeting span_configurations table at key: %s (table ID %d)\n",
127+
spanConfigTablePrefix, keys.SpanConfigurationsTableID)
128+
129+
log.Dev.Infof(ctx, "Configuring span_configurations table with custom zone settings...\n")
130+
131+
testKey, err := tc.ScratchRange()
132+
require.NoError(t, err)
133+
134+
testutils.SucceedsSoon(t, func() error {
135+
repl := store.LookupReplica(roachpb.RKey(testKey))
136+
if got := repl.GetMaxBytes(ctx); got != defaultMaxBytes {
137+
return errors.Errorf("range max bytes values did not start at %d; got %d", defaultMaxBytes, got)
138+
}
139+
return nil
140+
})
141+
142+
tableSpan := roachpb.Span{Key: spanConfigTablePrefix, EndKey: spanConfigTablePrefix.PrefixEnd()}
143+
144+
target := spanconfig.MakeTargetFromSpan(tableSpan)
145+
record, err := spanconfig.MakeRecord(target, systemSpanConfig)
146+
require.NoError(t, err)
147+
148+
kvaccessor := tc.SpanConfigKVAccessor().(spanconfig.KVAccessor)
149+
150+
err = kvaccessor.UpdateSpanConfigRecords(ctx, []spanconfig.Target{target}, []spanconfig.Record{record}, hlc.MinTimestamp, hlc.MaxTimestamp)
151+
require.NoError(t, err)
152+
153+
waitForSpanConfig(t, tc, spanConfigTablePrefix, overloadMaxRangeBytes)
154+
155+
// Wait for the zone configuration to be applied.
156+
log.Dev.Infof(ctx, "Waiting for zone configuration to be applied...\n")
157+
testutils.SucceedsSoon(t, func() error {
158+
repl := store.LookupReplica(keys.MustAddr(spanConfigTablePrefix))
159+
if repl == nil {
160+
return fmt.Errorf("replica not found")
161+
}
162+
conf, err := repl.LoadSpanConfig(ctx)
163+
if err != nil {
164+
return err
165+
}
166+
if conf.RangeMaxBytes != overloadMaxRangeBytes {
167+
return fmt.Errorf("expected RangeMaxBytes %d, got %d", overloadMaxRangeBytes, conf.RangeMaxBytes)
168+
}
169+
return nil
170+
})
171+
172+
log.Dev.Infof(ctx, "Zone configuration successfully applied!\n")
173+
174+
// Check if the range is using our custom config.
175+
repl := store.LookupReplica(keys.MustAddr(spanConfigTablePrefix))
176+
if repl != nil {
177+
conf, err := repl.LoadSpanConfig(ctx)
178+
if err != nil {
179+
log.Dev.Infof(ctx, "Error loading span config: %v\n", err)
180+
} else {
181+
log.Dev.Infof(ctx, "Current range config - RangeMaxBytes: %d bytes (%d MiB), RangeMinBytes: %d bytes (%d MiB)\n",
182+
conf.RangeMaxBytes, conf.RangeMaxBytes>>20,
183+
conf.RangeMinBytes, conf.RangeMinBytes>>20)
184+
}
185+
186+
stats := repl.GetMVCCStats()
187+
log.Dev.Infof(ctx, "Current range size: %d bytes (%d MiB)\n", stats.Total(), stats.Total()>>20)
188+
}
189+
190+
log.Dev.Infof(ctx, "Targeting span_configurations table at key: %s (table ID %d)\n",
191+
spanConfigTablePrefix, keys.SpanConfigurationsTableID)
192+
log.Dev.Infof(ctx, "Direct KV writes to span_configurations table range %d times...\n", numWrites)
193+
194+
// Create a single target for the scratch range (this will be stored in system.span_configurations)
195+
testTargetKey := testKey // Use the scratch range key we got earlier.
196+
testTarget := spanconfig.MakeTargetFromSpan(roachpb.Span{
197+
Key: testTargetKey,
198+
EndKey: testTargetKey.PrefixEnd(),
199+
})
200+
201+
// Create a record with the span configuration.
202+
testRecord, err := spanconfig.MakeRecord(testTarget, spanConfig2KiB)
203+
require.NoError(t, err)
204+
205+
// Write span configurations using KVAccessor.
206+
// We expect this to fail due to backpressure.
207+
var i int
208+
for i = 0; i < numWrites; i++ {
209+
// Use KVAccessor to update span configurations.
210+
err = kvaccessor.UpdateSpanConfigRecords(ctx, nil, []spanconfig.Record{testRecord}, hlc.MinTimestamp, hlc.MaxTimestamp)
211+
if log.V(1) {
212+
log.Dev.Infof(ctx, "KVAccessor write %d/%d: target=%q\n", i+1, numWrites, testTargetKey)
213+
}
214+
if err != nil {
215+
log.Dev.Infof(ctx, "ERROR! BREAKING OUT OF LOOP, numWrites successful: %d, error: %+v\n", i, err)
216+
break
217+
}
218+
}
219+
220+
// Assert that the operation failed due to backpressure.
221+
require.Error(t, err, "Expected span config writes to fail due to backpressure, but they succeeded")
222+
log.Dev.Infof(ctx, "Verified that span config writes fail due to backpressure: %v\n", err)
223+
224+
log.Dev.Infof(ctx, "Completed %d direct KV writes\n", i)
225+
226+
repl = store.LookupReplica(keys.MustAddr(spanConfigTablePrefix))
227+
if repl != nil {
228+
stats := repl.GetMVCCStats()
229+
log.Dev.Infof(ctx, "Range size after all writes: %d bytes (KeyCount: %d, LiveBytes: %d)\n", stats.Total(), stats.KeyCount, stats.LiveBytes)
230+
}
231+
232+
smallSpanConfig := roachpb.SpanConfig{
233+
GCPolicy: roachpb.GCPolicy{
234+
TTLSeconds: 0,
235+
},
236+
}
237+
238+
smallSpancofnRecord, err := spanconfig.MakeRecord(testTarget, smallSpanConfig)
239+
require.NoError(t, err)
240+
241+
log.Dev.Infof(ctx, "Testing one more write with a small span config...\n")
242+
243+
smallSpancofnRecordWriteErr := kvaccessor.UpdateSpanConfigRecords(ctx, []spanconfig.Target{testTarget}, []spanconfig.Record{smallSpancofnRecord}, hlc.MinTimestamp, hlc.MaxTimestamp)
244+
if smallSpancofnRecordWriteErr != nil {
245+
log.Dev.Infof(ctx, "ERROR: smallSpancofnRecord write failed: %v\n", smallSpancofnRecordWriteErr)
246+
}
247+
require.Error(t, smallSpancofnRecordWriteErr, "Expected smallSpancofnRecord write to succeed")
248+
log.Dev.Infof(ctx, "SUCCESS: smallSpancofnRecord write failed as expected; still getting backpressure\n")
249+
250+
}

0 commit comments

Comments
 (0)