Skip to content

Commit 5f0b662

Browse files
committed
kvserver: reproduce range-size backpressure for spanconfig updates
Spanconfig updates could be blocked by range-size backpressure when range-size gets too large, 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 - leading to cluster wide issues. This commit adds a 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: None Release note: None
1 parent e4db2dd commit 5f0b662

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+
// TestSpanConfigUpdatesBlockedByRangeSizeBackpressureOnDefaultRanges
31+
// verifies that spanconfig updates are blocked by backpressure when the
32+
// `system.span_configurations` table range becomes full, recreating the issue.
33+
//
34+
// Test strategy:
35+
// 1. Configure `system.span_configurations` table range the smallest possible
36+
// max size (64 MiB).
37+
// 2. Write many large spanconfig records (2 KiB each) to fill up the range.
38+
// 3. Verify spanconfig updates fail due to backpressure when the range is full,
39+
// 4. This test recreates the scenario where spanconfig updates are blocked by
40+
// backpressure.
41+
func TestSpanConfigUpdatesBlockedByRangeSizeBackpressureOnDefaultRanges(t *testing.T) {
42+
defer leaktest.AfterTest(t)()
43+
defer log.Scope(t).Close(t)
44+
45+
ctx := context.Background()
46+
47+
const (
48+
overloadMaxRangeBytes = 64 << 20 // set to 64 MiB, a saner value than the default of 512 MiB
49+
overloadMinRangeBytes = 16 << 10
50+
numWrites = 64 << 10 // 65,536 writes, this was calculated by (64 MiB / 2 KiB) * 2
51+
// (2 KiB is the size of the spanconfig record `spanConfig2KiB`).
52+
// See func exceedsMultipleOfSplitSize in /pkg/kv/kvserver/replica_metrics.go for the logic.
53+
defaultMaxBytes = 512 << 20 // Default max bytes for a range.
54+
)
55+
s := serverutils.StartServerOnly(t, base.TestServerArgs{
56+
Knobs: base.TestingKnobs{
57+
Store: &kvserver.StoreTestingKnobs{
58+
DisableMergeQueue: true,
59+
// keep split queue enabled to see natural behaviour
60+
}},
61+
})
62+
defer s.Stopper().Stop(ctx)
63+
64+
store, err := s.GetStores().(*kvserver.Stores).GetStore(s.GetFirstStoreID())
65+
require.NoError(t, err)
66+
67+
waitForSpanConfig := func(t *testing.T, tc serverutils.TestServerInterface,
68+
tablePrefix roachpb.Key, expRangeMaxBytes int64) {
69+
testutils.SucceedsSoon(t, func() error {
70+
_, r := getFirstStoreReplica(t, tc, tablePrefix)
71+
conf, err := r.LoadSpanConfig(ctx)
72+
if err != nil {
73+
return err
74+
}
75+
if conf.RangeMaxBytes != expRangeMaxBytes {
76+
return fmt.Errorf("expected RangeMaxBytes %d, got %d", expRangeMaxBytes, conf.RangeMaxBytes)
77+
}
78+
return nil
79+
})
80+
}
81+
82+
spanConfigTablePrefix := keys.SystemSQLCodec.TablePrefix(keys.SpanConfigurationsTableID)
83+
84+
t.Logf("Targeting span_configurations table at key: %s (table ID %d)\n",
85+
spanConfigTablePrefix, keys.SpanConfigurationsTableID)
86+
87+
testKey, err := s.ScratchRange()
88+
require.NoError(t, err)
89+
90+
testutils.SucceedsSoon(t, func() error {
91+
repl := store.LookupReplica(roachpb.RKey(testKey))
92+
if got := repl.GetMaxBytes(ctx); got != defaultMaxBytes {
93+
return errors.Errorf(
94+
"range max bytes values did not start at %d; got %d",
95+
defaultMaxBytes, got)
96+
}
97+
return nil
98+
})
99+
100+
tableSpan := roachpb.Span{
101+
Key: spanConfigTablePrefix,
102+
EndKey: spanConfigTablePrefix.PrefixEnd(),
103+
}
104+
105+
target := spanconfig.MakeTargetFromSpan(tableSpan)
106+
107+
// System spanconfig to set the range max bytes to 64 MiB.
108+
systemSpanConfig := roachpb.SpanConfig{
109+
RangeMaxBytes: overloadMaxRangeBytes, // 64 MiB.
110+
RangeMinBytes: overloadMinRangeBytes, // 16 MiB.
111+
}
112+
record, err := spanconfig.MakeRecord(target, systemSpanConfig)
113+
require.NoError(t, err)
114+
115+
kvaccessor := s.SpanConfigKVAccessor().(spanconfig.KVAccessor)
116+
117+
err = kvaccessor.UpdateSpanConfigRecords(
118+
ctx, []spanconfig.Target{target},
119+
[]spanconfig.Record{record}, hlc.MinTimestamp, hlc.MaxTimestamp)
120+
require.NoError(t, err)
121+
122+
waitForSpanConfig(t, s, spanConfigTablePrefix, overloadMaxRangeBytes)
123+
124+
t.Logf("Zone configuration successfully applied!\n")
125+
126+
// Check if the range is using our custom config.
127+
repl := store.LookupReplica(keys.MustAddr(spanConfigTablePrefix))
128+
if repl != nil {
129+
conf, err := repl.LoadSpanConfig(ctx)
130+
if err != nil {
131+
t.Logf("Error loading span config: %v\n", err)
132+
} else {
133+
t.Logf(`Current range config - RangeMaxBytes: %d bytes
134+
(%d MiB), RangeMinBytes: %d bytes (%d MiB)\n`,
135+
conf.RangeMaxBytes, conf.RangeMaxBytes>>20,
136+
conf.RangeMinBytes, conf.RangeMinBytes>>20)
137+
}
138+
139+
stats := repl.GetMVCCStats()
140+
log.Dev.Infof(ctx, "Current range size: %d bytes (%d MiB)\n",
141+
stats.Total(), stats.Total()>>20)
142+
}
143+
144+
t.Logf("Targeting span_configurations table at key: %s (table ID %d)\n",
145+
spanConfigTablePrefix, keys.SpanConfigurationsTableID)
146+
t.Logf(`Direct KV writes to span_configurations table
147+
range %d times...\n`, numWrites)
148+
149+
// Create a single target for the scratch range (this will be stored in system.span_configurations)
150+
testTargetKey := testKey // Use the scratch range key we got earlier.
151+
testTarget := spanconfig.MakeTargetFromSpan(roachpb.Span{
152+
Key: testTargetKey,
153+
EndKey: testTargetKey.PrefixEnd(),
154+
})
155+
156+
// This is a large spanconfig with all fields set to maximum int64 and int32 values.
157+
// This is done to have a spanconfig that is large enough to trigger backpressure
158+
// without having to write a million records.
159+
// Having this be 2 KiB gives us (64 << 20 / 2 << 10) * 2 = 65,536 writes.
160+
// See func exceedsMultipleOfSplitSize in /pkg/kv/kvserver/replica_metrics.go for the logic.
161+
spanConfig2KiB := roachpb.SpanConfig{ // 2078 bytes ~ 2 KiB.
162+
RangeMaxBytes: math.MaxInt64, // maximum int64 value
163+
RangeMinBytes: math.MaxInt64, // maximum int64 value
164+
GCPolicy: roachpb.GCPolicy{
165+
TTLSeconds: math.MaxInt32, // maximum int32 value
166+
ProtectionPolicies: []roachpb.ProtectionPolicy{
167+
{
168+
ProtectedTimestamp: hlc.MaxTimestamp,
169+
},
170+
{
171+
ProtectedTimestamp: hlc.MaxTimestamp,
172+
},
173+
},
174+
},
175+
NumReplicas: math.MaxInt32, // maximum int32 value
176+
GlobalReads: true,
177+
NumVoters: math.MaxInt32,
178+
VoterConstraints: []roachpb.ConstraintsConjunction{
179+
{
180+
Constraints: []roachpb.Constraint{
181+
{Key: "max_key", Value: strings.Repeat("x", 1024)}, // very long constraint value
182+
},
183+
},
184+
},
185+
LeasePreferences: []roachpb.LeasePreference{
186+
{
187+
Constraints: []roachpb.Constraint{
188+
{Key: "max_key", Value: strings.Repeat("y", 1024)}, // very long constraint value
189+
},
190+
},
191+
},
192+
}
193+
194+
configBytes, err := protoutil.Marshal(&spanConfig2KiB)
195+
require.NoError(t, err)
196+
197+
require.GreaterOrEqual(t, len(configBytes), 2048,
198+
"spanConfig2KiB should be at least 2 KiB in size")
199+
200+
// Create a record with the span configuration.
201+
testRecord, err := spanconfig.MakeRecord(testTarget, spanConfig2KiB)
202+
require.NoError(t, err)
203+
204+
// Write span configurations using KVAccessor.
205+
// We expect this to fail due to backpressure.
206+
var i int
207+
for i = 0; i < numWrites; i++ {
208+
// Use KVAccessor to update span configurations.
209+
err = kvaccessor.UpdateSpanConfigRecords(ctx, nil,
210+
[]spanconfig.Record{testRecord}, hlc.MinTimestamp, hlc.MaxTimestamp)
211+
t.Logf("KVAccessor write %d/%d: target=%q\n", i+1, numWrites, testTargetKey)
212+
if err != nil {
213+
break
214+
}
215+
}
216+
217+
// Assert that the operation failed due to backpressure.
218+
require.Error(t, err, `Expected span config writes to fail due to
219+
backpressure, but they succeeded`)
220+
221+
t.Logf("Completed %d direct KV writes\n", i)
222+
223+
repl = store.LookupReplica(keys.MustAddr(spanConfigTablePrefix))
224+
if repl != nil {
225+
stats := repl.GetMVCCStats()
226+
t.Logf(`Range size after all writes: %d bytes (KeyCount: %d,
227+
LiveBytes: %d)\n`, stats.Total(), stats.KeyCount, stats.LiveBytes)
228+
}
229+
230+
smallSpanConfig := roachpb.SpanConfig{
231+
GCPolicy: roachpb.GCPolicy{
232+
TTLSeconds: 0,
233+
},
234+
}
235+
236+
smallSpancofnRecord, err := spanconfig.MakeRecord(testTarget, smallSpanConfig)
237+
require.NoError(t, err)
238+
239+
t.Logf("Testing one more write with a small span config...\n")
240+
241+
smallSpancofnRecordWriteErr := kvaccessor.UpdateSpanConfigRecords(ctx,
242+
[]spanconfig.Target{testTarget}, []spanconfig.Record{smallSpancofnRecord},
243+
hlc.MinTimestamp, hlc.MaxTimestamp)
244+
245+
require.Error(t, smallSpancofnRecordWriteErr, `Expected smallSpancofnRecord
246+
write to fail`)
247+
t.Logf(`SUCCESS: smallSpancofnRecord write failed as expected;
248+
still getting backpressure\n`)
249+
250+
}

0 commit comments

Comments
 (0)