@@ -8,27 +8,36 @@ package kvserver_test
88import (
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