Skip to content

Commit b95d17c

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 e5b76a4 commit b95d17c

File tree

1 file changed

+238
-0
lines changed

1 file changed

+238
-0
lines changed

pkg/kv/kvserver/client_replica_backpressure_test.go

Lines changed: 238 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -8,27 +8,36 @@ package kvserver_test
88
import (
99
"context"
1010
"fmt"
11+
math "math"
1112
"net/url"
13+
"strings"
1214
"sync"
1315
"sync/atomic"
1416
"testing"
1517
"time"
1618

1719
"github.com/cockroachdb/cockroach/pkg/base"
1820
"github.com/cockroachdb/cockroach/pkg/keys"
21+
"github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord"
1922
"github.com/cockroachdb/cockroach/pkg/kv/kvpb"
2023
"github.com/cockroachdb/cockroach/pkg/kv/kvserver"
2124
"github.com/cockroachdb/cockroach/pkg/roachpb"
25+
"github.com/cockroachdb/cockroach/pkg/spanconfig"
26+
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
2227
"github.com/cockroachdb/cockroach/pkg/testutils"
2328
"github.com/cockroachdb/cockroach/pkg/testutils/pgurlutils"
29+
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
2430
"github.com/cockroachdb/cockroach/pkg/testutils/skip"
2531
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
2632
"github.com/cockroachdb/cockroach/pkg/testutils/testcluster"
33+
"github.com/cockroachdb/cockroach/pkg/util/hlc"
2734
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
2835
"github.com/cockroachdb/cockroach/pkg/util/log"
36+
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
2937
"github.com/cockroachdb/cockroach/pkg/util/randutil"
3038
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
3139
"github.com/cockroachdb/errors"
40+
"github.com/dustin/go-humanize"
3241
"github.com/jackc/pgx/v5"
3342
"github.com/stretchr/testify/require"
3443
)
@@ -332,3 +341,232 @@ func TestBackpressureNotAppliedWhenReducingRangeSize(t *testing.T) {
332341
require.Error(t, <-upsertErrCh)
333342
})
334343
}
344+
345+
// TestSpanConfigUpdatesBlockedByRangeSizeBackpressureOnDefaultRanges
346+
// verifies that spanconfig updates are blocked by backpressure when the
347+
// `system.span_configurations` table range becomes full, recreating the issue.
348+
//
349+
// Test strategy:
350+
// 1. Configure `system.span_configurations` table range to be a small size (8 KiB).
351+
// 2. Write many large spanconfig records (2 KiB each) to fill up the range.
352+
// 3. Verify spanconfig updates fail due to backpressure when the range is full,
353+
// 4. This test recreates the scenario where spanconfig updates are blocked by
354+
// backpressure.
355+
func TestSpanConfigUpdatesBlockedByRangeSizeBackpressureOnDefaultRanges(t *testing.T) {
356+
defer leaktest.AfterTest(t)()
357+
defer log.Scope(t).Close(t)
358+
359+
ctx := context.Background()
360+
361+
const (
362+
overloadMaxRangeBytes = 8 << 10 // 8 KiB, a saner value than default 512 MiB for testing
363+
overloadMinRangeBytes = 2 << 10 // 2 KiB
364+
numWrites = 16 // enough to hit backpressure for 8 KiB range & 2 KiB spanconfig
365+
defaultMaxBytes = 512 << 20 // default max bytes for a range
366+
)
367+
s := serverutils.StartServerOnly(t, base.TestServerArgs{})
368+
defer s.Stopper().Stop(ctx)
369+
370+
store, err := s.GetStores().(*kvserver.Stores).GetStore(s.GetFirstStoreID())
371+
require.NoError(t, err)
372+
373+
waitForSpanConfig := func(t *testing.T, tc serverutils.TestServerInterface,
374+
tablePrefix roachpb.Key, expRangeMaxBytes int64) {
375+
testutils.SucceedsSoon(t, func() error {
376+
_, r := getFirstStoreReplica(t, tc, tablePrefix)
377+
conf, err := r.LoadSpanConfig(ctx)
378+
if err != nil {
379+
return err
380+
}
381+
if conf.RangeMaxBytes != expRangeMaxBytes {
382+
return fmt.Errorf("expected RangeMaxBytes %d, got %d",
383+
expRangeMaxBytes, conf.RangeMaxBytes)
384+
}
385+
return nil
386+
})
387+
}
388+
389+
spanConfigTablePrefix := keys.SystemSQLCodec.TablePrefix(
390+
keys.SpanConfigurationsTableID)
391+
392+
t.Logf("targeting span_configurations table at key: %s (table ID %d)\n",
393+
spanConfigTablePrefix, keys.SpanConfigurationsTableID)
394+
395+
scratchKey, err := s.ScratchRange()
396+
require.NoError(t, err)
397+
398+
testutils.SucceedsSoon(t, func() error {
399+
repl := store.LookupReplica(roachpb.RKey(scratchKey))
400+
if got := repl.GetMaxBytes(ctx); got != defaultMaxBytes {
401+
return errors.Errorf(
402+
"range max bytes values did not start at %d; got %d",
403+
defaultMaxBytes, got)
404+
}
405+
return nil
406+
})
407+
408+
systemSpanConfigurationsTableSpan := roachpb.Span{
409+
Key: spanConfigTablePrefix,
410+
EndKey: spanConfigTablePrefix.PrefixEnd(),
411+
}
412+
413+
target := spanconfig.MakeTargetFromSpan(systemSpanConfigurationsTableSpan)
414+
415+
systemSpanConfig := roachpb.SpanConfig{
416+
RangeMaxBytes: overloadMaxRangeBytes,
417+
RangeMinBytes: overloadMinRangeBytes,
418+
}
419+
420+
configBytessdfsdf, err := protoutil.Marshal(&systemSpanConfig)
421+
require.NoError(t, err)
422+
t.Logf("marshalled systemSpanConfig size: %d bytes", len(configBytessdfsdf))
423+
424+
record, err := spanconfig.MakeRecord(target, systemSpanConfig)
425+
require.NoError(t, err)
426+
427+
kvaccessor := s.SpanConfigKVAccessor().(spanconfig.KVAccessor)
428+
429+
err = kvaccessor.UpdateSpanConfigRecords(
430+
ctx, []spanconfig.Target{target},
431+
[]spanconfig.Record{record}, hlc.MinTimestamp, hlc.MaxTimestamp)
432+
require.NoError(t, err)
433+
434+
waitForSpanConfig(t, s, spanConfigTablePrefix, overloadMaxRangeBytes)
435+
436+
// Check if the range is using our custom config.
437+
repl := store.LookupReplica(keys.MustAddr(spanConfigTablePrefix))
438+
if repl != nil {
439+
conf, err := repl.LoadSpanConfig(ctx)
440+
require.NoError(t, err)
441+
t.Logf("current range config - RangeMaxBytes: %d bytes (%d MiB), "+
442+
"RangeMinBytes: %d bytes (%d MiB)",
443+
conf.RangeMaxBytes, humanize.Bytes(uint64(conf.RangeMaxBytes)),
444+
conf.RangeMinBytes, humanize.Bytes(uint64(conf.RangeMinBytes)))
445+
446+
}
447+
448+
t.Logf("targeting span_configurations table at key: %s (table ID %d)\n",
449+
spanConfigTablePrefix, keys.SpanConfigurationsTableID)
450+
451+
// Create a single target for the scratch range (this will be stored in system.span_configurations)
452+
scratchTarget := spanconfig.MakeTargetFromSpan(roachpb.Span{
453+
Key: scratchKey,
454+
EndKey: scratchKey.PrefixEnd(),
455+
})
456+
457+
// This is a large spanconfig for a scratch range with relevant fields set
458+
// to maximum int64 and int32 values. This is done to have a spanconfig that
459+
// is large enough to trigger backpressure without having to write a million
460+
// records.
461+
// We want this config to be relatively large - this is done via setting
462+
// values to have max values and multiple fields as this config gets
463+
// marshalled into a protobuf and protobuf uses variant encoding, which
464+
// means larger values take more bytes to encode.
465+
spanConfig2KiB := roachpb.SpanConfig{ // 2078 bytes ~ 2 KiB.
466+
RangeMaxBytes: math.MaxInt64,
467+
RangeMinBytes: math.MaxInt64,
468+
GCPolicy: roachpb.GCPolicy{
469+
TTLSeconds: math.MaxInt32,
470+
ProtectionPolicies: []roachpb.ProtectionPolicy{
471+
{
472+
ProtectedTimestamp: hlc.MaxTimestamp,
473+
},
474+
{
475+
ProtectedTimestamp: hlc.MaxTimestamp,
476+
},
477+
},
478+
},
479+
NumReplicas: math.MaxInt32,
480+
GlobalReads: true,
481+
NumVoters: math.MaxInt32,
482+
VoterConstraints: []roachpb.ConstraintsConjunction{
483+
{
484+
Constraints: []roachpb.Constraint{
485+
{Key: "max_key", Value: strings.Repeat("x", 1024)}, // very long constraint value
486+
},
487+
},
488+
},
489+
LeasePreferences: []roachpb.LeasePreference{
490+
{
491+
Constraints: []roachpb.Constraint{
492+
{Key: "max_key", Value: strings.Repeat("y", 1024)}, // very long constraint value
493+
},
494+
},
495+
},
496+
}
497+
498+
configBytes, err := protoutil.Marshal(&spanConfig2KiB)
499+
require.NoError(t, err)
500+
501+
require.GreaterOrEqual(t, len(configBytes), 2048,
502+
"spanConfig2KiB should be at least 2 KiB in size")
503+
504+
// Create a record with the span configuration.
505+
testRecord, err := spanconfig.MakeRecord(scratchTarget, spanConfig2KiB)
506+
require.NoError(t, err)
507+
508+
// Write span configurations using KVAccessor.
509+
// We expect this to fail due to backpressure.
510+
var i int
511+
for i = 0; i < numWrites; i++ {
512+
// Use KVAccessor to update span configurations.
513+
err = kvaccessor.UpdateSpanConfigRecords(ctx, nil,
514+
[]spanconfig.Record{testRecord}, hlc.MinTimestamp, hlc.MaxTimestamp)
515+
if err != nil {
516+
break
517+
}
518+
}
519+
520+
// Assert that the operation failed due to backpressure.
521+
require.Error(t, err,
522+
"expected span config writes to fail due to backpressure, but they succeeded")
523+
524+
systemSpanConfigurationsTableSpanMVCCStats := roachpb.Span{
525+
Key: keys.SystemSQLCodec.TablePrefix(keys.SpanConfigurationsTableID),
526+
EndKey: keys.SystemSQLCodec.TablePrefix(keys.SpanConfigurationsTableID + 1),
527+
}
528+
529+
distSender := s.DistSenderI().(*kvcoord.DistSender)
530+
531+
// Track aggregate MVCC stats across all SpanConfigurationsTable ranges
532+
var aggregateStats enginepb.MVCCStats
533+
var rangeCount int
534+
535+
for key := systemSpanConfigurationsTableSpanMVCCStats.Key; key.Compare(systemSpanConfigurationsTableSpanMVCCStats.EndKey) < 0; {
536+
desc, err := distSender.RangeDescriptorCache().Lookup(ctx, keys.MustAddr(key))
537+
require.NoError(t, err)
538+
d := desc.Desc
539+
540+
rangeRepl := store.LookupReplica(d.StartKey)
541+
if rangeRepl != nil {
542+
stats := rangeRepl.GetMVCCStats()
543+
aggregateStats.Add(stats)
544+
rangeCount++
545+
}
546+
547+
// Move to next range.
548+
key = d.EndKey.AsRawKey()
549+
if key.Equal(roachpb.KeyMax) {
550+
break
551+
}
552+
}
553+
554+
require.Greater(t, aggregateStats.Total(), int64(overloadMaxRangeBytes))
555+
556+
smallSpanConfig := roachpb.SpanConfig{
557+
GCPolicy: roachpb.GCPolicy{
558+
TTLSeconds: 0,
559+
},
560+
}
561+
562+
smallSpanconfigRecord, err := spanconfig.MakeRecord(scratchTarget, smallSpanConfig)
563+
require.NoError(t, err)
564+
565+
smallSpanconfigRecordWriteErr := kvaccessor.UpdateSpanConfigRecords(ctx,
566+
[]spanconfig.Target{scratchTarget}, []spanconfig.Record{smallSpanconfigRecord},
567+
hlc.MinTimestamp, hlc.MaxTimestamp)
568+
569+
require.Error(t, smallSpanconfigRecordWriteErr,
570+
"expected smallSpanconfigRecord write to fail due to backpressure")
571+
572+
}

0 commit comments

Comments
 (0)