@@ -7,18 +7,12 @@ package rowblk
77import (
88 "bytes"
99 "fmt"
10- "math"
11- "os"
12- "runtime"
13- "strconv"
1410 "strings"
1511 "testing"
1612 "unsafe"
1713
1814 "github.com/cockroachdb/datadriven"
19- "github.com/cockroachdb/errors"
2015 "github.com/cockroachdb/pebble/internal/base"
21- "github.com/cockroachdb/pebble/internal/buildtags"
2216 "github.com/cockroachdb/pebble/internal/itertest"
2317 "github.com/cockroachdb/pebble/internal/testkeys"
2418 "github.com/cockroachdb/pebble/sstable/block"
@@ -468,177 +462,6 @@ func TestBlockSyntheticSuffix(t *testing.T) {
468462 }
469463}
470464
471- // TestSingularKVBlockRestartsOverflow tests a scenario where a large key-value
472- // pair is written to a block, such that the total block size exceeds 4GiB. This
473- // works becasue the restart table never needs to encode a restart offset beyond
474- // the 1st key-value pair. The offset of the restarts table itself may exceed
475- // 2^32-1 but the iterator takes care to support this.
476- func TestSingularKVBlockRestartsOverflow (t * testing.T ) {
477- _ , isCI := os .LookupEnv ("CI" )
478- if isCI {
479- t .Skip ("Skipping test: requires too much memory for CI now." )
480- }
481- if buildtags .SlowBuild {
482- t .Skip ("Skipping test: requires too much memory for instrumented builds" )
483- }
484-
485- // Test that SeekGE() and SeekLT() function correctly
486- // with a singular large KV > 2GB.
487-
488- // Skip this test on 32-bit architectures because they may not
489- // have sufficient memory to reliably execute this test.
490- if runtime .GOARCH == "386" || runtime .GOARCH == "arm" || strconv .IntSize == 32 {
491- t .Skip ("Skipping test: not supported on 32-bit architecture" )
492- }
493-
494- const largeKeySize = 2 << 30 // 2GB key size
495- const largeValueSize = 2 << 30 // 2GB value size
496-
497- largeKey := bytes .Repeat ([]byte ("k" ), largeKeySize )
498- largeValue := bytes .Repeat ([]byte ("v" ), largeValueSize )
499-
500- writer := & Writer {RestartInterval : 1 }
501- require .NoError (t , writer .Add (base.InternalKey {UserKey : largeKey }, largeValue ))
502- blockData := writer .Finish ()
503- iter , err := NewIter (bytes .Compare , nil , nil , blockData , block .NoTransforms )
504- require .NoError (t , err , "failed to create iterator for block" )
505-
506- // Ensure that SeekGE() does not raise panic due to integer overflow
507- // indexing problems.
508- kv := iter .SeekGE (largeKey , base .SeekGEFlagsNone )
509-
510- // Ensure that SeekGE() finds the correct KV
511- require .NotNil (t , kv , "failed to find the key" )
512- require .Equal (t , largeKey , kv .K .UserKey , "unexpected key" )
513- require .Equal (t , largeValue , kv .InPlaceValue (), "unexpected value" )
514-
515- // Ensure that SeekGE() does not raise panic due to integer overflow
516- // indexing problems.
517- kv = iter .SeekLT ([]byte ("z" ), base .SeekLTFlagsNone )
518-
519- // Ensure that SeekLT() finds the correct KV
520- require .NotNil (t , kv , "failed to find the key" )
521- require .Equal (t , largeKey , kv .K .UserKey , "unexpected key" )
522- require .Equal (t , largeValue , kv .InPlaceValue (), "unexpected value" )
523- }
524-
525- // TestExceedingMaximumRestartOffset tests that writing a block that exceeds the
526- // maximum restart offset errors.
527- func TestExceedingMaximumRestartOffset (t * testing.T ) {
528- _ , isCI := os .LookupEnv ("CI" )
529- if isCI {
530- t .Skip ("Skipping test: requires too much memory for CI now." )
531- }
532- if buildtags .SlowBuild {
533- t .Skip ("Skipping test: requires too much memory for instrumented builds" )
534- }
535-
536- // Test that writing to a block that is already >= 2GiB
537- // returns an error.
538- //
539- // Skip this test on 32-bit architectures because they may not
540- // have sufficient memory to reliably execute this test.
541- if runtime .GOARCH == "386" || runtime .GOARCH == "arm" || strconv .IntSize == 32 {
542- t .Skip ("Skipping test: not supported on 32-bit architecture" )
543- }
544-
545- // Adding 512 KVs each with size 4MiB will create a block
546- // size of >= 2GiB.
547- const numKVs = 512
548- const valueSize = (4 << 20 )
549-
550- type KVTestPair struct {
551- key []byte
552- value []byte
553- }
554-
555- KVTestPairs := make ([]KVTestPair , numKVs )
556- value4MB := bytes .Repeat ([]byte ("a" ), valueSize )
557- for i := 0 ; i < numKVs ; i ++ {
558- key := fmt .Sprintf ("key-%04d" , i )
559- KVTestPairs [i ] = KVTestPair {key : []byte (key ), value : value4MB }
560- }
561- writer := & Writer {RestartInterval : 1 }
562- for _ , KVPair := range KVTestPairs {
563- require .NoError (t , writer .Add (base.InternalKey {UserKey : KVPair .key }, KVPair .value ))
564- }
565-
566- // Check that buffer is larger than 2GiB.
567- require .Greater (t , len (writer .buf ), MaximumRestartOffset )
568-
569- // Check that an error is returned after the final write after the 2GiB
570- // threshold has been crossed
571- err := writer .Add (base.InternalKey {UserKey : []byte ("arbitrary-last-key" )}, []byte ("arbitrary-last-value" ))
572- require .NotNil (t , err )
573- require .True (t , errors .Is (err , ErrBlockTooBig ))
574- }
575-
576- // TestMultipleKVBlockRestartsOverflow tests that SeekGE() works when
577- // iter.restarts is greater than math.MaxUint32 for multiple KVs. Test writes
578- // <2GiB to the block and then 4GiB causing iter.restarts to be an int >
579- // math.MaxUint32. Reaching just shy of 2GiB before adding 4GiB allows the
580- // final write to succeed without surpassing 2GiB limit. Then verify that
581- // SeekGE() returns valid output without integer overflow.
582- //
583- // Although the block exceeds math.MaxUint32 bytes, no individual KV pair has an
584- // offset that exceeds MaximumRestartOffset.
585- func TestMultipleKVBlockRestartsOverflow (t * testing.T ) {
586- if _ , isCI := os .LookupEnv ("CI" ); isCI {
587- t .Skip ("Skipping test: requires too much memory for CI." )
588- }
589- if buildtags .SlowBuild {
590- t .Skip ("Skipping test: requires too much memory for instrumented builds" )
591- }
592-
593- // Skip this test on 32-bit architectures because they may not
594- // have sufficient memory to reliably execute this test.
595- if runtime .GOARCH == "386" || runtime .GOARCH == "arm" || strconv .IntSize == 32 {
596- t .Skip ("Skipping test: not supported on 32-bit architecture" )
597- }
598-
599- // Write just shy of 2GiB to the block 511 * 4MiB < 2GiB.
600- const numKVs = 511
601- const valueSize = 4 * (1 << 20 )
602- const fourGB = 4 * (1 << 30 )
603-
604- type KVTestPair struct {
605- key []byte
606- value []byte
607- }
608-
609- KVTestPairs := make ([]KVTestPair , numKVs )
610- value4MB := bytes .Repeat ([]byte ("a" ), valueSize )
611- for i := 0 ; i < numKVs ; i ++ {
612- key := fmt .Sprintf ("key-%04d" , i )
613- KVTestPairs [i ] = KVTestPair {key : []byte (key ), value : value4MB }
614- }
615-
616- writer := & Writer {RestartInterval : 1 }
617- for _ , KVPair := range KVTestPairs {
618- writer .Add (base.InternalKey {UserKey : KVPair .key }, KVPair .value )
619- }
620-
621- // Add the 4GiB KV, causing iter.restarts >= math.MaxUint32.
622- // Ensure that SeekGE() works thereafter without integer
623- // overflows.
624- writer .Add (base.InternalKey {UserKey : []byte ("large-kv" )}, bytes .Repeat ([]byte ("v" ), fourGB ))
625-
626- blockData := writer .Finish ()
627- iter , err := NewIter (bytes .Compare , nil , nil , blockData , block .NoTransforms )
628- require .NoError (t , err , "failed to create iterator for block" )
629- require .Greater (t , int64 (iter .restarts ), int64 (MaximumRestartOffset ), "check iter.restarts > 2GiB" )
630- require .Greater (t , int64 (iter .restarts ), int64 (math .MaxUint32 ), "check iter.restarts > 2^32-1" )
631-
632- for i := 0 ; i < numKVs ; i ++ {
633- key := []byte (fmt .Sprintf ("key-%04d" , i ))
634- value := bytes .Repeat ([]byte ("a" ), valueSize )
635- kv := iter .SeekGE (key , base .SeekGEFlagsNone )
636- require .NotNil (t , kv , "failed to find the large key" )
637- require .Equal (t , key , kv .K .UserKey , "unexpected key" )
638- require .Equal (t , value , kv .InPlaceValue (), "unexpected value" )
639- }
640- }
641-
642465func ikey (s string ) base.InternalKey {
643466 return base.InternalKey {UserKey : []byte (s )}
644467}
0 commit comments