From 144e517502fb5dedbcf7d33b38382743ee44f551 Mon Sep 17 00:00:00 2001 From: Radu Berinde Date: Thu, 27 Jun 2024 14:46:58 -0700 Subject: [PATCH] rowblk: reorganize tests Split the `rowblk` tests into multiple files. --- sstable/rowblk/rowblk_bench_test.go | 215 ++++++++++++ .../{rowblk_test.go => rowblk_iter_test.go} | 310 +----------------- sstable/rowblk/rowblk_writer_test.go | 117 +++++++ .../rowblk/testdata/{block => rowblk_iter} | 0 4 files changed, 333 insertions(+), 309 deletions(-) create mode 100644 sstable/rowblk/rowblk_bench_test.go rename sstable/rowblk/{rowblk_test.go => rowblk_iter_test.go} (60%) create mode 100644 sstable/rowblk/rowblk_writer_test.go rename sstable/rowblk/testdata/{block => rowblk_iter} (100%) diff --git a/sstable/rowblk/rowblk_bench_test.go b/sstable/rowblk/rowblk_bench_test.go new file mode 100644 index 0000000000..e7d00b3b1f --- /dev/null +++ b/sstable/rowblk/rowblk_bench_test.go @@ -0,0 +1,215 @@ +// Copyright 2024 The LevelDB-Go and Pebble Authors. All rights reserved. Use +// of this source code is governed by a BSD-style license that can be found in +// the LICENSE file. + +package rowblk + +import ( + "bytes" + "fmt" + "testing" + "time" + + "github.com/cockroachdb/pebble/internal/base" + "github.com/cockroachdb/pebble/internal/testkeys" + "github.com/cockroachdb/pebble/sstable/block" + "golang.org/x/exp/rand" +) + +var ( + benchSynthSuffix = []byte("@15") + benchPrefix = []byte("2_") + + // Use testkeys.Comparer.Compare which approximates EngineCompare by ordering + // multiple keys with same prefix in descending suffix order. + benchCmp = testkeys.Comparer.Compare + benchSplit = testkeys.Comparer.Split +) + +// choosOrigSuffix randomly chooses a suffix that is either 1 or 2 bytes large. +// This ensures we benchmark when suffix replacement adds a larger suffix. +func chooseOrigSuffix(rng *rand.Rand) []byte { + origSuffix := []byte("@10") + if rng.Intn(10)%2 == 0 { + origSuffix = []byte("@9") + } + return origSuffix +} + +// createBenchBlock writes a block of keys and outputs a list of keys that will +// be surfaced from the block, and the expected synthetic suffix and prefix the +// block should be read with. +func createBenchBlock( + blockSize int, w *Writer, rng *rand.Rand, withSyntheticPrefix, withSyntheticSuffix bool, +) ([][]byte, []byte, []byte) { + + origSuffix := chooseOrigSuffix(rng) + var ikey base.InternalKey + var readKeys [][]byte + + var writtenPrefix []byte + if !withSyntheticPrefix { + // If the keys will not be read with a synthetic prefix, write the prefix to + // the block for a more comparable benchmark comparison between a block iter + // with and without prefix synthesis. + writtenPrefix = benchPrefix + } + for i := 0; w.EstimatedSize() < blockSize; i++ { + key := []byte(fmt.Sprintf("%s%05d%s", string(writtenPrefix), i, origSuffix)) + ikey.UserKey = key + w.Add(ikey, nil) + var readKey []byte + if withSyntheticPrefix { + readKey = append(readKey, benchPrefix...) + } + readKey = append(readKey, key...) + readKeys = append(readKeys, readKey) + } + + var syntheticSuffix []byte + var syntheticPrefix []byte + if withSyntheticSuffix { + syntheticSuffix = benchSynthSuffix + } + if withSyntheticPrefix { + syntheticPrefix = []byte(benchPrefix) + } + return readKeys, syntheticPrefix, syntheticSuffix +} + +func BenchmarkBlockIterSeekGE(b *testing.B) { + const blockSize = 32 << 10 + for _, withSyntheticPrefix := range []bool{false, true} { + for _, withSyntheticSuffix := range []bool{false, true} { + for _, restartInterval := range []int{16} { + b.Run(fmt.Sprintf("syntheticPrefix=%t;syntheticSuffix=%t;restart=%d", withSyntheticPrefix, withSyntheticSuffix, restartInterval), + func(b *testing.B) { + w := &Writer{RestartInterval: restartInterval} + rng := rand.New(rand.NewSource(uint64(time.Now().UnixNano()))) + + keys, syntheticPrefix, syntheticSuffix := createBenchBlock(blockSize, w, rng, withSyntheticPrefix, withSyntheticSuffix) + + it, err := NewIter(benchCmp, benchSplit, w.Finish(), block.IterTransforms{SyntheticSuffix: syntheticSuffix, SyntheticPrefix: syntheticPrefix}) + if err != nil { + b.Fatal(err) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + k := keys[rng.Intn(len(keys))] + it.SeekGE(k, base.SeekGEFlagsNone) + if testing.Verbose() { + if !it.Valid() && !withSyntheticSuffix { + b.Fatal("expected to find key") + } + if !bytes.Equal(k, it.Key().UserKey) && !withSyntheticSuffix { + b.Fatalf("expected %s, but found %s", k, it.Key().UserKey) + } + } + } + }) + } + } + } +} + +func BenchmarkBlockIterSeekLT(b *testing.B) { + const blockSize = 32 << 10 + for _, withSyntheticPrefix := range []bool{false, true} { + for _, withSyntheticSuffix := range []bool{false, true} { + for _, restartInterval := range []int{16} { + b.Run(fmt.Sprintf("syntheticPrefix=%t;syntheticSuffix=%t;restart=%d", withSyntheticPrefix, withSyntheticSuffix, restartInterval), + func(b *testing.B) { + w := &Writer{RestartInterval: restartInterval} + rng := rand.New(rand.NewSource(uint64(time.Now().UnixNano()))) + + keys, syntheticPrefix, syntheticSuffix := createBenchBlock(blockSize, w, rng, withSyntheticPrefix, withSyntheticSuffix) + + it, err := NewIter(benchCmp, benchSplit, w.Finish(), block.IterTransforms{SyntheticSuffix: syntheticSuffix, SyntheticPrefix: syntheticPrefix}) + if err != nil { + b.Fatal(err) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + j := rng.Intn(len(keys)) + it.SeekLT(keys[j], base.SeekLTFlagsNone) + if testing.Verbose() { + if j == 0 { + if it.Valid() && !withSyntheticSuffix { + b.Fatal("unexpected key") + } + } else { + if !it.Valid() && !withSyntheticSuffix { + b.Fatal("expected to find key") + } + k := keys[j-1] + if !bytes.Equal(k, it.Key().UserKey) && !withSyntheticSuffix { + b.Fatalf("expected %s, but found %s", k, it.Key().UserKey) + } + } + } + } + }) + } + } + } +} + +func BenchmarkBlockIterNext(b *testing.B) { + const blockSize = 32 << 10 + for _, withSyntheticPrefix := range []bool{false, true} { + for _, withSyntheticSuffix := range []bool{false, true} { + for _, restartInterval := range []int{16} { + b.Run(fmt.Sprintf("syntheticPrefix=%t;syntheticSuffix=%t;restart=%d", withSyntheticPrefix, withSyntheticSuffix, restartInterval), + func(b *testing.B) { + w := &Writer{RestartInterval: restartInterval} + rng := rand.New(rand.NewSource(uint64(time.Now().UnixNano()))) + + _, syntheticPrefix, syntheticSuffix := createBenchBlock(blockSize, w, rng, withSyntheticPrefix, withSyntheticSuffix) + + it, err := NewIter(benchCmp, benchSplit, w.Finish(), block.IterTransforms{SyntheticSuffix: syntheticSuffix, SyntheticPrefix: syntheticPrefix}) + if err != nil { + b.Fatal(err) + } + + b.ResetTimer() + for i := 0; i < b.N; i++ { + if !it.Valid() { + it.First() + } + it.Next() + } + }) + } + } + } +} + +func BenchmarkBlockIterPrev(b *testing.B) { + const blockSize = 32 << 10 + for _, withSyntheticPrefix := range []bool{false, true} { + for _, withSyntheticSuffix := range []bool{false, true} { + for _, restartInterval := range []int{16} { + b.Run(fmt.Sprintf("syntheticPrefix=%t;syntheticSuffix=%t;restart=%d", withSyntheticPrefix, withSyntheticSuffix, restartInterval), + func(b *testing.B) { + w := &Writer{RestartInterval: restartInterval} + rng := rand.New(rand.NewSource(uint64(time.Now().UnixNano()))) + + _, syntheticPrefix, syntheticSuffix := createBenchBlock(blockSize, w, rng, withSyntheticPrefix, withSyntheticSuffix) + + it, err := NewIter(benchCmp, benchSplit, w.Finish(), block.IterTransforms{SyntheticSuffix: syntheticSuffix, SyntheticPrefix: syntheticPrefix}) + if err != nil { + b.Fatal(err) + } + + b.ResetTimer() + for i := 0; i < b.N; i++ { + if !it.Valid() { + it.Last() + } + it.Prev() + } + }) + } + } + } +} diff --git a/sstable/rowblk/rowblk_test.go b/sstable/rowblk/rowblk_iter_test.go similarity index 60% rename from sstable/rowblk/rowblk_test.go rename to sstable/rowblk/rowblk_iter_test.go index ad1928e722..c802d761d1 100644 --- a/sstable/rowblk/rowblk_test.go +++ b/sstable/rowblk/rowblk_iter_test.go @@ -9,7 +9,6 @@ import ( "fmt" "strings" "testing" - "time" "unsafe" "github.com/cockroachdb/datadriven" @@ -18,7 +17,6 @@ import ( "github.com/cockroachdb/pebble/internal/testkeys" "github.com/cockroachdb/pebble/sstable/block" "github.com/stretchr/testify/require" - "golang.org/x/exp/rand" ) func TestInvalidInternalKeyDecoding(t *testing.T) { @@ -116,7 +114,7 @@ func TestBlockIter2(t *testing.T) { for _, r := range []int{1, 2, 3, 4} { t.Run(fmt.Sprintf("restart=%d", r), func(t *testing.T) { - datadriven.RunTest(t, "testdata/block", func(t *testing.T, d *datadriven.TestData) string { + datadriven.RunTest(t, "testdata/rowblk_iter", func(t *testing.T, d *datadriven.TestData) string { switch d.Cmd { case "build": w := &Writer{RestartInterval: r} @@ -459,312 +457,6 @@ func TestBlockSyntheticSuffix(t *testing.T) { } } -func testBlockCleared(t *testing.T, w, b *Writer) { - require.Equal(t, w.RestartInterval, b.RestartInterval) - require.Equal(t, w.nEntries, b.nEntries) - require.Equal(t, w.nextRestart, b.nextRestart) - require.Equal(t, len(w.buf), len(b.buf)) - require.Equal(t, len(w.restarts), len(b.restarts)) - require.Equal(t, len(w.curKey), len(b.curKey)) - require.Equal(t, len(w.prevKey), len(b.prevKey)) - require.Equal(t, len(w.curValue), len(b.curValue)) - require.Equal(t, w.tmp, b.tmp) - - // Make sure that we didn't lose the allocated byte slices. - require.True(t, cap(w.buf) > 0 && cap(b.buf) == 0) - require.True(t, cap(w.restarts) > 0 && cap(b.restarts) == 0) - require.True(t, cap(w.curKey) > 0 && cap(b.curKey) == 0) - require.True(t, cap(w.prevKey) > 0 && cap(b.prevKey) == 0) - require.True(t, cap(w.curValue) > 0 && cap(b.curValue) == 0) -} - -func TestBlockClear(t *testing.T) { - w := Writer{RestartInterval: 16} - w.Add(ikey("apple"), nil) - w.Add(ikey("apricot"), nil) - w.Add(ikey("banana"), nil) - - w.Reset() - - // Once a block is cleared, we expect its fields to be cleared, but we expect - // it to keep its allocated byte slices. - b := Writer{} - testBlockCleared(t, &w, &b) -} - -func TestBlockWriter(t *testing.T) { - w := &Writer{RestartInterval: 16} - w.AddRawString("apple", nil) - w.AddRawString("apricot", nil) - w.AddRawString("banana", nil) - block := w.Finish() - - expected := []byte( - "\x00\x05\x00apple" + - "\x02\x05\x00ricot" + - "\x00\x06\x00banana" + - "\x00\x00\x00\x00\x01\x00\x00\x00") - if !bytes.Equal(expected, block) { - t.Fatalf("expected\n%q\nfound\n%q", expected, block) - } -} - -func TestBlockWriterWithPrefix(t *testing.T) { - w := &Writer{RestartInterval: 2} - curKey := func() string { - return string(base.DecodeInternalKey(w.curKey).UserKey) - } - addAdapter := func( - key base.InternalKey, - value []byte, - addValuePrefix bool, - valuePrefix block.ValuePrefix, - setHasSameKeyPrefix bool) { - w.AddWithOptionalValuePrefix( - key, false, value, len(key.UserKey), addValuePrefix, valuePrefix, setHasSameKeyPrefix) - } - addAdapter( - ikey("apple"), []byte("red"), false, 0, true) - require.Equal(t, "apple", curKey()) - require.Equal(t, "red", string(w.CurValue())) - addAdapter( - ikey("apricot"), []byte("orange"), true, '\xff', false) - require.Equal(t, "apricot", curKey()) - require.Equal(t, "orange", string(w.CurValue())) - // Even though this call has setHasSameKeyPrefix=true, the previous call, - // which was after the last restart set it to false. So the restart encoded - // with banana has this cumulative bit set to false. - addAdapter( - ikey("banana"), []byte("yellow"), true, '\x00', true) - require.Equal(t, "banana", curKey()) - require.Equal(t, "yellow", string(w.CurValue())) - addAdapter( - ikey("cherry"), []byte("red"), false, 0, true) - require.Equal(t, "cherry", curKey()) - require.Equal(t, "red", string(w.CurValue())) - // All intervening calls has setHasSameKeyPrefix=true, so the cumulative bit - // will be set to true in this restart. - addAdapter( - ikey("mango"), []byte("juicy"), false, 0, true) - require.Equal(t, "mango", curKey()) - require.Equal(t, "juicy", string(w.CurValue())) - - blk := w.Finish() - - expected := []byte( - "\x00\x0d\x03apple\x00\x00\x00\x00\x00\x00\x00\x00red" + - "\x02\x0d\x07ricot\x00\x00\x00\x00\x00\x00\x00\x00\xfforange" + - "\x00\x0e\x07banana\x00\x00\x00\x00\x00\x00\x00\x00\x00yellow" + - "\x00\x0e\x03cherry\x00\x00\x00\x00\x00\x00\x00\x00red" + - "\x00\x0d\x05mango\x00\x00\x00\x00\x00\x00\x00\x00juicy" + - // Restarts are: - // 00000000 (restart at apple), 2a000000 (restart at banana), 56000080 (restart at mango) - // 03000000 (number of restart, i.e., 3). The restart at mango has 1 in the most significant - // bit of the uint32, so the last byte in the little endian encoding is \x80. - "\x00\x00\x00\x00\x2a\x00\x00\x00\x56\x00\x00\x80\x03\x00\x00\x00") - if !bytes.Equal(expected, blk) { - t.Fatalf("expected\n%x\nfound\n%x", expected, blk) - } -} - func ikey(s string) base.InternalKey { return base.InternalKey{UserKey: []byte(s)} } - -var ( - benchSynthSuffix = []byte("@15") - benchPrefix = []byte("2_") - - // Use testkeys.Comparer.Compare which approximates EngineCompare by ordering - // multiple keys with same prefix in descending suffix order. - benchCmp = testkeys.Comparer.Compare - benchSplit = testkeys.Comparer.Split -) - -// choosOrigSuffix randomly chooses a suffix that is either 1 or 2 bytes large. -// This ensures we benchmark when suffix replacement adds a larger suffix. -func chooseOrigSuffix(rng *rand.Rand) []byte { - origSuffix := []byte("@10") - if rng.Intn(10)%2 == 0 { - origSuffix = []byte("@9") - } - return origSuffix -} - -// createBenchBlock writes a block of keys and outputs a list of keys that will -// be surfaced from the block, and the expected synthetic suffix and prefix the -// block should be read with. -func createBenchBlock( - blockSize int, w *Writer, rng *rand.Rand, withSyntheticPrefix, withSyntheticSuffix bool, -) ([][]byte, []byte, []byte) { - - origSuffix := chooseOrigSuffix(rng) - var ikey base.InternalKey - var readKeys [][]byte - - var writtenPrefix []byte - if !withSyntheticPrefix { - // If the keys will not be read with a synthetic prefix, write the prefix to - // the block for a more comparable benchmark comparison between a block iter - // with and without prefix synthesis. - writtenPrefix = benchPrefix - } - for i := 0; w.EstimatedSize() < blockSize; i++ { - key := []byte(fmt.Sprintf("%s%05d%s", string(writtenPrefix), i, origSuffix)) - ikey.UserKey = key - w.Add(ikey, nil) - var readKey []byte - if withSyntheticPrefix { - readKey = append(readKey, benchPrefix...) - } - readKey = append(readKey, key...) - readKeys = append(readKeys, readKey) - } - - var syntheticSuffix []byte - var syntheticPrefix []byte - if withSyntheticSuffix { - syntheticSuffix = benchSynthSuffix - } - if withSyntheticPrefix { - syntheticPrefix = []byte(benchPrefix) - } - return readKeys, syntheticPrefix, syntheticSuffix -} - -func BenchmarkBlockIterSeekGE(b *testing.B) { - const blockSize = 32 << 10 - for _, withSyntheticPrefix := range []bool{false, true} { - for _, withSyntheticSuffix := range []bool{false, true} { - for _, restartInterval := range []int{16} { - b.Run(fmt.Sprintf("syntheticPrefix=%t;syntheticSuffix=%t;restart=%d", withSyntheticPrefix, withSyntheticSuffix, restartInterval), - func(b *testing.B) { - w := &Writer{RestartInterval: restartInterval} - rng := rand.New(rand.NewSource(uint64(time.Now().UnixNano()))) - - keys, syntheticPrefix, syntheticSuffix := createBenchBlock(blockSize, w, rng, withSyntheticPrefix, withSyntheticSuffix) - - it, err := NewIter(benchCmp, benchSplit, w.Finish(), block.IterTransforms{SyntheticSuffix: syntheticSuffix, SyntheticPrefix: syntheticPrefix}) - if err != nil { - b.Fatal(err) - } - b.ResetTimer() - for i := 0; i < b.N; i++ { - k := keys[rng.Intn(len(keys))] - it.SeekGE(k, base.SeekGEFlagsNone) - if testing.Verbose() { - if !it.Valid() && !withSyntheticSuffix { - b.Fatal("expected to find key") - } - if !bytes.Equal(k, it.Key().UserKey) && !withSyntheticSuffix { - b.Fatalf("expected %s, but found %s", k, it.Key().UserKey) - } - } - } - }) - } - } - } -} - -func BenchmarkBlockIterSeekLT(b *testing.B) { - const blockSize = 32 << 10 - for _, withSyntheticPrefix := range []bool{false, true} { - for _, withSyntheticSuffix := range []bool{false, true} { - for _, restartInterval := range []int{16} { - b.Run(fmt.Sprintf("syntheticPrefix=%t;syntheticSuffix=%t;restart=%d", withSyntheticPrefix, withSyntheticSuffix, restartInterval), - func(b *testing.B) { - w := &Writer{RestartInterval: restartInterval} - rng := rand.New(rand.NewSource(uint64(time.Now().UnixNano()))) - - keys, syntheticPrefix, syntheticSuffix := createBenchBlock(blockSize, w, rng, withSyntheticPrefix, withSyntheticSuffix) - - it, err := NewIter(benchCmp, benchSplit, w.Finish(), block.IterTransforms{SyntheticSuffix: syntheticSuffix, SyntheticPrefix: syntheticPrefix}) - if err != nil { - b.Fatal(err) - } - b.ResetTimer() - for i := 0; i < b.N; i++ { - j := rng.Intn(len(keys)) - it.SeekLT(keys[j], base.SeekLTFlagsNone) - if testing.Verbose() { - if j == 0 { - if it.Valid() && !withSyntheticSuffix { - b.Fatal("unexpected key") - } - } else { - if !it.Valid() && !withSyntheticSuffix { - b.Fatal("expected to find key") - } - k := keys[j-1] - if !bytes.Equal(k, it.Key().UserKey) && !withSyntheticSuffix { - b.Fatalf("expected %s, but found %s", k, it.Key().UserKey) - } - } - } - } - }) - } - } - } -} - -func BenchmarkBlockIterNext(b *testing.B) { - const blockSize = 32 << 10 - for _, withSyntheticPrefix := range []bool{false, true} { - for _, withSyntheticSuffix := range []bool{false, true} { - for _, restartInterval := range []int{16} { - b.Run(fmt.Sprintf("syntheticPrefix=%t;syntheticSuffix=%t;restart=%d", withSyntheticPrefix, withSyntheticSuffix, restartInterval), - func(b *testing.B) { - w := &Writer{RestartInterval: restartInterval} - rng := rand.New(rand.NewSource(uint64(time.Now().UnixNano()))) - - _, syntheticPrefix, syntheticSuffix := createBenchBlock(blockSize, w, rng, withSyntheticPrefix, withSyntheticSuffix) - - it, err := NewIter(benchCmp, benchSplit, w.Finish(), block.IterTransforms{SyntheticSuffix: syntheticSuffix, SyntheticPrefix: syntheticPrefix}) - if err != nil { - b.Fatal(err) - } - - b.ResetTimer() - for i := 0; i < b.N; i++ { - if !it.Valid() { - it.First() - } - it.Next() - } - }) - } - } - } -} - -func BenchmarkBlockIterPrev(b *testing.B) { - const blockSize = 32 << 10 - for _, withSyntheticPrefix := range []bool{false, true} { - for _, withSyntheticSuffix := range []bool{false, true} { - for _, restartInterval := range []int{16} { - b.Run(fmt.Sprintf("syntheticPrefix=%t;syntheticSuffix=%t;restart=%d", withSyntheticPrefix, withSyntheticSuffix, restartInterval), - func(b *testing.B) { - w := &Writer{RestartInterval: restartInterval} - rng := rand.New(rand.NewSource(uint64(time.Now().UnixNano()))) - - _, syntheticPrefix, syntheticSuffix := createBenchBlock(blockSize, w, rng, withSyntheticPrefix, withSyntheticSuffix) - - it, err := NewIter(benchCmp, benchSplit, w.Finish(), block.IterTransforms{SyntheticSuffix: syntheticSuffix, SyntheticPrefix: syntheticPrefix}) - if err != nil { - b.Fatal(err) - } - - b.ResetTimer() - for i := 0; i < b.N; i++ { - if !it.Valid() { - it.Last() - } - it.Prev() - } - }) - } - } - } -} diff --git a/sstable/rowblk/rowblk_writer_test.go b/sstable/rowblk/rowblk_writer_test.go new file mode 100644 index 0000000000..8c91ba68af --- /dev/null +++ b/sstable/rowblk/rowblk_writer_test.go @@ -0,0 +1,117 @@ +// Copyright 2024 The LevelDB-Go and Pebble Authors. All rights reserved. Use +// of this source code is governed by a BSD-style license that can be found in +// the LICENSE file. + +package rowblk + +import ( + "bytes" + "testing" + + "github.com/cockroachdb/pebble/internal/base" + "github.com/cockroachdb/pebble/sstable/block" + "github.com/stretchr/testify/require" +) + +func TestBlockWriterClear(t *testing.T) { + w := Writer{RestartInterval: 16} + w.Add(ikey("apple"), nil) + w.Add(ikey("apricot"), nil) + w.Add(ikey("banana"), nil) + + w.Reset() + + // Once a block is cleared, we expect its fields to be cleared, but we expect + // it to keep its allocated byte slices. + require.Equal(t, w.RestartInterval, 0) + require.Equal(t, w.nEntries, 0) + require.Equal(t, w.nextRestart, 0) + require.Equal(t, len(w.buf), 0) + require.Equal(t, len(w.restarts), 0) + require.Equal(t, len(w.curKey), 0) + require.Equal(t, len(w.prevKey), 0) + require.Equal(t, len(w.curValue), 0) + require.Equal(t, w.tmp, [4]byte{}) + + // Make sure that we didn't lose the allocated byte slices. + require.True(t, cap(w.buf) > 0) + require.True(t, cap(w.restarts) > 0) + require.True(t, cap(w.curKey) > 0) + require.True(t, cap(w.prevKey) > 0) + require.True(t, cap(w.curValue) > 0) +} + +func TestBlockWriter(t *testing.T) { + w := &Writer{RestartInterval: 16} + w.AddRawString("apple", nil) + w.AddRawString("apricot", nil) + w.AddRawString("banana", nil) + block := w.Finish() + + expected := []byte( + "\x00\x05\x00apple" + + "\x02\x05\x00ricot" + + "\x00\x06\x00banana" + + "\x00\x00\x00\x00\x01\x00\x00\x00") + if !bytes.Equal(expected, block) { + t.Fatalf("expected\n%q\nfound\n%q", expected, block) + } +} + +func TestBlockWriterWithPrefix(t *testing.T) { + w := &Writer{RestartInterval: 2} + curKey := func() string { + return string(base.DecodeInternalKey(w.curKey).UserKey) + } + addAdapter := func( + key base.InternalKey, + value []byte, + addValuePrefix bool, + valuePrefix block.ValuePrefix, + setHasSameKeyPrefix bool) { + w.AddWithOptionalValuePrefix( + key, false, value, len(key.UserKey), addValuePrefix, valuePrefix, setHasSameKeyPrefix) + } + addAdapter( + ikey("apple"), []byte("red"), false, 0, true) + require.Equal(t, "apple", curKey()) + require.Equal(t, "red", string(w.CurValue())) + addAdapter( + ikey("apricot"), []byte("orange"), true, '\xff', false) + require.Equal(t, "apricot", curKey()) + require.Equal(t, "orange", string(w.CurValue())) + // Even though this call has setHasSameKeyPrefix=true, the previous call, + // which was after the last restart set it to false. So the restart encoded + // with banana has this cumulative bit set to false. + addAdapter( + ikey("banana"), []byte("yellow"), true, '\x00', true) + require.Equal(t, "banana", curKey()) + require.Equal(t, "yellow", string(w.CurValue())) + addAdapter( + ikey("cherry"), []byte("red"), false, 0, true) + require.Equal(t, "cherry", curKey()) + require.Equal(t, "red", string(w.CurValue())) + // All intervening calls has setHasSameKeyPrefix=true, so the cumulative bit + // will be set to true in this restart. + addAdapter( + ikey("mango"), []byte("juicy"), false, 0, true) + require.Equal(t, "mango", curKey()) + require.Equal(t, "juicy", string(w.CurValue())) + + blk := w.Finish() + + expected := []byte( + "\x00\x0d\x03apple\x00\x00\x00\x00\x00\x00\x00\x00red" + + "\x02\x0d\x07ricot\x00\x00\x00\x00\x00\x00\x00\x00\xfforange" + + "\x00\x0e\x07banana\x00\x00\x00\x00\x00\x00\x00\x00\x00yellow" + + "\x00\x0e\x03cherry\x00\x00\x00\x00\x00\x00\x00\x00red" + + "\x00\x0d\x05mango\x00\x00\x00\x00\x00\x00\x00\x00juicy" + + // Restarts are: + // 00000000 (restart at apple), 2a000000 (restart at banana), 56000080 (restart at mango) + // 03000000 (number of restart, i.e., 3). The restart at mango has 1 in the most significant + // bit of the uint32, so the last byte in the little endian encoding is \x80. + "\x00\x00\x00\x00\x2a\x00\x00\x00\x56\x00\x00\x80\x03\x00\x00\x00") + if !bytes.Equal(expected, blk) { + t.Fatalf("expected\n%x\nfound\n%x", expected, blk) + } +} diff --git a/sstable/rowblk/testdata/block b/sstable/rowblk/testdata/rowblk_iter similarity index 100% rename from sstable/rowblk/testdata/block rename to sstable/rowblk/testdata/rowblk_iter