Skip to content

Commit

Permalink
colblk: implement synthetic prefix and suffix
Browse files Browse the repository at this point in the history
`DataBlockIter` now takes into account the synthetic prefix and suffix
transforms.
  • Loading branch information
RaduBerinde committed Oct 6, 2024
1 parent 9568f43 commit 5924e60
Show file tree
Hide file tree
Showing 12 changed files with 551 additions and 116 deletions.
11 changes: 3 additions & 8 deletions internal/crdbtest/crdbtest.go
Original file line number Diff line number Diff line change
Expand Up @@ -155,7 +155,9 @@ func EncodeTimestamp(key []byte, walltime uint64, logical uint32) []byte {
}

// DecodeTimestamp decodes a MVCC timestamp from a serialized MVCC key.
func DecodeTimestamp(mvccKey []byte) ([]byte, []byte, uint64, uint32) {
func DecodeTimestamp(
mvccKey []byte,
) (prefix []byte, untypedSuffix []byte, wallTime uint64, logicalTime uint32) {
tsLen := int(mvccKey[len(mvccKey)-1])
keyPartEnd := len(mvccKey) - tsLen
if keyPartEnd < 0 {
Expand Down Expand Up @@ -450,10 +452,3 @@ func (g *cockroachKeyGen) randTimestamp() (wallTime uint64, logicalTime uint32)
}
return wallTime, logicalTime
}

//func randCockroachKey(rng *rand.Rand, cfg KeyConfig, blockPrefix []byte) []byte {
// key := make([]byte, 0, cfg.PrefixLen+MaxSuffixLen)
// key = append(key, blockPrefix...)
// wallTime, logicalTime := g.rand
// return EncodeTimestamp(key, wallTime, cfg.Logical)
//}
10 changes: 9 additions & 1 deletion sstable/block/block.go
Original file line number Diff line number Diff line change
Expand Up @@ -256,6 +256,14 @@ type IterTransforms struct {
// NoTransforms is the default value for IterTransforms.
var NoTransforms = IterTransforms{}

// NoTransforms returns true if there are no transforms enabled.
func (t *IterTransforms) NoTransforms() bool {
return t.SyntheticSeqNum == 0 &&
!t.HideObsoletePoints &&
!t.SyntheticPrefix.IsSet() &&
!t.SyntheticSuffix.IsSet()
}

// FragmentIterTransforms allow on-the-fly transformation of range deletion or
// range key data at iteration time.
type FragmentIterTransforms struct {
Expand Down Expand Up @@ -293,7 +301,7 @@ const NoSyntheticSeqNum SyntheticSeqNum = 0
// RangeKeyUnset keys are not supported when a synthetic suffix is used.
type SyntheticSuffix []byte

// IsSet returns true if the synthetic suffix is not enpty.
// IsSet returns true if the synthetic suffix is not empty.
func (ss SyntheticSuffix) IsSet() bool {
return len(ss) > 0
}
Expand Down
38 changes: 38 additions & 0 deletions sstable/block/block_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,38 @@
// 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 block

import (
"testing"

"github.com/stretchr/testify/require"
)

func TestIterTransforms(t *testing.T) {
require.True(t, NoTransforms.NoTransforms())
var transforms IterTransforms
require.True(t, transforms.NoTransforms())
require.False(t, transforms.SyntheticPrefix.IsSet())
require.False(t, transforms.SyntheticSuffix.IsSet())
transforms.SyntheticPrefix = []byte{}
require.False(t, transforms.SyntheticPrefix.IsSet())
transforms.SyntheticSuffix = []byte{}
require.False(t, transforms.SyntheticSuffix.IsSet())
require.True(t, transforms.NoTransforms())
transforms.HideObsoletePoints = true
require.False(t, transforms.NoTransforms())

transforms = NoTransforms
transforms.SyntheticSeqNum = 123
require.False(t, transforms.NoTransforms())

transforms = NoTransforms
transforms.SyntheticPrefix = []byte{1}
require.False(t, transforms.NoTransforms())

transforms = NoTransforms
transforms.SyntheticSuffix = []byte{1}
require.False(t, transforms.NoTransforms())
}
54 changes: 49 additions & 5 deletions sstable/colblk/cockroach_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -10,6 +10,7 @@ import (
"encoding/binary"
"fmt"
"io"
"slices"
"sync"
"testing"
"time"
Expand All @@ -19,6 +20,7 @@ import (
"github.com/cockroachdb/crlib/crstrings"
"github.com/cockroachdb/pebble/internal/base"
"github.com/cockroachdb/pebble/internal/crdbtest"
"github.com/cockroachdb/pebble/internal/invariants"
"github.com/cockroachdb/pebble/sstable/block"
"github.com/stretchr/testify/require"
"golang.org/x/exp/rand"
Expand Down Expand Up @@ -198,12 +200,18 @@ func (ks *cockroachKeySeeker) Init(r *DataBlockReader) error {
// contained within the data block. It's equivalent to performing
//
// Compare(firstUserKey, k)
func (ks *cockroachKeySeeker) IsLowerBound(k []byte) bool {
func (ks *cockroachKeySeeker) IsLowerBound(k []byte, syntheticSuffix []byte) bool {
prefix, untypedSuffix, wallTime, logicalTime := crdbtest.DecodeTimestamp(k)
if v := crdbtest.Compare(ks.prefixes.UnsafeFirstSlice(), prefix); v != 0 {
return v > 0
}
if len(syntheticSuffix) > 0 {
return crdbtest.Compare(syntheticSuffix, k[len(prefix):]) >= 0
}
if len(untypedSuffix) > 0 {
if invariants.Enabled && ks.mvccWallTimes.At(0) != 0 {
panic("comparing timestamp with untyped suffix")
}
return crdbtest.Compare(ks.untypedSuffixes.At(0), untypedSuffix) >= 0
}
if v := cmp.Compare(ks.mvccWallTimes.At(0), wallTime); v != 0 {
Expand Down Expand Up @@ -250,12 +258,12 @@ func (ks *cockroachKeySeeker) seekGEOnSuffix(index int, seekSuffix []byte) (row
seekWallTime = binary.LittleEndian.Uint64(seekSuffix)
default:
// The suffix is untyped. Compare the untyped suffixes.
// Binary search between [index, prefixChanged.SeekSetBitGE(index)].
// Binary search between [index, prefixChanged.SeekSetBitGE(index+1)].
//
// Define f(l-1) == false and f(u) == true.
// Invariant: f(l-1) == false, f(u) == true.
l := index
u := ks.reader.prefixChanged.SeekSetBitGE(index)
u := ks.reader.prefixChanged.SeekSetBitGE(index + 1)
for l < u {
h := int(uint(l+u) >> 1) // avoid overflow when computing h
// l ≤ h < u
Expand Down Expand Up @@ -309,6 +317,7 @@ func (ks *cockroachKeySeeker) MaterializeUserKey(ki *PrefixBytesIter, prevRow, r
if mvccWall == 0 && mvccLogical == 0 {
// This is not an MVCC key. Use the untyped suffix.
untypedSuffixed := ks.untypedSuffixes.At(row)
// Slice first, to check that the capacity is sufficient.
res := ki.buf[:len(ki.buf)+len(untypedSuffixed)]
memmove(ptr, unsafe.Pointer(unsafe.SliceData(untypedSuffixed)), uintptr(len(untypedSuffixed)))
return res
Expand Down Expand Up @@ -341,6 +350,23 @@ func (ks *cockroachKeySeeker) MaterializeUserKey(ki *PrefixBytesIter, prevRow, r
return ki.buf[:len(ki.buf)+13]
}

// MaterializeUserKeyWithSyntheticSuffix is part of the KeySeeker interface.
func (ks *cockroachKeySeeker) MaterializeUserKeyWithSyntheticSuffix(
ki *PrefixBytesIter, suffix []byte, prevRow, row int,
) []byte {
if prevRow+1 == row && prevRow >= 0 {
ks.prefixes.SetNext(ki)
} else {
ks.prefixes.SetAt(ki, row)
}

// Slice first, to check that the capacity is sufficient.
res := ki.buf[:len(ki.buf)+len(suffix)]
ptr := unsafe.Pointer(uintptr(unsafe.Pointer(unsafe.SliceData(ki.buf))) + uintptr(len(ki.buf)))
memmove(ptr, unsafe.Pointer(unsafe.SliceData(suffix)), uintptr(len(suffix)))
return res
}

// Release is part of the KeySeeker interface.
func (ks *cockroachKeySeeker) Release() {
*ks = cockroachKeySeeker{}
Expand Down Expand Up @@ -370,7 +396,7 @@ func TestCockroachDataBlock(t *testing.T) {
serializedBlock, _ := w.Finish(w.Rows(), w.Size())
var reader DataBlockReader
var it DataBlockIter
it.InitOnce(cockroachKeySchema, getLazyValuer(func([]byte) base.LazyValue {
it.InitOnce(cockroachKeySchema, crdbtest.Compare, crdbtest.Split, getLazyValuer(func([]byte) base.LazyValue {
return base.LazyValue{ValueOrHandle: []byte("mock external value")}
}))
reader.Init(cockroachKeySchema, serializedBlock)
Expand Down Expand Up @@ -528,6 +554,18 @@ func BenchmarkCockroachDataBlockIterTransforms(b *testing.B) {
HideObsoletePoints: true,
},
},
{
description: "SyntheticPrefix",
transforms: block.IterTransforms{
SyntheticPrefix: []byte("prefix_"),
},
},
{
description: "SyntheticSuffix",
transforms: block.IterTransforms{
SyntheticSuffix: crdbtest.EncodeTimestamp(make([]byte, 0, 20), 1_000_000_000_000, 0)[1:],
},
},
}
for _, cfg := range shortBenchConfigs {
for _, t := range transforms {
Expand Down Expand Up @@ -571,7 +609,7 @@ func benchmarkCockroachDataBlockIter(
serializedBlock, _ := w.Finish(w.Rows(), w.Size())
var reader DataBlockReader
var it DataBlockIter
it.InitOnce(cockroachKeySchema, getLazyValuer(func([]byte) base.LazyValue {
it.InitOnce(cockroachKeySchema, crdbtest.Compare, crdbtest.Split, getLazyValuer(func([]byte) base.LazyValue {
return base.LazyValue{ValueOrHandle: []byte("mock external value")}
}))
reader.Init(cockroachKeySchema, serializedBlock)
Expand All @@ -580,6 +618,12 @@ func benchmarkCockroachDataBlockIter(
}
avgRowSize := float64(len(serializedBlock)) / float64(count)

if transforms.SyntheticPrefix.IsSet() {
for i := range keys {
keys[i] = slices.Concat(transforms.SyntheticPrefix, keys[i])
}
}

b.Run("Next", func(b *testing.B) {
kv := it.First()
b.ResetTimer()
Expand Down
Loading

0 comments on commit 5924e60

Please sign in to comment.